From ff1a95cb813574974ca75f3bfc37a2ea6e791765 Mon Sep 17 00:00:00 2001 From: Becker Ewing Date: Tue, 25 Nov 2025 11:54:49 -0500 Subject: [PATCH 001/201] Set data file `sort_order_id` in manifest for writes from Spark --- .../apache/iceberg/util/SortOrderUtil.java | 17 +++ .../iceberg/util/TestSortOrderUtil.java | 64 ++++++++++ .../apache/iceberg/spark/SparkWriteConf.java | 32 +++++ .../iceberg/spark/SparkWriteOptions.java | 1 + .../iceberg/spark/SparkWriteRequirements.java | 31 ++++- .../apache/iceberg/spark/SparkWriteUtil.java | 111 +++++++++++++----- .../SparkShufflingFileRewriteRunner.java | 16 +++ .../spark/source/SparkPositionDeltaWrite.java | 9 +- .../iceberg/spark/source/SparkWrite.java | 9 +- .../spark/source/SparkWriteBuilder.java | 4 +- .../iceberg/spark/TestSparkWriteConf.java | 22 ++++ .../actions/TestRewriteDataFilesAction.java | 74 ++++++++++++ .../spark/source/TestSparkDataWrite.java | 51 ++++++++ .../apache/iceberg/spark/SparkWriteConf.java | 32 +++++ .../iceberg/spark/SparkWriteOptions.java | 1 + .../iceberg/spark/SparkWriteRequirements.java | 31 ++++- .../apache/iceberg/spark/SparkWriteUtil.java | 111 +++++++++++++----- .../SparkShufflingFileRewriteRunner.java | 16 +++ .../spark/source/SparkPositionDeltaWrite.java | 9 +- .../iceberg/spark/source/SparkWrite.java | 9 +- .../spark/source/SparkWriteBuilder.java | 4 +- .../iceberg/spark/TestSparkWriteConf.java | 22 ++++ .../actions/TestRewriteDataFilesAction.java | 74 ++++++++++++ .../spark/source/TestSparkDataWrite.java | 51 ++++++++ 24 files changed, 727 insertions(+), 74 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/util/SortOrderUtil.java b/core/src/main/java/org/apache/iceberg/util/SortOrderUtil.java index 37e0c1fffab0..4d7a631ab559 100644 --- a/core/src/main/java/org/apache/iceberg/util/SortOrderUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SortOrderUtil.java @@ -46,6 +46,23 @@ public static SortOrder buildSortOrder(Table table, SortOrder sortOrder) { return buildSortOrder(table.schema(), table.spec(), sortOrder); } + /** + * Attempts to match a user-supplied {@link SortOrder} with an equivalent sort order from a {@link + * Table}. + * + * @param table the table to try and match the sort order against + * @param userSuppliedSortOrder the user supplied sort order to try and match with a table sort + * order + * @return the matching {@link SortOrder} from the table (with the orderId set) or {@link + * SortOrder#unsorted()} if no match is found. + */ + public static SortOrder maybeFindTableSortOrder(Table table, SortOrder userSuppliedSortOrder) { + return table.sortOrders().values().stream() + .filter(sortOrder -> sortOrder.sameOrder(userSuppliedSortOrder)) + .findFirst() + .orElseGet(SortOrder::unsorted); + } + /** * Build a final sort order that satisfies the clustering required by the partition spec. * diff --git a/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java b/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java index 02c81de93222..3757b70dd334 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java @@ -287,4 +287,68 @@ public void testSortOrderClusteringWithRedundantPartitionFieldsMissing() { .as("Should add spec fields as prefix") .isEqualTo(expected); } + + @Test + public void testFindSortOrderForTable() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + SortOrder order = SortOrder.builderFor(SCHEMA).withOrderId(1).asc("id", NULLS_LAST).build(); + TestTables.TestTable table = TestTables.create(tableDir, "test", SCHEMA, spec, order, 2); + + SortOrder tableSortOrder = table.sortOrder(); + + SortOrder actualOrder = SortOrderUtil.maybeFindTableSortOrder(table, tableSortOrder); + + assertThat(actualOrder).as("Should find current table sort order").isEqualTo(table.sortOrder()); + } + + @Test + public void testFindSortOrderForTableWithoutFieldId() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + SortOrder order = SortOrder.builderFor(SCHEMA).withOrderId(1).asc("id", NULLS_LAST).build(); + TestTables.TestTable table = TestTables.create(tableDir, "test", SCHEMA, spec, order, 2); + + SortOrder userSuppliedOrder = + SortOrder.builderFor(table.schema()).asc("id", NULLS_LAST).build(); + + SortOrder actualOrder = SortOrderUtil.maybeFindTableSortOrder(table, userSuppliedOrder); + + assertThat(actualOrder).as("Should find current table sort order").isEqualTo(table.sortOrder()); + } + + @Test + public void testFindSortOrderForTableThatIsNotCurrentOrder() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + SortOrder order = SortOrder.builderFor(SCHEMA).withOrderId(1).asc("id", NULLS_LAST).build(); + TestTables.TestTable table = TestTables.create(tableDir, "test", SCHEMA, spec, order, 2); + + table.replaceSortOrder().asc("data").desc("ts").commit(); + + SortOrder userSuppliedOrder = + SortOrder.builderFor(table.schema()).asc("id", NULLS_LAST).build(); + + SortOrder actualOrder = SortOrderUtil.maybeFindTableSortOrder(table, userSuppliedOrder); + + assertThat(actualOrder) + .as("Should find first sorted table sort order") + .isEqualTo(table.sortOrders().get(1)); + } + + @Test + public void testReturnsEmptyForFindingNonMatchingSortOrder() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + SortOrder order = SortOrder.builderFor(SCHEMA).withOrderId(1).asc("id", NULLS_LAST).build(); + TestTables.TestTable table = TestTables.create(tableDir, "test", SCHEMA, spec, order, 2); + + table.replaceSortOrder().asc("data").desc("ts").commit(); + + SortOrder userSuppliedOrder = + SortOrder.builderFor(table.schema()).desc("id", NULLS_LAST).build(); + + SortOrder actualOrder = SortOrderUtil.maybeFindTableSortOrder(table, userSuppliedOrder); + + assertThat(actualOrder) + .as( + "Should return unsorted order if user supplied order does not match any table sort order") + .isEqualTo(SortOrder.unsorted()); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java index b3e8af5fe056..9a750caf920a 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java @@ -42,6 +42,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.IsolationLevel; import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableUtil; @@ -162,6 +163,22 @@ public int outputSpecId() { return outputSpecId; } + public SortOrder outputSortOrder() { + int outputSortOrderId = + confParser + .intConf() + .option(SparkWriteOptions.OUTPUT_SORT_ORDER_ID) + .defaultValue(SortOrder.unsorted().orderId()) + .parse(); + + Preconditions.checkArgument( + table.sortOrders().containsKey(outputSortOrderId), + "Output sort order id %s is not a valid sort order id for table", + outputSortOrderId); + + return table.sortOrders().get(outputSortOrderId); + } + public FileFormat dataFileFormat() { String valueAsString = confParser @@ -280,6 +297,21 @@ public SparkWriteRequirements writeRequirements() { table, distributionMode(), fanoutWriterEnabled(), dataAdvisoryPartitionSize()); } + public SparkWriteRequirements rewriteFilesWriteRequirements() { + Preconditions.checkNotNull( + rewrittenFileSetId(), "Can only use rewrite files write requirements during rewrite job!"); + + SortOrder outputSortOrder = outputSortOrder(); + if (outputSortOrder.isSorted()) { + LOG.info( + "Found explicit sort order {} set in job configuration. Going to apply that to the sort-order-id of the rewritten files", + Spark3Util.describe(outputSortOrder)); + return writeRequirements().withTableSortOrder(outputSortOrder); + } + + return writeRequirements(); + } + @VisibleForTesting DistributionMode distributionMode() { String modeName = diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java index 33db70bae587..1be02feaf0c0 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java @@ -54,6 +54,7 @@ private SparkWriteOptions() {} public static final String REWRITTEN_FILE_SCAN_TASK_SET_ID = "rewritten-file-scan-task-set-id"; public static final String OUTPUT_SPEC_ID = "output-spec-id"; + public static final String OUTPUT_SORT_ORDER_ID = "output-sort-order-id"; public static final String OVERWRITE_MODE = "overwrite-mode"; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java index 833e0e44e391..dd4bc863912f 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java @@ -26,18 +26,32 @@ /** A set of requirements such as distribution and ordering reported to Spark during writes. */ public class SparkWriteRequirements { + public static final long NO_ADVISORY_PARTITION_SIZE = 0; public static final SparkWriteRequirements EMPTY = - new SparkWriteRequirements(Distributions.unspecified(), new SortOrder[0], 0); + new SparkWriteRequirements( + Distributions.unspecified(), + new SortOrder[0], + org.apache.iceberg.SortOrder.unsorted(), + NO_ADVISORY_PARTITION_SIZE); private final Distribution distribution; private final SortOrder[] ordering; + private final org.apache.iceberg.SortOrder icebergOrdering; private final long advisoryPartitionSize; SparkWriteRequirements( - Distribution distribution, SortOrder[] ordering, long advisoryPartitionSize) { + Distribution distribution, + SortOrder[] ordering, + org.apache.iceberg.SortOrder icebergOrdering, + long advisoryPartitionSize) { this.distribution = distribution; this.ordering = ordering; - this.advisoryPartitionSize = advisoryPartitionSize; + this.icebergOrdering = icebergOrdering; + // Spark prohibits requesting a particular advisory partition size without distribution + this.advisoryPartitionSize = + distribution instanceof UnspecifiedDistribution + ? NO_ADVISORY_PARTITION_SIZE + : advisoryPartitionSize; } public Distribution distribution() { @@ -48,12 +62,19 @@ public SortOrder[] ordering() { return ordering; } + public org.apache.iceberg.SortOrder icebergOrdering() { + return icebergOrdering; + } + public boolean hasOrdering() { return ordering.length != 0; } public long advisoryPartitionSize() { - // Spark prohibits requesting a particular advisory partition size without distribution - return distribution instanceof UnspecifiedDistribution ? 0 : advisoryPartitionSize; + return advisoryPartitionSize; + } + + public SparkWriteRequirements withTableSortOrder(org.apache.iceberg.SortOrder sortOrder) { + return new SparkWriteRequirements(distribution, ordering, sortOrder, advisoryPartitionSize); } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java index 0d68a0d8cdd0..535674aba977 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java @@ -55,13 +55,23 @@ public class SparkWriteUtil { private static final Expression[] PARTITION_FILE_CLUSTERING = clusterBy(SPEC_ID, PARTITION, FILE_PATH); - private static final SortOrder[] EMPTY_ORDERING = new SortOrder[0]; - private static final SortOrder[] EXISTING_ROW_ORDERING = orderBy(FILE_PATH, ROW_POSITION); - private static final SortOrder[] PARTITION_ORDERING = orderBy(SPEC_ID, PARTITION); - private static final SortOrder[] PARTITION_FILE_ORDERING = orderBy(SPEC_ID, PARTITION, FILE_PATH); - private static final SortOrder[] POSITION_DELETE_ORDERING = + private static final SortOrder[] EMPTY_SPARK_ORDERING = new SortOrder[0]; + private static final SortOrder[] EXISTING_ROW_SPARK_ORDERING = orderBy(FILE_PATH, ROW_POSITION); + private static final SortOrder[] PARTITION_SPARK_ORDERING = orderBy(SPEC_ID, PARTITION); + private static final SortOrder[] PARTITION_FILE_SPARK_ORDERING = + orderBy(SPEC_ID, PARTITION, FILE_PATH); + private static final SortOrder[] POSITION_DELETE_SPARK_ORDERING = orderBy(SPEC_ID, PARTITION, FILE_PATH, ROW_POSITION); + private static final SparkAndIcebergOrdering EXISTING_ROW_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(EXISTING_ROW_SPARK_ORDERING); + private static final SparkAndIcebergOrdering PARTITION_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(PARTITION_SPARK_ORDERING); + private static final SparkAndIcebergOrdering PARTITION_FILE_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(PARTITION_FILE_SPARK_ORDERING); + private static final SparkAndIcebergOrdering POSITION_DELETE_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(POSITION_DELETE_SPARK_ORDERING); + private SparkWriteUtil() {} /** Builds requirements for batch and micro-batch writes such as append or overwrite. */ @@ -69,8 +79,9 @@ public static SparkWriteRequirements writeRequirements( Table table, DistributionMode mode, boolean fanoutEnabled, long advisoryPartitionSize) { Distribution distribution = writeDistribution(table, mode); - SortOrder[] ordering = writeOrdering(table, fanoutEnabled); - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = writeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } private static Distribution writeDistribution(Table table, DistributionMode mode) { @@ -82,7 +93,7 @@ private static Distribution writeDistribution(Table table, DistributionMode mode return Distributions.clustered(clustering(table)); case RANGE: - return Distributions.ordered(ordering(table)); + return Distributions.ordered(ordering(table).sparkOrder()); default: throw new IllegalArgumentException("Unsupported distribution mode: " + mode); @@ -99,8 +110,9 @@ public static SparkWriteRequirements copyOnWriteRequirements( if (command == DELETE || command == UPDATE) { Distribution distribution = copyOnWriteDeleteUpdateDistribution(table, mode); - SortOrder[] ordering = writeOrdering(table, fanoutEnabled); - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = writeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } else { return writeRequirements(table, mode, fanoutEnabled, advisoryPartitionSize); } @@ -122,9 +134,9 @@ private static Distribution copyOnWriteDeleteUpdateDistribution( case RANGE: if (table.spec().isPartitioned() || table.sortOrder().isSorted()) { - return Distributions.ordered(ordering(table)); + return Distributions.ordered(ordering(table).sparkOrder()); } else { - return Distributions.ordered(EXISTING_ROW_ORDERING); + return Distributions.ordered(EXISTING_ROW_ORDERING.sparkOrder()); } default: @@ -142,12 +154,15 @@ public static SparkWriteRequirements positionDeltaRequirements( if (command == UPDATE || command == MERGE) { Distribution distribution = positionDeltaUpdateMergeDistribution(table, mode); - SortOrder[] ordering = positionDeltaUpdateMergeOrdering(table, fanoutEnabled); - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = positionDeltaUpdateMergeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } else { Distribution distribution = positionDeltaDeleteDistribution(table, mode); - SortOrder[] ordering = fanoutEnabled ? EMPTY_ORDERING : POSITION_DELETE_ORDERING; - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = + fanoutEnabled ? SparkAndIcebergOrdering.unsorted() : POSITION_DELETE_ORDERING; + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } } @@ -167,9 +182,15 @@ private static Distribution positionDeltaUpdateMergeDistribution( case RANGE: if (table.spec().isUnpartitioned()) { - return Distributions.ordered(concat(PARTITION_FILE_ORDERING, ordering(table))); + return Distributions.ordered( + SparkAndIcebergOrdering.forTable(table) + .prependOrder(PARTITION_FILE_SPARK_ORDERING) + .sparkOrder()); } else { - return Distributions.ordered(concat(PARTITION_ORDERING, ordering(table))); + return Distributions.ordered( + SparkAndIcebergOrdering.forTable(table) + .prependOrder(PARTITION_SPARK_ORDERING) + .sparkOrder()); } default: @@ -177,11 +198,12 @@ private static Distribution positionDeltaUpdateMergeDistribution( } } - private static SortOrder[] positionDeltaUpdateMergeOrdering(Table table, boolean fanoutEnabled) { + private static SparkAndIcebergOrdering positionDeltaUpdateMergeOrdering( + Table table, boolean fanoutEnabled) { if (fanoutEnabled && table.sortOrder().isUnsorted()) { - return EMPTY_ORDERING; + return SparkAndIcebergOrdering.unsorted(); } else { - return concat(POSITION_DELETE_ORDERING, ordering(table)); + return SparkAndIcebergOrdering.forTable(table).prependOrder(POSITION_DELETE_SPARK_ORDERING); } } @@ -199,9 +221,9 @@ private static Distribution positionDeltaDeleteDistribution(Table table, Distrib case RANGE: if (table.spec().isUnpartitioned()) { - return Distributions.ordered(PARTITION_FILE_ORDERING); + return Distributions.ordered(PARTITION_FILE_ORDERING.sparkOrder()); } else { - return Distributions.ordered(PARTITION_ORDERING); + return Distributions.ordered(PARTITION_ORDERING.sparkOrder()); } default: @@ -213,9 +235,9 @@ private static Distribution positionDeltaDeleteDistribution(Table table, Distrib // - there is a defined table sort order, so it is clear how the data should be ordered // - the table is partitioned and fanout writers are disabled, // so records for one partition must be co-located within a task - private static SortOrder[] writeOrdering(Table table, boolean fanoutEnabled) { + private static SparkAndIcebergOrdering writeOrdering(Table table, boolean fanoutEnabled) { if (fanoutEnabled && table.sortOrder().isUnsorted()) { - return EMPTY_ORDERING; + return SparkAndIcebergOrdering.unsorted(); } else { return ordering(table); } @@ -225,8 +247,8 @@ private static Expression[] clustering(Table table) { return Spark3Util.toTransforms(table.spec()); } - private static SortOrder[] ordering(Table table) { - return Spark3Util.toOrdering(SortOrderUtil.buildSortOrder(table)); + private static SparkAndIcebergOrdering ordering(Table table) { + return SparkAndIcebergOrdering.forTable(table); } private static Expression[] concat(Expression[] clustering, Expression... otherClustering) { @@ -256,4 +278,39 @@ private static SortOrder[] orderBy(Expression... exprs) { private static SortOrder sort(Expression expr) { return Expressions.sort(expr, SortDirection.ASCENDING); } + + private static class SparkAndIcebergOrdering { + private static final SparkAndIcebergOrdering UNSORTED = + new SparkAndIcebergOrdering(org.apache.iceberg.SortOrder.unsorted(), EMPTY_SPARK_ORDERING); + + private final org.apache.iceberg.SortOrder icebergSortOrder; + private final SortOrder[] sparkSortOrder; + + private SparkAndIcebergOrdering( + org.apache.iceberg.SortOrder icebergSortOrder, SortOrder[] sparkSortOrder) { + this.icebergSortOrder = icebergSortOrder; + this.sparkSortOrder = sparkSortOrder; + } + + public static SparkAndIcebergOrdering forTable(Table table) { + return new SparkAndIcebergOrdering( + table.sortOrder(), Spark3Util.toOrdering(SortOrderUtil.buildSortOrder(table))); + } + + public static SparkAndIcebergOrdering unsorted() { + return UNSORTED; + } + + public SparkAndIcebergOrdering prependOrder(SortOrder[] ordering) { + return new SparkAndIcebergOrdering(icebergSortOrder, concat(ordering, sparkSortOrder)); + } + + public org.apache.iceberg.SortOrder icebergOrder() { + return icebergSortOrder; + } + + public SortOrder[] sparkOrder() { + return sparkSortOrder; + } + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java index b1c5a5c0901a..a6178eff316b 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java @@ -47,10 +47,14 @@ import org.apache.spark.sql.connector.expressions.SortOrder; import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering; import org.apache.spark.sql.execution.datasources.v2.DistributionAndOrderingUtils$; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.Option; abstract class SparkShufflingFileRewriteRunner extends SparkDataFileRewriteRunner { + private static final Logger LOG = LoggerFactory.getLogger(SparkShufflingFileRewriteRunner.class); + /** * The number of shuffle partitions to use for each output file. By default, this file rewriter * assumes each shuffle partition would become a separate output file. Attempting to generate @@ -119,6 +123,17 @@ public void doRewrite(String groupId, RewriteFileGroup fileGroup) { spec(fileGroup.outputSpecId()), fileGroup.expectedOutputFiles())); + org.apache.iceberg.SortOrder sortOrderInJobSpec = sortOrder(); + + org.apache.iceberg.SortOrder maybeMatchingTableSortOrder = + SortOrderUtil.maybeFindTableSortOrder(table(), sortOrder()); + + if (sortOrderInJobSpec.isSorted() && maybeMatchingTableSortOrder.isUnsorted()) { + LOG.warn( + "Sort order specified for job {} doesn't match any table sort orders, so going to not mark rewritten files as sorted in the manifest files", + Spark3Util.describe(sortOrderInJobSpec)); + } + sortedDF .write() .format("iceberg") @@ -126,6 +141,7 @@ public void doRewrite(String groupId, RewriteFileGroup fileGroup) { .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, fileGroup.maxOutputFileSize()) .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") .option(SparkWriteOptions.OUTPUT_SPEC_ID, fileGroup.outputSpecId()) + .option(SparkWriteOptions.OUTPUT_SORT_ORDER_ID, maybeMatchingTableSortOrder.orderId()) .mode("append") .save(groupId); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index ddad1a749aa9..5d92ea2cd67d 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -180,7 +180,8 @@ public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { broadcastRewritableDeletes(), command, context, - writeProperties); + writeProperties, + writeRequirements.icebergOrdering()); } private Broadcast> broadcastRewritableDeletes() { @@ -390,18 +391,21 @@ private static class PositionDeltaWriteFactory implements DeltaWriterFactory { private final Command command; private final Context context; private final Map writeProperties; + private final org.apache.iceberg.SortOrder sortOrder; PositionDeltaWriteFactory( Broadcast tableBroadcast, Broadcast> rewritableDeletesBroadcast, Command command, Context context, - Map writeProperties) { + Map writeProperties, + org.apache.iceberg.SortOrder sortOrder) { this.tableBroadcast = tableBroadcast; this.rewritableDeletesBroadcast = rewritableDeletesBroadcast; this.command = command; this.context = context; this.writeProperties = writeProperties; + this.sortOrder = sortOrder; } @Override @@ -428,6 +432,7 @@ public DeltaWriter createWriter(int partitionId, long taskId) { .deleteFileFormat(context.deleteFileFormat()) .positionDeleteSparkType(context.deleteSparkType()) .writeProperties(writeProperties) + .dataSortOrder(sortOrder) .build(); if (command == DELETE) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index 15c70e4a6621..97360307a297 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -202,7 +202,8 @@ private WriterFactory createWriterFactory() { writeSchema, dsSchema, useFanoutWriter, - writeProperties); + writeProperties, + writeRequirements.icebergOrdering()); } private void commitOperation(SnapshotUpdate operation, String description) { @@ -672,6 +673,7 @@ private static class WriterFactory implements DataWriterFactory, StreamingDataWr private final boolean useFanoutWriter; private final String queryId; private final Map writeProperties; + private final org.apache.iceberg.SortOrder sortOrder; protected WriterFactory( Broadcast
tableBroadcast, @@ -682,7 +684,8 @@ protected WriterFactory( Schema writeSchema, StructType dsSchema, boolean useFanoutWriter, - Map writeProperties) { + Map writeProperties, + org.apache.iceberg.SortOrder sortOrder) { this.tableBroadcast = tableBroadcast; this.format = format; this.outputSpecId = outputSpecId; @@ -692,6 +695,7 @@ protected WriterFactory( this.useFanoutWriter = useFanoutWriter; this.queryId = queryId; this.writeProperties = writeProperties; + this.sortOrder = sortOrder; } @Override @@ -716,6 +720,7 @@ public DataWriter createWriter(int partitionId, long taskId, long e .dataSchema(writeSchema) .dataSparkType(dsSchema) .writeProperties(writeProperties) + .dataSortOrder(sortOrder) .build(); if (spec.isUnpartitioned()) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java index e608a40b72ad..4b76904e7c3e 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java @@ -182,7 +182,9 @@ public StreamingWrite toStreaming() { } private SparkWriteRequirements writeRequirements() { - if (overwriteFiles) { + if (rewrittenFileSetId != null) { + return writeConf.rewriteFilesWriteRequirements(); + } else if (overwriteFiles) { return writeConf.copyOnWriteRequirements(copyOnWriteCommand); } else { return writeConf.writeRequirements(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java index a9b5d1a237b4..add19571ca44 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java @@ -45,6 +45,7 @@ import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.MERGE; import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.UPDATE; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.time.Duration; @@ -552,6 +553,27 @@ public void testDVWriteConf() { assertThat(writeConf.deleteFileFormat()).isEqualTo(FileFormat.PUFFIN); } + @TestTemplate + public void testSortOrderWriteConf() { + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").commit(); + + SparkWriteConf writeConf = + new SparkWriteConf( + spark, table, ImmutableMap.of(SparkWriteOptions.OUTPUT_SORT_ORDER_ID, "1")); + + assertThat(writeConf.outputSortOrder()).isEqualTo(table.sortOrder()); + + SparkWriteConf writeConfForUnknownSortOrder = + new SparkWriteConf( + spark, table, ImmutableMap.of(SparkWriteOptions.OUTPUT_SORT_ORDER_ID, "999")); + + assertThatIllegalArgumentException() + .isThrownBy(writeConfForUnknownSortOrder::outputSortOrder) + .withMessage("Output sort order id 999 is not a valid sort order id for table"); + } + private void testWriteProperties(List> propertiesSuite) { withSQLConf( propertiesSuite.get(0), diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 2b4a2a211ec0..1d55f15c7878 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -1584,6 +1584,7 @@ public void testSimpleSort() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + dataFilesSortOrderShouldMatchTableSortOrder(table); } @TestTemplate @@ -1620,6 +1621,7 @@ public void testSortAfterPartitionChange() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + dataFilesSortOrderShouldMatchTableSortOrder(table); } @TestTemplate @@ -1651,6 +1653,9 @@ public void testSortCustomSortOrder() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + // Since the SortOrder isn't in the table spec, these data files should report having the + // default table sort order e.g. unsorted + dataFilesShouldHaveSortOrderIdMatching(table, SortOrder.unsorted()); } @TestTemplate @@ -1691,6 +1696,58 @@ public void testSortCustomSortOrderRequiresRepartition() { shouldHaveMultipleFiles(table); shouldHaveLastCommitUnsorted(table, "c2"); shouldHaveLastCommitSorted(table, "c3"); + // Since the table ordering is on C2, but we rewrote explicitly with C3 which isn't a sort order + // spec, the table files should report unsorted + dataFilesShouldHaveSortOrderIdMatching(table, SortOrder.unsorted()); + } + + @TestTemplate + public void testSortPastTableSortOrderGetsAppliedToFiles() { + int partitions = 4; + Table table = createTable(); + writeRecords(20, SCALE, partitions); + shouldHaveLastCommitUnsorted(table, "c3"); + + // Add a partition column so this requires repartitioning + table.updateSpec().addField("c1").commit(); + + // Add the sort order we want to use during the rewrite job + table.replaceSortOrder().asc("c3").commit(); + SortOrder c3SortOrder = table.sortOrder(); + + // Replace that sort order with a newer one that we aren't going to use, but is the current + // table ordering + table.replaceSortOrder().asc("c2").commit(); + shouldHaveFiles(table, 20); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + RewriteDataFiles.Result result = + basicRewrite(table) + .sort(SortOrder.builderFor(table.schema()).asc("c3").build()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + RewriteDataFiles.TARGET_FILE_SIZE_BYTES, + Integer.toString(averageFileSize(table) / partitions)) + .execute(); + + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + shouldHaveMultipleFiles(table); + shouldHaveLastCommitUnsorted(table, "c2"); + shouldHaveLastCommitSorted(table, "c3"); + // Since the table ordering is on C2, but we rewrote explicitly with C3 which is in the table + // sort order spec, the table files should report C3 sort order + dataFilesShouldHaveSortOrderIdMatching(table, c3SortOrder); } @TestTemplate @@ -1731,6 +1788,9 @@ public void testAutoSortShuffleOutput() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + // Since the sort order being applied here isn't anywhere on the table spec, all files despite + // being physically sorted should report unsorted in the manifest entry + dataFilesShouldHaveSortOrderIdMatching(table, SortOrder.unsorted()); } @TestTemplate @@ -2634,4 +2694,18 @@ public boolean matches(RewriteFileGroup argument) { return groupIDs.contains(argument.info().globalIndex()); } } + + private void dataFilesSortOrderShouldMatchTableSortOrder(Table table) { + dataFilesShouldHaveSortOrderIdMatching(table, table.sortOrder()); + } + + private void dataFilesShouldHaveSortOrderIdMatching(Table table, SortOrder sortOrder) { + try (CloseableIterable files = table.newScan().planFiles()) { + assertThat(files) + .extracting(fileScanTask -> fileScanTask.file().sortOrderId()) + .containsOnly(sortOrder.orderId()); + } catch (IOException e) { + throw new RuntimeException("Failed to close file scan tasks", e); + } + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index 4ccbf86f1257..c74236992bdd 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -44,6 +44,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.CommitStateUnknownException; @@ -154,6 +155,7 @@ public void testBasicWrite() { assertThat(file.splitOffsets()).as("Split offsets not present").isNotNull(); } assertThat(file.recordCount()).as("Should have reported record count as 1").isEqualTo(1); + assertThat(file.sortOrderId()).isEqualTo(SortOrder.unsorted().orderId()); // TODO: append more metric info if (format.equals(FileFormat.PARQUET)) { assertThat(file.columnSizes()).as("Column sizes metric not present").isNotNull(); @@ -555,6 +557,55 @@ public void testViewsReturnRecentResults() { assertThat(actual2).hasSameSizeAs(expected2).isEqualTo(expected2); } + @TestTemplate + public void testWriteDataFilesInTableSortOrder() { + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + table.replaceSortOrder().asc("id").commit(); + + List expected = Lists.newArrayListWithCapacity(4000); + for (int i = 0; i < 4000; i++) { + expected.add(new SimpleRecord(i, "a")); + } + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + createBranch(table); + table.refresh(); + + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + + List files = Lists.newArrayList(); + for (ManifestFile manifest : + SnapshotUtil.latestSnapshot(table, branch).allManifests(table.io())) { + for (DataFile file : ManifestFiles.read(manifest, table.io())) { + files.add(file); + } + } + + assertThat(files) + .extracting(DataFile::sortOrderId) + .as("All DataFiles are written with the table sort order id") + .containsOnly(table.sortOrder().orderId()); + } + public void partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType option) { File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test"); diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java index 96131e0e56dd..f85fb0dfb9ff 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java @@ -42,6 +42,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.IsolationLevel; import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableUtil; @@ -164,6 +165,22 @@ public int outputSpecId() { return outputSpecId; } + public SortOrder outputSortOrder() { + int outputSortOrderId = + confParser + .intConf() + .option(SparkWriteOptions.OUTPUT_SORT_ORDER_ID) + .defaultValue(SortOrder.unsorted().orderId()) + .parse(); + + Preconditions.checkArgument( + table.sortOrders().containsKey(outputSortOrderId), + "Output sort order id %s is not a valid sort order id for table", + outputSortOrderId); + + return table.sortOrders().get(outputSortOrderId); + } + public FileFormat dataFileFormat() { String valueAsString = confParser @@ -284,6 +301,21 @@ public SparkWriteRequirements writeRequirements() { table, distributionMode(), fanoutWriterEnabled(), dataAdvisoryPartitionSize()); } + public SparkWriteRequirements rewriteFilesWriteRequirements() { + Preconditions.checkNotNull( + rewrittenFileSetId(), "Can only use rewrite files write requirements during rewrite job!"); + + SortOrder outputSortOrder = outputSortOrder(); + if (outputSortOrder.isSorted()) { + LOG.info( + "Found explicit sort order {} set in job configuration. Going to apply that to the sort-order-id of the rewritten files", + Spark3Util.describe(outputSortOrder)); + return writeRequirements().withTableSortOrder(outputSortOrder); + } + + return writeRequirements(); + } + @VisibleForTesting DistributionMode distributionMode() { String modeName = diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java index 33db70bae587..1be02feaf0c0 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java @@ -54,6 +54,7 @@ private SparkWriteOptions() {} public static final String REWRITTEN_FILE_SCAN_TASK_SET_ID = "rewritten-file-scan-task-set-id"; public static final String OUTPUT_SPEC_ID = "output-spec-id"; + public static final String OUTPUT_SORT_ORDER_ID = "output-sort-order-id"; public static final String OVERWRITE_MODE = "overwrite-mode"; diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java index 833e0e44e391..dd4bc863912f 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java @@ -26,18 +26,32 @@ /** A set of requirements such as distribution and ordering reported to Spark during writes. */ public class SparkWriteRequirements { + public static final long NO_ADVISORY_PARTITION_SIZE = 0; public static final SparkWriteRequirements EMPTY = - new SparkWriteRequirements(Distributions.unspecified(), new SortOrder[0], 0); + new SparkWriteRequirements( + Distributions.unspecified(), + new SortOrder[0], + org.apache.iceberg.SortOrder.unsorted(), + NO_ADVISORY_PARTITION_SIZE); private final Distribution distribution; private final SortOrder[] ordering; + private final org.apache.iceberg.SortOrder icebergOrdering; private final long advisoryPartitionSize; SparkWriteRequirements( - Distribution distribution, SortOrder[] ordering, long advisoryPartitionSize) { + Distribution distribution, + SortOrder[] ordering, + org.apache.iceberg.SortOrder icebergOrdering, + long advisoryPartitionSize) { this.distribution = distribution; this.ordering = ordering; - this.advisoryPartitionSize = advisoryPartitionSize; + this.icebergOrdering = icebergOrdering; + // Spark prohibits requesting a particular advisory partition size without distribution + this.advisoryPartitionSize = + distribution instanceof UnspecifiedDistribution + ? NO_ADVISORY_PARTITION_SIZE + : advisoryPartitionSize; } public Distribution distribution() { @@ -48,12 +62,19 @@ public SortOrder[] ordering() { return ordering; } + public org.apache.iceberg.SortOrder icebergOrdering() { + return icebergOrdering; + } + public boolean hasOrdering() { return ordering.length != 0; } public long advisoryPartitionSize() { - // Spark prohibits requesting a particular advisory partition size without distribution - return distribution instanceof UnspecifiedDistribution ? 0 : advisoryPartitionSize; + return advisoryPartitionSize; + } + + public SparkWriteRequirements withTableSortOrder(org.apache.iceberg.SortOrder sortOrder) { + return new SparkWriteRequirements(distribution, ordering, sortOrder, advisoryPartitionSize); } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java index 0d68a0d8cdd0..535674aba977 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java @@ -55,13 +55,23 @@ public class SparkWriteUtil { private static final Expression[] PARTITION_FILE_CLUSTERING = clusterBy(SPEC_ID, PARTITION, FILE_PATH); - private static final SortOrder[] EMPTY_ORDERING = new SortOrder[0]; - private static final SortOrder[] EXISTING_ROW_ORDERING = orderBy(FILE_PATH, ROW_POSITION); - private static final SortOrder[] PARTITION_ORDERING = orderBy(SPEC_ID, PARTITION); - private static final SortOrder[] PARTITION_FILE_ORDERING = orderBy(SPEC_ID, PARTITION, FILE_PATH); - private static final SortOrder[] POSITION_DELETE_ORDERING = + private static final SortOrder[] EMPTY_SPARK_ORDERING = new SortOrder[0]; + private static final SortOrder[] EXISTING_ROW_SPARK_ORDERING = orderBy(FILE_PATH, ROW_POSITION); + private static final SortOrder[] PARTITION_SPARK_ORDERING = orderBy(SPEC_ID, PARTITION); + private static final SortOrder[] PARTITION_FILE_SPARK_ORDERING = + orderBy(SPEC_ID, PARTITION, FILE_PATH); + private static final SortOrder[] POSITION_DELETE_SPARK_ORDERING = orderBy(SPEC_ID, PARTITION, FILE_PATH, ROW_POSITION); + private static final SparkAndIcebergOrdering EXISTING_ROW_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(EXISTING_ROW_SPARK_ORDERING); + private static final SparkAndIcebergOrdering PARTITION_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(PARTITION_SPARK_ORDERING); + private static final SparkAndIcebergOrdering PARTITION_FILE_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(PARTITION_FILE_SPARK_ORDERING); + private static final SparkAndIcebergOrdering POSITION_DELETE_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(POSITION_DELETE_SPARK_ORDERING); + private SparkWriteUtil() {} /** Builds requirements for batch and micro-batch writes such as append or overwrite. */ @@ -69,8 +79,9 @@ public static SparkWriteRequirements writeRequirements( Table table, DistributionMode mode, boolean fanoutEnabled, long advisoryPartitionSize) { Distribution distribution = writeDistribution(table, mode); - SortOrder[] ordering = writeOrdering(table, fanoutEnabled); - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = writeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } private static Distribution writeDistribution(Table table, DistributionMode mode) { @@ -82,7 +93,7 @@ private static Distribution writeDistribution(Table table, DistributionMode mode return Distributions.clustered(clustering(table)); case RANGE: - return Distributions.ordered(ordering(table)); + return Distributions.ordered(ordering(table).sparkOrder()); default: throw new IllegalArgumentException("Unsupported distribution mode: " + mode); @@ -99,8 +110,9 @@ public static SparkWriteRequirements copyOnWriteRequirements( if (command == DELETE || command == UPDATE) { Distribution distribution = copyOnWriteDeleteUpdateDistribution(table, mode); - SortOrder[] ordering = writeOrdering(table, fanoutEnabled); - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = writeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } else { return writeRequirements(table, mode, fanoutEnabled, advisoryPartitionSize); } @@ -122,9 +134,9 @@ private static Distribution copyOnWriteDeleteUpdateDistribution( case RANGE: if (table.spec().isPartitioned() || table.sortOrder().isSorted()) { - return Distributions.ordered(ordering(table)); + return Distributions.ordered(ordering(table).sparkOrder()); } else { - return Distributions.ordered(EXISTING_ROW_ORDERING); + return Distributions.ordered(EXISTING_ROW_ORDERING.sparkOrder()); } default: @@ -142,12 +154,15 @@ public static SparkWriteRequirements positionDeltaRequirements( if (command == UPDATE || command == MERGE) { Distribution distribution = positionDeltaUpdateMergeDistribution(table, mode); - SortOrder[] ordering = positionDeltaUpdateMergeOrdering(table, fanoutEnabled); - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = positionDeltaUpdateMergeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } else { Distribution distribution = positionDeltaDeleteDistribution(table, mode); - SortOrder[] ordering = fanoutEnabled ? EMPTY_ORDERING : POSITION_DELETE_ORDERING; - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = + fanoutEnabled ? SparkAndIcebergOrdering.unsorted() : POSITION_DELETE_ORDERING; + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } } @@ -167,9 +182,15 @@ private static Distribution positionDeltaUpdateMergeDistribution( case RANGE: if (table.spec().isUnpartitioned()) { - return Distributions.ordered(concat(PARTITION_FILE_ORDERING, ordering(table))); + return Distributions.ordered( + SparkAndIcebergOrdering.forTable(table) + .prependOrder(PARTITION_FILE_SPARK_ORDERING) + .sparkOrder()); } else { - return Distributions.ordered(concat(PARTITION_ORDERING, ordering(table))); + return Distributions.ordered( + SparkAndIcebergOrdering.forTable(table) + .prependOrder(PARTITION_SPARK_ORDERING) + .sparkOrder()); } default: @@ -177,11 +198,12 @@ private static Distribution positionDeltaUpdateMergeDistribution( } } - private static SortOrder[] positionDeltaUpdateMergeOrdering(Table table, boolean fanoutEnabled) { + private static SparkAndIcebergOrdering positionDeltaUpdateMergeOrdering( + Table table, boolean fanoutEnabled) { if (fanoutEnabled && table.sortOrder().isUnsorted()) { - return EMPTY_ORDERING; + return SparkAndIcebergOrdering.unsorted(); } else { - return concat(POSITION_DELETE_ORDERING, ordering(table)); + return SparkAndIcebergOrdering.forTable(table).prependOrder(POSITION_DELETE_SPARK_ORDERING); } } @@ -199,9 +221,9 @@ private static Distribution positionDeltaDeleteDistribution(Table table, Distrib case RANGE: if (table.spec().isUnpartitioned()) { - return Distributions.ordered(PARTITION_FILE_ORDERING); + return Distributions.ordered(PARTITION_FILE_ORDERING.sparkOrder()); } else { - return Distributions.ordered(PARTITION_ORDERING); + return Distributions.ordered(PARTITION_ORDERING.sparkOrder()); } default: @@ -213,9 +235,9 @@ private static Distribution positionDeltaDeleteDistribution(Table table, Distrib // - there is a defined table sort order, so it is clear how the data should be ordered // - the table is partitioned and fanout writers are disabled, // so records for one partition must be co-located within a task - private static SortOrder[] writeOrdering(Table table, boolean fanoutEnabled) { + private static SparkAndIcebergOrdering writeOrdering(Table table, boolean fanoutEnabled) { if (fanoutEnabled && table.sortOrder().isUnsorted()) { - return EMPTY_ORDERING; + return SparkAndIcebergOrdering.unsorted(); } else { return ordering(table); } @@ -225,8 +247,8 @@ private static Expression[] clustering(Table table) { return Spark3Util.toTransforms(table.spec()); } - private static SortOrder[] ordering(Table table) { - return Spark3Util.toOrdering(SortOrderUtil.buildSortOrder(table)); + private static SparkAndIcebergOrdering ordering(Table table) { + return SparkAndIcebergOrdering.forTable(table); } private static Expression[] concat(Expression[] clustering, Expression... otherClustering) { @@ -256,4 +278,39 @@ private static SortOrder[] orderBy(Expression... exprs) { private static SortOrder sort(Expression expr) { return Expressions.sort(expr, SortDirection.ASCENDING); } + + private static class SparkAndIcebergOrdering { + private static final SparkAndIcebergOrdering UNSORTED = + new SparkAndIcebergOrdering(org.apache.iceberg.SortOrder.unsorted(), EMPTY_SPARK_ORDERING); + + private final org.apache.iceberg.SortOrder icebergSortOrder; + private final SortOrder[] sparkSortOrder; + + private SparkAndIcebergOrdering( + org.apache.iceberg.SortOrder icebergSortOrder, SortOrder[] sparkSortOrder) { + this.icebergSortOrder = icebergSortOrder; + this.sparkSortOrder = sparkSortOrder; + } + + public static SparkAndIcebergOrdering forTable(Table table) { + return new SparkAndIcebergOrdering( + table.sortOrder(), Spark3Util.toOrdering(SortOrderUtil.buildSortOrder(table))); + } + + public static SparkAndIcebergOrdering unsorted() { + return UNSORTED; + } + + public SparkAndIcebergOrdering prependOrder(SortOrder[] ordering) { + return new SparkAndIcebergOrdering(icebergSortOrder, concat(ordering, sparkSortOrder)); + } + + public org.apache.iceberg.SortOrder icebergOrder() { + return icebergSortOrder; + } + + public SortOrder[] sparkOrder() { + return sparkSortOrder; + } + } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java index 569eb252cba5..1ba4c7e2fac2 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java @@ -47,10 +47,14 @@ import org.apache.spark.sql.connector.expressions.SortOrder; import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering; import org.apache.spark.sql.execution.datasources.v2.DistributionAndOrderingUtils$; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.Option; abstract class SparkShufflingFileRewriteRunner extends SparkDataFileRewriteRunner { + private static final Logger LOG = LoggerFactory.getLogger(SparkShufflingFileRewriteRunner.class); + /** * The number of shuffle partitions to use for each output file. By default, this file rewriter * assumes each shuffle partition would become a separate output file. Attempting to generate @@ -119,6 +123,17 @@ public void doRewrite(String groupId, RewriteFileGroup fileGroup) { spec(fileGroup.outputSpecId()), fileGroup.expectedOutputFiles())); + org.apache.iceberg.SortOrder sortOrderInJobSpec = sortOrder(); + + org.apache.iceberg.SortOrder maybeMatchingTableSortOrder = + SortOrderUtil.maybeFindTableSortOrder(table(), sortOrder()); + + if (sortOrderInJobSpec.isSorted() && maybeMatchingTableSortOrder.isUnsorted()) { + LOG.warn( + "Sort order specified for job {} doesn't match any table sort orders, so going to not mark rewritten files as sorted in the manifest files", + Spark3Util.describe(sortOrderInJobSpec)); + } + sortedDF .write() .format("iceberg") @@ -126,6 +141,7 @@ public void doRewrite(String groupId, RewriteFileGroup fileGroup) { .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, fileGroup.maxOutputFileSize()) .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") .option(SparkWriteOptions.OUTPUT_SPEC_ID, fileGroup.outputSpecId()) + .option(SparkWriteOptions.OUTPUT_SORT_ORDER_ID, maybeMatchingTableSortOrder.orderId()) .mode("append") .save(groupId); } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index d072397dc6a3..e0c842e9a6d7 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -182,7 +182,8 @@ public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { broadcastRewritableDeletes(), command, context, - writeProperties); + writeProperties, + writeRequirements.icebergOrdering()); } private Broadcast> broadcastRewritableDeletes() { @@ -392,18 +393,21 @@ private static class PositionDeltaWriteFactory implements DeltaWriterFactory { private final Command command; private final Context context; private final Map writeProperties; + private final org.apache.iceberg.SortOrder sortOrder; PositionDeltaWriteFactory( Broadcast
tableBroadcast, Broadcast> rewritableDeletesBroadcast, Command command, Context context, - Map writeProperties) { + Map writeProperties, + org.apache.iceberg.SortOrder sortOrder) { this.tableBroadcast = tableBroadcast; this.rewritableDeletesBroadcast = rewritableDeletesBroadcast; this.command = command; this.context = context; this.writeProperties = writeProperties; + this.sortOrder = sortOrder; } @Override @@ -430,6 +434,7 @@ public DeltaWriter createWriter(int partitionId, long taskId) { .deleteFileFormat(context.deleteFileFormat()) .positionDeleteSparkType(context.deleteSparkType()) .writeProperties(writeProperties) + .dataSortOrder(sortOrder) .build(); if (command == DELETE) { diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index c9a94090ef89..521cb8a2287f 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -205,7 +205,8 @@ private WriterFactory createWriterFactory() { writeSchema, dsSchema, useFanoutWriter, - writeProperties); + writeProperties, + writeRequirements.icebergOrdering()); } private void commitOperation(SnapshotUpdate operation, String description) { @@ -675,6 +676,7 @@ private static class WriterFactory implements DataWriterFactory, StreamingDataWr private final boolean useFanoutWriter; private final String queryId; private final Map writeProperties; + private final org.apache.iceberg.SortOrder sortOrder; protected WriterFactory( Broadcast
tableBroadcast, @@ -685,7 +687,8 @@ protected WriterFactory( Schema writeSchema, StructType dsSchema, boolean useFanoutWriter, - Map writeProperties) { + Map writeProperties, + org.apache.iceberg.SortOrder sortOrder) { this.tableBroadcast = tableBroadcast; this.format = format; this.outputSpecId = outputSpecId; @@ -695,6 +698,7 @@ protected WriterFactory( this.useFanoutWriter = useFanoutWriter; this.queryId = queryId; this.writeProperties = writeProperties; + this.sortOrder = sortOrder; } @Override @@ -719,6 +723,7 @@ public DataWriter createWriter(int partitionId, long taskId, long e .dataSchema(writeSchema) .dataSparkType(dsSchema) .writeProperties(writeProperties) + .dataSortOrder(sortOrder) .build(); Function rowLineageExtractor = new ExtractRowLineage(writeSchema); diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java index 89af7740d988..70230a91fc28 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java @@ -190,7 +190,9 @@ public StreamingWrite toStreaming() { } private SparkWriteRequirements writeRequirements() { - if (overwriteFiles) { + if (rewrittenFileSetId != null) { + return writeConf.rewriteFilesWriteRequirements(); + } else if (overwriteFiles) { return writeConf.copyOnWriteRequirements(copyOnWriteCommand); } else { return writeConf.writeRequirements(); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java index 61aacfa4589d..247119523756 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java @@ -45,6 +45,7 @@ import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.MERGE; import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.UPDATE; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.time.Duration; @@ -600,6 +601,27 @@ public void testDVWriteConf() { assertThat(writeConf.deleteFileFormat()).isEqualTo(FileFormat.PUFFIN); } + @TestTemplate + public void testSortOrderWriteConf() { + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").commit(); + + SparkWriteConf writeConf = + new SparkWriteConf( + spark, table, ImmutableMap.of(SparkWriteOptions.OUTPUT_SORT_ORDER_ID, "1")); + + assertThat(writeConf.outputSortOrder()).isEqualTo(table.sortOrder()); + + SparkWriteConf writeConfForUnknownSortOrder = + new SparkWriteConf( + spark, table, ImmutableMap.of(SparkWriteOptions.OUTPUT_SORT_ORDER_ID, "999")); + + assertThatIllegalArgumentException() + .isThrownBy(writeConfForUnknownSortOrder::outputSortOrder) + .withMessage("Output sort order id 999 is not a valid sort order id for table"); + } + private void testWriteProperties(List> propertiesSuite) { withSQLConf( propertiesSuite.get(0), diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 6d965f3dcc62..5d8a6da51c4e 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -1587,6 +1587,7 @@ public void testSimpleSort() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + dataFilesSortOrderShouldMatchTableSortOrder(table); } @TestTemplate @@ -1623,6 +1624,7 @@ public void testSortAfterPartitionChange() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + dataFilesSortOrderShouldMatchTableSortOrder(table); } @TestTemplate @@ -1654,6 +1656,9 @@ public void testSortCustomSortOrder() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + // Since the SortOrder isn't in the table spec, these data files should report having the + // default table sort order e.g. unsorted + dataFilesShouldHaveSortOrderIdMatching(table, SortOrder.unsorted()); } @TestTemplate @@ -1694,6 +1699,58 @@ public void testSortCustomSortOrderRequiresRepartition() { shouldHaveMultipleFiles(table); shouldHaveLastCommitUnsorted(table, "c2"); shouldHaveLastCommitSorted(table, "c3"); + // Since the table ordering is on C2, but we rewrote explicitly with C3 which isn't a sort order + // spec, the table files should report unsorted + dataFilesShouldHaveSortOrderIdMatching(table, SortOrder.unsorted()); + } + + @TestTemplate + public void testSortPastTableSortOrderGetsAppliedToFiles() { + int partitions = 4; + Table table = createTable(); + writeRecords(20, SCALE, partitions); + shouldHaveLastCommitUnsorted(table, "c3"); + + // Add a partition column so this requires repartitioning + table.updateSpec().addField("c1").commit(); + + // Add the sort order we want to use during the rewrite job + table.replaceSortOrder().asc("c3").commit(); + SortOrder c3SortOrder = table.sortOrder(); + + // Replace that sort order with a newer one that we aren't going to use, but is the current + // table ordering + table.replaceSortOrder().asc("c2").commit(); + shouldHaveFiles(table, 20); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + RewriteDataFiles.Result result = + basicRewrite(table) + .sort(SortOrder.builderFor(table.schema()).asc("c3").build()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + RewriteDataFiles.TARGET_FILE_SIZE_BYTES, + Integer.toString(averageFileSize(table) / partitions)) + .execute(); + + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + shouldHaveMultipleFiles(table); + shouldHaveLastCommitUnsorted(table, "c2"); + shouldHaveLastCommitSorted(table, "c3"); + // Since the table ordering is on C2, but we rewrote explicitly with C3 which is in the table + // sort order spec, the table files should report C3 sort order + dataFilesShouldHaveSortOrderIdMatching(table, c3SortOrder); } @TestTemplate @@ -1734,6 +1791,9 @@ public void testAutoSortShuffleOutput() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + // Since the sort order being applied here isn't anywhere on the table spec, all files despite + // being physically sorted should report unsorted in the manifest entry + dataFilesShouldHaveSortOrderIdMatching(table, SortOrder.unsorted()); } @TestTemplate @@ -2657,4 +2717,18 @@ public boolean matches(RewriteFileGroup argument) { return groupIDs.contains(argument.info().globalIndex()); } } + + private void dataFilesSortOrderShouldMatchTableSortOrder(Table table) { + dataFilesShouldHaveSortOrderIdMatching(table, table.sortOrder()); + } + + private void dataFilesShouldHaveSortOrderIdMatching(Table table, SortOrder sortOrder) { + try (CloseableIterable files = table.newScan().planFiles()) { + assertThat(files) + .extracting(fileScanTask -> fileScanTask.file().sortOrderId()) + .containsOnly(sortOrder.orderId()); + } catch (IOException e) { + throw new RuntimeException("Failed to close file scan tasks", e); + } + } } diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index 4ccbf86f1257..c74236992bdd 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -44,6 +44,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.CommitStateUnknownException; @@ -154,6 +155,7 @@ public void testBasicWrite() { assertThat(file.splitOffsets()).as("Split offsets not present").isNotNull(); } assertThat(file.recordCount()).as("Should have reported record count as 1").isEqualTo(1); + assertThat(file.sortOrderId()).isEqualTo(SortOrder.unsorted().orderId()); // TODO: append more metric info if (format.equals(FileFormat.PARQUET)) { assertThat(file.columnSizes()).as("Column sizes metric not present").isNotNull(); @@ -555,6 +557,55 @@ public void testViewsReturnRecentResults() { assertThat(actual2).hasSameSizeAs(expected2).isEqualTo(expected2); } + @TestTemplate + public void testWriteDataFilesInTableSortOrder() { + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + table.replaceSortOrder().asc("id").commit(); + + List expected = Lists.newArrayListWithCapacity(4000); + for (int i = 0; i < 4000; i++) { + expected.add(new SimpleRecord(i, "a")); + } + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + createBranch(table); + table.refresh(); + + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + + List files = Lists.newArrayList(); + for (ManifestFile manifest : + SnapshotUtil.latestSnapshot(table, branch).allManifests(table.io())) { + for (DataFile file : ManifestFiles.read(manifest, table.io())) { + files.add(file); + } + } + + assertThat(files) + .extracting(DataFile::sortOrderId) + .as("All DataFiles are written with the table sort order id") + .containsOnly(table.sortOrder().orderId()); + } + public void partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType option) { File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test"); From 47d5f5009eafbbb526e6e2c9cbeac3105bf34670 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Wed, 26 Nov 2025 03:26:49 +0900 Subject: [PATCH 002/201] Docs: Add gc.enabled table property (#14676) --------- Co-authored-by: nk1506 Co-authored-by: Kevin Liu --- docs/docs/configuration.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/docs/configuration.md b/docs/docs/configuration.md index b97608e10985..161fe8b6e78a 100644 --- a/docs/docs/configuration.md +++ b/docs/docs/configuration.md @@ -108,6 +108,7 @@ Iceberg tables support table properties to configure table behavior, like the de | history.expire.max-snapshot-age-ms | 432000000 (5 days) | Default max age of snapshots to keep on the table and all of its branches while expiring snapshots | | history.expire.min-snapshots-to-keep | 1 | Default min number of snapshots to keep on the table and all of its branches while expiring snapshots | | history.expire.max-ref-age-ms | `Long.MAX_VALUE` (forever) | For snapshot references except the `main` branch, default max age of snapshot references to keep while expiring snapshots. The `main` branch never expires. | +| gc.enabled | true | Allows garbage collection operations such as expiring snapshots and removing orphan files | ### Reserved table properties Reserved table properties are only used to control behaviors when creating or updating a table. From 6a54bc1c379d5af78a2dc746fa66c5f0706d0838 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 25 Nov 2025 15:00:13 -0800 Subject: [PATCH 003/201] Core: Add idempotency-key-lifetime to ConfigResponse (#14649) --- .../rest/responses/ConfigResponse.java | 31 ++++++++++++- .../rest/responses/ConfigResponseParser.java | 10 +++++ .../org/apache/iceberg/util/JsonUtil.java | 18 ++++++++ .../responses/TestConfigResponseParser.java | 43 +++++++++++++++++++ .../org/apache/iceberg/util/TestJsonUtil.java | 23 ++++++++++ 5 files changed, 123 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponse.java index da22ca287b30..3acf1b8c3b4c 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponse.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponse.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import javax.annotation.Nullable; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -51,16 +52,22 @@ public class ConfigResponse implements RESTResponse { private Map defaults; private Map overrides; private List endpoints; + // Optional ISO-8601 duration string indicating server support for idempotency keys + private String idempotencyKeyLifetime; public ConfigResponse() { // Required for Jackson deserialization } private ConfigResponse( - Map defaults, Map overrides, List endpoints) { + Map defaults, + Map overrides, + List endpoints, + String idempotencyKeyLifetime) { this.defaults = defaults; this.overrides = overrides; this.endpoints = endpoints; + this.idempotencyKeyLifetime = idempotencyKeyLifetime; validate(); } @@ -97,6 +104,17 @@ public List endpoints() { return null != endpoints ? endpoints : ImmutableList.of(); } + /** + * Optional server-advertised reuse window for idempotency keys. Presence indicates that the + * server supports Idempotency-Key semantics on mutation endpoints. + * + * @return ISO-8601 duration string (e.g., PT30M) or null if not supported/advertised + */ + @Nullable + public String idempotencyKeyLifetime() { + return idempotencyKeyLifetime; + } + /** * Merge client-provided config with server side provided configuration to return a single * properties map which will be used for instantiating and configuring the REST catalog. @@ -125,6 +143,7 @@ public String toString() { .add("defaults", defaults) .add("overrides", overrides) .add("endpoints", endpoints) + .add("idempotencyKeyLifetime", idempotencyKeyLifetime) .toString(); } @@ -136,11 +155,13 @@ public static class Builder { private final Map defaults; private final Map overrides; private final List endpoints; + private String idempotencyKeyLifetime; private Builder() { this.defaults = Maps.newHashMap(); this.overrides = Maps.newHashMap(); this.endpoints = Lists.newArrayList(); + this.idempotencyKeyLifetime = null; } public Builder withDefault(String key, String value) { @@ -178,8 +199,14 @@ public Builder withEndpoints(List endpointsToAdd) { return this; } + /** Sets the optional idempotency key lifetime advertised by the server. */ + public Builder withIdempotencyKeyLifetime(String lifetime) { + this.idempotencyKeyLifetime = lifetime; + return this; + } + public ConfigResponse build() { - return new ConfigResponse(defaults, overrides, endpoints); + return new ConfigResponse(defaults, overrides, endpoints, idempotencyKeyLifetime); } } } diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java index acadcce6d4bf..c2927659b38f 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java @@ -31,6 +31,7 @@ public class ConfigResponseParser { private static final String DEFAULTS = "defaults"; private static final String OVERRIDES = "overrides"; private static final String ENDPOINTS = "endpoints"; + private static final String IDEMPOTENCY_KEY_LIFETIME = "idempotency-key-lifetime"; private ConfigResponseParser() {} @@ -56,6 +57,10 @@ public static void toJson(ConfigResponse response, JsonGenerator gen) throws IOE gen); } + if (response.idempotencyKeyLifetime() != null) { + gen.writeStringField(IDEMPOTENCY_KEY_LIFETIME, response.idempotencyKeyLifetime()); + } + gen.writeEndObject(); } @@ -83,6 +88,11 @@ public static ConfigResponse fromJson(JsonNode json) { .collect(Collectors.toList())); } + if (json.hasNonNull(IDEMPOTENCY_KEY_LIFETIME)) { + builder.withIdempotencyKeyLifetime( + JsonUtil.getDurationStringOrNull(IDEMPOTENCY_KEY_LIFETIME, json)); + } + return builder.build(); } } diff --git a/core/src/main/java/org/apache/iceberg/util/JsonUtil.java b/core/src/main/java/org/apache/iceberg/util/JsonUtil.java index ed5dd0924e79..d8bb1f919096 100644 --- a/core/src/main/java/org/apache/iceberg/util/JsonUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/JsonUtil.java @@ -189,6 +189,24 @@ public static String getStringOrNull(String property, JsonNode node) { return getString(property, node); } + public static String getDurationStringOrNull(String property, JsonNode node) { + String value = getStringOrNull(property, node); + if (value == null) { + return null; + } + + try { + java.time.Duration.parse(value); + } catch (RuntimeException e) { + throw new IllegalArgumentException( + String.format( + "Cannot parse to a duration string value: %s: %s", property, node.get(property)), + e); + } + + return value; + } + public static ByteBuffer getByteBufferOrNull(String property, JsonNode node) { if (!node.has(property) || node.get(property).isNull()) { return null; diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java index 81ec7cc5585c..e2f109aa30ea 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java @@ -214,4 +214,47 @@ public void roundTripSerdeWithEndpoints() { assertThat(ConfigResponseParser.toJson(ConfigResponseParser.fromJson(json), true)) .isEqualTo(expectedJson); } + + @Test + public void idempotencyLifetimeOnly() { + ConfigResponse response = ConfigResponse.builder().withIdempotencyKeyLifetime("PT30M").build(); + + String expectedJson = + "{\n" + + " \"defaults\" : { },\n" + + " \"overrides\" : { },\n" + + " \"idempotency-key-lifetime\" : \"PT30M\"\n" + + "}"; + + String json = ConfigResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + ConfigResponse roundTripped = ConfigResponseParser.fromJson(json); + assertThat(roundTripped.idempotencyKeyLifetime()).isEqualTo("PT30M"); + } + + @Test + public void invalidIdempotencyLifetime() { + assertThatThrownBy( + () -> + ConfigResponseParser.fromJson( + "{\n" + + " \"defaults\" : { },\n" + + " \"overrides\" : { },\n" + + " \"idempotency-key-lifetime\" : \"not-a-duration\"\n" + + "}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot parse to a duration string value: idempotency-key-lifetime: \"not-a-duration\""); + + assertThatThrownBy( + () -> + ConfigResponseParser.fromJson( + "{\n" + + " \"defaults\" : { },\n" + + " \"overrides\" : { },\n" + + " \"idempotency-key-lifetime\" : \"\"\n" + + "}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse to a duration string value: idempotency-key-lifetime: \"\""); + } } diff --git a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java index 0ce154abd5bf..91cd96e9088f 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java @@ -156,6 +156,29 @@ public void getStringOrNull() throws JsonProcessingException { .hasMessage("Cannot parse to a string value: x: 23"); } + @Test + public void getDurationStringOrNull() throws JsonProcessingException { + assertThat(JsonUtil.getDurationStringOrNull("x", JsonUtil.mapper().readTree("{}"))).isNull(); + assertThat(JsonUtil.getDurationStringOrNull("x", JsonUtil.mapper().readTree("{\"x\": null}"))) + .isNull(); + assertThat( + JsonUtil.getDurationStringOrNull("x", JsonUtil.mapper().readTree("{\"x\": \"PT30M\"}"))) + .isEqualTo("PT30M"); + + assertThatThrownBy( + () -> + JsonUtil.getDurationStringOrNull( + "x", JsonUtil.mapper().readTree("{\"x\": \"30M\"}"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse to a duration string value: x: \"30M\""); + + assertThatThrownBy( + () -> + JsonUtil.getDurationStringOrNull("x", JsonUtil.mapper().readTree("{\"x\": \"\"}"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse to a duration string value: x: \"\""); + } + @Test public void getByteBufferOrNull() throws JsonProcessingException { assertThat(JsonUtil.getByteBufferOrNull("x", JsonUtil.mapper().readTree("{}"))).isNull(); From 078fbeb1050bc22a665513b5a20a5742d690660e Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Wed, 26 Nov 2025 14:29:40 +0800 Subject: [PATCH 004/201] Build: Don't ignore major version upgrade for GH actions in dependabot (#14687) --- .github/dependabot.yml | 3 --- 1 file changed, 3 deletions(-) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index 51a34758fe09..6acbbacda25d 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -24,9 +24,6 @@ updates: schedule: interval: "weekly" day: "sunday" - ignore: - - dependency-name: "*" - update-types: ["version-update:semver-major"] - package-ecosystem: "gradle" directory: "/" schedule: From 790a82065ca30ccf23a97b5a22497a42ed19c7df Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 25 Nov 2025 23:33:57 -0800 Subject: [PATCH 005/201] Build: Bump actions/labeler from 5 to 6 (#14689) Bumps [actions/labeler](https://github.com/actions/labeler) from 5 to 6. - [Release notes](https://github.com/actions/labeler/releases) - [Commits](https://github.com/actions/labeler/compare/v5...v6) --- updated-dependencies: - dependency-name: actions/labeler dependency-version: '6' dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/labeler.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/labeler.yml b/.github/workflows/labeler.yml index a34b62c1014e..4d8a367df591 100644 --- a/.github/workflows/labeler.yml +++ b/.github/workflows/labeler.yml @@ -28,7 +28,7 @@ jobs: triage: runs-on: ubuntu-24.04 steps: - - uses: actions/labeler@v5 + - uses: actions/labeler@v6 with: repo-token: "${{ secrets.GITHUB_TOKEN }}" sync-labels: true From 5e166b5d77fe6021aba4954c72273db17af0d041 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 25 Nov 2025 23:34:34 -0800 Subject: [PATCH 006/201] Build: Bump actions/setup-python from 5 to 6 (#14690) Bumps [actions/setup-python](https://github.com/actions/setup-python) from 5 to 6. - [Release notes](https://github.com/actions/setup-python/releases) - [Commits](https://github.com/actions/setup-python/compare/v5...v6) --- updated-dependencies: - dependency-name: actions/setup-python dependency-version: '6' dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/docs-ci.yml | 2 +- .github/workflows/open-api.yml | 2 +- .github/workflows/site-ci.yml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/docs-ci.yml b/.github/workflows/docs-ci.yml index 67d054f8f576..388be10aa625 100644 --- a/.github/workflows/docs-ci.yml +++ b/.github/workflows/docs-ci.yml @@ -33,7 +33,7 @@ jobs: os: [ubuntu-latest, macos-latest] steps: - uses: actions/checkout@v4 - - uses: actions/setup-python@v5 + - uses: actions/setup-python@v6 with: python-version: 3.x - name: Build Iceberg documentation diff --git a/.github/workflows/open-api.yml b/.github/workflows/open-api.yml index ab2109e1df6a..8a96407a3105 100644 --- a/.github/workflows/open-api.yml +++ b/.github/workflows/open-api.yml @@ -42,7 +42,7 @@ jobs: steps: - uses: actions/checkout@v4 - - uses: actions/setup-python@v5 + - uses: actions/setup-python@v6 with: python-version: 3.9 - name: Install diff --git a/.github/workflows/site-ci.yml b/.github/workflows/site-ci.yml index b8661f990c96..4dc82d7156ac 100644 --- a/.github/workflows/site-ci.yml +++ b/.github/workflows/site-ci.yml @@ -31,7 +31,7 @@ jobs: runs-on: ubuntu-latest steps: - uses: actions/checkout@v4 - - uses: actions/setup-python@v5 + - uses: actions/setup-python@v6 with: python-version: 3.x - name: Deploy Iceberg documentation From b7549da63b4bb5cc6d2604058bff08042c8a1cd9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 25 Nov 2025 23:35:14 -0800 Subject: [PATCH 007/201] Build: Bump actions/stale from 9.1.0 to 10.1.0 (#14692) Bumps [actions/stale](https://github.com/actions/stale) from 9.1.0 to 10.1.0. - [Release notes](https://github.com/actions/stale/releases) - [Changelog](https://github.com/actions/stale/blob/main/CHANGELOG.md) - [Commits](https://github.com/actions/stale/compare/v9.1.0...v10.1.0) --- updated-dependencies: - dependency-name: actions/stale dependency-version: 10.1.0 dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/stale.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index 6846d53f538b..a7118a4ffdd9 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -32,7 +32,7 @@ jobs: if: github.repository_owner == 'apache' runs-on: ubuntu-24.04 steps: - - uses: actions/stale@v9.1.0 + - uses: actions/stale@v10.1.0 with: # stale issues stale-issue-label: 'stale' From 0846ed515691461545ea635cb871fd7f06176e9f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 25 Nov 2025 23:42:38 -0800 Subject: [PATCH 008/201] Build: Bump software.amazon.awssdk:bom from 2.39.2 to 2.39.4 (#14693) Bumps software.amazon.awssdk:bom from 2.39.2 to 2.39.4. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-version: 2.39.4 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 3c803c49cb9d..034138681b06 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -31,7 +31,7 @@ arrow = "15.0.2" avro = "1.12.1" assertj-core = "3.27.6" awaitility = "4.3.0" -awssdk-bom = "2.39.2" +awssdk-bom = "2.39.4" azuresdk-bom = "1.3.2" awssdk-s3accessgrants = "2.3.0" bson-ver = "4.11.5" From 87174fcbd4f3130d4b863458011467eab42b0224 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 25 Nov 2025 23:48:54 -0800 Subject: [PATCH 009/201] Build: Bump actions/upload-artifact from 4 to 5 (#14688) Bumps [actions/upload-artifact](https://github.com/actions/upload-artifact) from 4 to 5. - [Release notes](https://github.com/actions/upload-artifact/releases) - [Commits](https://github.com/actions/upload-artifact/compare/v4...v5) --- updated-dependencies: - dependency-name: actions/upload-artifact dependency-version: '5' dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/api-binary-compatibility.yml | 2 +- .github/workflows/delta-conversion-ci.yml | 4 ++-- .github/workflows/flink-ci.yml | 2 +- .github/workflows/hive-ci.yml | 2 +- .github/workflows/java-ci.yml | 2 +- .github/workflows/jmh-benchmarks.yml | 2 +- .github/workflows/kafka-connect-ci.yml | 2 +- .github/workflows/recurring-jmh-benchmarks.yml | 2 +- .github/workflows/spark-ci.yml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/.github/workflows/api-binary-compatibility.yml b/.github/workflows/api-binary-compatibility.yml index 60e89b2fd03c..f71aa0476812 100644 --- a/.github/workflows/api-binary-compatibility.yml +++ b/.github/workflows/api-binary-compatibility.yml @@ -58,7 +58,7 @@ jobs: - run: | echo "Using the old version tag, as per git describe, of $(git describe)"; - run: ./gradlew revapi --rerun-tasks - - uses: actions/upload-artifact@v4 + - uses: actions/upload-artifact@v5 if: failure() with: name: test logs diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index e8da8d33bafc..a4396a3eccd3 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -89,7 +89,7 @@ jobs: restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.12 -DkafkaVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - - uses: actions/upload-artifact@v4 + - uses: actions/upload-artifact@v5 if: failure() with: name: test logs @@ -118,7 +118,7 @@ jobs: restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.13 -DkafkaVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - - uses: actions/upload-artifact@v4 + - uses: actions/upload-artifact@v5 if: failure() with: name: test logs diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 365744ef906f..2da1fcd368d0 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -92,7 +92,7 @@ jobs: restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - run: ./gradlew -DsparkVersions= -DkafkaVersions= -DflinkVersions=${{ matrix.flink }} :iceberg-flink:iceberg-flink-${{ matrix.flink }}:check :iceberg-flink:iceberg-flink-runtime-${{ matrix.flink }}:check -Pquick=true -x javadoc -DtestParallelism=auto - - uses: actions/upload-artifact@v4 + - uses: actions/upload-artifact@v5 if: failure() with: name: test logs diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index a01697664cc9..b6edf55c7955 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -90,7 +90,7 @@ jobs: restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - run: ./gradlew -DsparkVersions= -DflinkVersions= -DkafkaVersions= -Pquick=true :iceberg-mr:check -x javadoc - - uses: actions/upload-artifact@v4 + - uses: actions/upload-artifact@v5 if: failure() with: name: test logs diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 6e25a1745f0c..34e6408ddd72 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -85,7 +85,7 @@ jobs: restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - run: ./gradlew check -DsparkVersions= -DflinkVersions= -DkafkaVersions= -Pquick=true -x javadoc - - uses: actions/upload-artifact@v4 + - uses: actions/upload-artifact@v5 if: failure() with: name: test logs diff --git a/.github/workflows/jmh-benchmarks.yml b/.github/workflows/jmh-benchmarks.yml index 3018f24b4925..426a517c07cb 100644 --- a/.github/workflows/jmh-benchmarks.yml +++ b/.github/workflows/jmh-benchmarks.yml @@ -95,7 +95,7 @@ jobs: - name: Run Benchmark run: ./gradlew :iceberg-spark:${{ github.event.inputs.spark_version }}:jmh -PjmhIncludeRegex=${{ matrix.benchmark }} -PjmhOutputPath=benchmark/${{ matrix.benchmark }}.txt - - uses: actions/upload-artifact@v4 + - uses: actions/upload-artifact@v5 if: ${{ always() }} with: name: benchmark-results diff --git a/.github/workflows/kafka-connect-ci.yml b/.github/workflows/kafka-connect-ci.yml index 72a55dfd991b..8f189021ca0c 100644 --- a/.github/workflows/kafka-connect-ci.yml +++ b/.github/workflows/kafka-connect-ci.yml @@ -96,7 +96,7 @@ jobs: :iceberg-kafka-connect:iceberg-kafka-connect:check \ :iceberg-kafka-connect:iceberg-kafka-connect-runtime:check \ -Pquick=true -x javadoc - - uses: actions/upload-artifact@v4 + - uses: actions/upload-artifact@v5 if: failure() with: name: test logs diff --git a/.github/workflows/recurring-jmh-benchmarks.yml b/.github/workflows/recurring-jmh-benchmarks.yml index 521a09414f4c..cc811d14cef7 100644 --- a/.github/workflows/recurring-jmh-benchmarks.yml +++ b/.github/workflows/recurring-jmh-benchmarks.yml @@ -65,7 +65,7 @@ jobs: - name: Run Benchmark run: ./gradlew :iceberg-spark:${{ matrix.spark_version }}:jmh -PjmhIncludeRegex=${{ matrix.benchmark }} -PjmhOutputPath=benchmark/${{ matrix.benchmark }}.txt -PjmhJsonOutputPath=benchmark/${{ matrix.benchmark }}.json - - uses: actions/upload-artifact@v4 + - uses: actions/upload-artifact@v5 if: ${{ always() }} with: name: benchmark-results diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index 66193515efb7..b5cfa3569498 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -108,7 +108,7 @@ jobs: :iceberg-spark:iceberg-spark-extensions-${{ matrix.spark }}_${{ matrix.scala }}:check \ :iceberg-spark:iceberg-spark-runtime-${{ matrix.spark }}_${{ matrix.scala }}:check \ -Pquick=true -x javadoc - - uses: actions/upload-artifact@v4 + - uses: actions/upload-artifact@v5 if: failure() with: name: test logs From fe6f78b32839bfb3ef6153fc84d836e68b447581 Mon Sep 17 00:00:00 2001 From: Tamas Mate <50709850+tmater@users.noreply.github.com> Date: Wed, 26 Nov 2025 15:39:16 +0100 Subject: [PATCH 010/201] Core: Allow overriding view location for subclasses (#14653) --- .../apache/iceberg/view/ViewCatalogTests.java | 33 ++++++++++--------- 1 file changed, 18 insertions(+), 15 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index 71e6f324380f..fdf4138f3ce5 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -24,7 +24,6 @@ import static org.assertj.core.api.Assumptions.assumeThat; import java.nio.file.Path; -import java.nio.file.Paths; import java.util.UUID; import org.apache.iceberg.Schema; import org.apache.iceberg.Transaction; @@ -42,6 +41,7 @@ import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.LocationUtil; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; @@ -63,6 +63,16 @@ public abstract class ViewCatalogTests Date: Wed, 26 Nov 2025 09:01:29 -0800 Subject: [PATCH 011/201] Core: Fix server side planning on empty tables in CatalogHandlers (#14660) --- .../apache/iceberg/rest/CatalogHandlers.java | 70 +++++++++++++------ 1 file changed, 48 insertions(+), 22 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java index b08455408d56..82695eaf7875 100644 --- a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java @@ -685,24 +685,33 @@ public static PlanTableScanResponse planTableScan( } String planId = "sync-" + UUID.randomUUID(); - planFilesFor( - configuredScan, - planId, - table.uuid().toString(), - tasksPerPlanTask.applyAsInt(configuredScan)); - Pair, String> initial = IN_MEMORY_PLANNING_STATE.initialScanTasksFor(planId); - return PlanTableScanResponse.builder() - .withPlanStatus(PlanStatus.COMPLETED) - .withPlanId(planId) - .withPlanTasks(IN_MEMORY_PLANNING_STATE.nextPlanTask(initial.second())) - .withFileScanTasks(initial.first()) - .withDeleteFiles( - initial.first().stream() - .flatMap(task -> task.deletes().stream()) - .distinct() - .collect(Collectors.toList())) - .withSpecsById(table.specs()) - .build(); + Pair, String> initial = + planFilesFor( + configuredScan, + planId, + table.uuid().toString(), + tasksPerPlanTask.applyAsInt(configuredScan)); + List nextPlanTasks = + initial.second() == null + ? Collections.emptyList() + : IN_MEMORY_PLANNING_STATE.nextPlanTask(initial.second()); + PlanTableScanResponse.Builder builder = + PlanTableScanResponse.builder() + .withPlanStatus(PlanStatus.COMPLETED) + .withPlanId(planId) + .withFileScanTasks(initial.first()) + .withDeleteFiles( + initial.first().stream() + .flatMap(task -> task.deletes().stream()) + .distinct() + .collect(Collectors.toList())) + .withSpecsById(table.specs()); + + if (!nextPlanTasks.isEmpty()) { + builder.withPlanTasks(nextPlanTasks); + } + + return builder.build(); } /** @@ -807,22 +816,39 @@ static void clearPlanningState() { * @param planId the unique identifier for this plan * @param tableId the uuid of the table being scanned * @param tasksPerPlanTask number of file scan tasks to group per plan task + * @return the initial file scan tasks and the first plan task key */ - private static void planFilesFor( + private static Pair, String> planFilesFor( Scan scan, String planId, String tableId, int tasksPerPlanTask) { - Iterable> taskGroupings = - Iterables.partition(scan.planFiles(), tasksPerPlanTask); + Iterable planTasks = scan.planFiles(); + String planTaskPrefix = planId + "-" + tableId + "-"; + + // Handle empty table scans + if (!planTasks.iterator().hasNext()) { + String planTaskKey = planTaskPrefix + "0"; + // Add empty scan to planning state so async calls know the scan completed + IN_MEMORY_PLANNING_STATE.addPlanTask(planTaskKey, Collections.emptyList()); + return Pair.of(Collections.emptyList(), planTaskKey); + } + + Iterable> taskGroupings = Iterables.partition(planTasks, tasksPerPlanTask); int planTaskSequence = 0; String previousPlanTask = null; + String firstPlanTaskKey = null; + List initialFileScanTasks = null; for (List taskGrouping : taskGroupings) { - String planTaskKey = String.format("%s-%s-%s", planId, tableId, planTaskSequence++); + String planTaskKey = planTaskPrefix + planTaskSequence++; IN_MEMORY_PLANNING_STATE.addPlanTask(planTaskKey, taskGrouping); if (previousPlanTask != null) { IN_MEMORY_PLANNING_STATE.addNextPlanTask(previousPlanTask, planTaskKey); + } else { + firstPlanTaskKey = planTaskKey; + initialFileScanTasks = taskGrouping; } previousPlanTask = planTaskKey; } + return Pair.of(initialFileScanTasks, firstPlanTaskKey); } @SuppressWarnings("FutureReturnValueIgnored") From 42719ef41eefc56968b528c51550f3ac63682eb2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 27 Nov 2025 14:57:09 +0100 Subject: [PATCH 012/201] Build: Bump actions/checkout from 3 to 6 (#14691) Bumps [actions/checkout](https://github.com/actions/checkout) from 3 to 6. - [Release notes](https://github.com/actions/checkout/releases) - [Changelog](https://github.com/actions/checkout/blob/main/CHANGELOG.md) - [Commits](https://github.com/actions/checkout/compare/v3...v6) --- updated-dependencies: - dependency-name: actions/checkout dependency-version: '6' dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/api-binary-compatibility.yml | 2 +- .github/workflows/delta-conversion-ci.yml | 4 ++-- .github/workflows/docs-ci.yml | 2 +- .github/workflows/flink-ci.yml | 2 +- .github/workflows/hive-ci.yml | 2 +- .github/workflows/java-ci.yml | 6 +++--- .github/workflows/jmh-benchmarks.yml | 4 ++-- .github/workflows/kafka-connect-ci.yml | 2 +- .github/workflows/license-check.yml | 2 +- .github/workflows/open-api.yml | 2 +- .github/workflows/publish-iceberg-rest-fixture-docker.yml | 2 +- .github/workflows/publish-snapshot.yml | 2 +- .github/workflows/recurring-jmh-benchmarks.yml | 2 +- .github/workflows/site-ci.yml | 2 +- .github/workflows/spark-ci.yml | 2 +- 15 files changed, 19 insertions(+), 19 deletions(-) diff --git a/.github/workflows/api-binary-compatibility.yml b/.github/workflows/api-binary-compatibility.yml index f71aa0476812..2b6f8ec323e4 100644 --- a/.github/workflows/api-binary-compatibility.yml +++ b/.github/workflows/api-binary-compatibility.yml @@ -43,7 +43,7 @@ jobs: revapi: runs-on: ubuntu-24.04 steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 with: # fetch-depth of zero ensures that the tags are pulled in and we're not in a detached HEAD state # revapi depends on the tags, specifically the tag from git describe, to find the relevant override diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index a4396a3eccd3..cc6b2d89331e 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -75,7 +75,7 @@ jobs: env: SPARK_LOCAL_IP: localhost steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - uses: actions/setup-java@v5 with: distribution: zulu @@ -104,7 +104,7 @@ jobs: env: SPARK_LOCAL_IP: localhost steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - uses: actions/setup-java@v5 with: distribution: zulu diff --git a/.github/workflows/docs-ci.yml b/.github/workflows/docs-ci.yml index 388be10aa625..48da4785dadb 100644 --- a/.github/workflows/docs-ci.yml +++ b/.github/workflows/docs-ci.yml @@ -32,7 +32,7 @@ jobs: matrix: os: [ubuntu-latest, macos-latest] steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - uses: actions/setup-python@v6 with: python-version: 3.x diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 2da1fcd368d0..8b8c9426e1ca 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -78,7 +78,7 @@ jobs: env: SPARK_LOCAL_IP: localhost steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - uses: actions/setup-java@v5 with: distribution: zulu diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index b6edf55c7955..282cf9e8454d 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -76,7 +76,7 @@ jobs: env: SPARK_LOCAL_IP: localhost steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - uses: actions/setup-java@v5 with: distribution: zulu diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 34e6408ddd72..5505bb25db83 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -71,7 +71,7 @@ jobs: env: SPARK_LOCAL_IP: localhost steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - uses: actions/setup-java@v5 with: distribution: zulu @@ -98,7 +98,7 @@ jobs: matrix: jvm: [11, 17, 21] steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - uses: actions/setup-java@v5 with: distribution: zulu @@ -111,7 +111,7 @@ jobs: matrix: jvm: [11, 17, 21] steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - uses: actions/setup-java@v5 with: distribution: zulu diff --git a/.github/workflows/jmh-benchmarks.yml b/.github/workflows/jmh-benchmarks.yml index 426a517c07cb..0536fd7c88c7 100644 --- a/.github/workflows/jmh-benchmarks.yml +++ b/.github/workflows/jmh-benchmarks.yml @@ -42,7 +42,7 @@ jobs: matrix: ${{ steps.set-matrix.outputs.matrix }} foundlabel: ${{ steps.set-matrix.outputs.foundlabel }} steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 with: repository: ${{ github.event.inputs.repo }} ref: ${{ github.event.inputs.ref }} @@ -75,7 +75,7 @@ jobs: env: SPARK_LOCAL_IP: localhost steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 with: repository: ${{ github.event.inputs.repo }} ref: ${{ github.event.inputs.ref }} diff --git a/.github/workflows/kafka-connect-ci.yml b/.github/workflows/kafka-connect-ci.yml index 8f189021ca0c..168d64a6c1d5 100644 --- a/.github/workflows/kafka-connect-ci.yml +++ b/.github/workflows/kafka-connect-ci.yml @@ -76,7 +76,7 @@ jobs: env: SPARK_LOCAL_IP: localhost steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - uses: actions/setup-java@v5 with: distribution: zulu diff --git a/.github/workflows/license-check.yml b/.github/workflows/license-check.yml index ab55763c9aef..7640d40f10b2 100644 --- a/.github/workflows/license-check.yml +++ b/.github/workflows/license-check.yml @@ -24,6 +24,6 @@ jobs: rat: runs-on: ubuntu-24.04 steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - run: | dev/check-license diff --git a/.github/workflows/open-api.yml b/.github/workflows/open-api.yml index 8a96407a3105..b20aef0fa63d 100644 --- a/.github/workflows/open-api.yml +++ b/.github/workflows/open-api.yml @@ -41,7 +41,7 @@ jobs: runs-on: ubuntu-24.04 steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - uses: actions/setup-python@v6 with: python-version: 3.9 diff --git a/.github/workflows/publish-iceberg-rest-fixture-docker.yml b/.github/workflows/publish-iceberg-rest-fixture-docker.yml index 09609aa58f43..4b6b4696bec7 100644 --- a/.github/workflows/publish-iceberg-rest-fixture-docker.yml +++ b/.github/workflows/publish-iceberg-rest-fixture-docker.yml @@ -37,7 +37,7 @@ jobs: if: github.repository_owner == 'apache' runs-on: ubuntu-latest steps: - - uses: actions/checkout@v3 + - uses: actions/checkout@v6 - uses: actions/setup-java@v5 with: distribution: zulu diff --git a/.github/workflows/publish-snapshot.yml b/.github/workflows/publish-snapshot.yml index 3ad3d9273b5b..14e87044132d 100644 --- a/.github/workflows/publish-snapshot.yml +++ b/.github/workflows/publish-snapshot.yml @@ -30,7 +30,7 @@ jobs: if: github.repository_owner == 'apache' runs-on: ubuntu-24.04 steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 with: # we need to fetch all tags so that getProjectVersion() in build.gradle correctly determines the next SNAPSHOT version from the newest tag fetch-depth: 0 diff --git a/.github/workflows/recurring-jmh-benchmarks.yml b/.github/workflows/recurring-jmh-benchmarks.yml index cc811d14cef7..fa5a2e79e1e6 100644 --- a/.github/workflows/recurring-jmh-benchmarks.yml +++ b/.github/workflows/recurring-jmh-benchmarks.yml @@ -45,7 +45,7 @@ jobs: env: SPARK_LOCAL_IP: localhost steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 with: repository: ${{ github.event.inputs.repo }} ref: ${{ github.event.inputs.ref }} diff --git a/.github/workflows/site-ci.yml b/.github/workflows/site-ci.yml index 4dc82d7156ac..7efbf94ba884 100644 --- a/.github/workflows/site-ci.yml +++ b/.github/workflows/site-ci.yml @@ -30,7 +30,7 @@ jobs: deploy: runs-on: ubuntu-latest steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - uses: actions/setup-python@v6 with: python-version: 3.x diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index b5cfa3569498..3488dfdfea52 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -86,7 +86,7 @@ jobs: env: SPARK_LOCAL_IP: localhost steps: - - uses: actions/checkout@v4 + - uses: actions/checkout@v6 - uses: actions/setup-java@v5 with: distribution: zulu From cf27769762d678a8a790481318b6346c2dd480ff Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 28 Nov 2025 02:41:34 +0100 Subject: [PATCH 013/201] Spark: Fix scala warnings in View code (#14703) --- .../org/apache/spark/sql/catalyst/analysis/CheckViews.scala | 2 +- .../apache/spark/sql/catalyst/analysis/ResolveViews.scala | 4 ++-- .../spark/sql/catalyst/analysis/RewriteViewCommands.scala | 2 +- .../sql/execution/datasources/v2/CreateV2ViewExec.scala | 6 +++--- .../sql/execution/datasources/v2/DescribeV2ViewExec.scala | 2 +- .../sql/execution/datasources/v2/ShowCreateV2ViewExec.scala | 2 +- .../execution/datasources/v2/ShowV2ViewPropertiesExec.scala | 2 +- .../org/apache/spark/sql/catalyst/analysis/CheckViews.scala | 2 +- .../apache/spark/sql/catalyst/analysis/ResolveViews.scala | 4 ++-- .../spark/sql/catalyst/analysis/RewriteViewCommands.scala | 2 +- .../sql/execution/datasources/v2/CreateV2ViewExec.scala | 6 +++--- .../sql/execution/datasources/v2/DescribeV2ViewExec.scala | 2 +- .../sql/execution/datasources/v2/ShowCreateV2ViewExec.scala | 2 +- .../execution/datasources/v2/ShowV2ViewPropertiesExec.scala | 2 +- .../org/apache/spark/sql/catalyst/analysis/CheckViews.scala | 2 +- .../apache/spark/sql/catalyst/analysis/ResolveViews.scala | 4 ++-- .../spark/sql/catalyst/analysis/RewriteViewCommands.scala | 2 +- .../sql/execution/datasources/v2/CreateV2ViewExec.scala | 6 +++--- .../sql/execution/datasources/v2/DescribeV2ViewExec.scala | 4 ++-- .../sql/execution/datasources/v2/ShowCreateV2ViewExec.scala | 2 +- .../execution/datasources/v2/ShowV2ViewPropertiesExec.scala | 2 +- 21 files changed, 31 insertions(+), 31 deletions(-) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala index 685b85a0d75f..44b22e68d076 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -41,7 +41,7 @@ object CheckViews extends (LogicalPlan => Unit) { case CreateIcebergView(resolvedIdent@ResolvedIdentifier(_: ViewCatalog, _), _, query, columnAliases, _, _, _, _, _, replace, _) => verifyColumnCount(resolvedIdent, columnAliases, query) - SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames, SQLConf.get.resolver) + SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames.toIndexedSeq, SQLConf.get.resolver) if (replace) { val viewIdent: Seq[String] = resolvedIdent.catalog.name() +: resolvedIdent.identifier.asMultipartIdentifier checkCyclicViewReference(viewIdent, query, Seq(viewIdent)) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala index 37791dbbec2b..b77cc033268d 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala @@ -64,7 +64,7 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look case c@CreateIcebergView(ResolvedIdentifier(_, _), _, query, columnAliases, columnComments, _, _, _, _, _, _) if query.resolved && !c.rewritten => val aliased = aliasColumns(query, columnAliases, columnComments) - c.copy(query = aliased, queryColumnNames = query.schema.fieldNames, rewritten = true) + c.copy(query = aliased, queryColumnNames = query.schema.fieldNames.toIndexedSeq, rewritten = true) } private def aliasColumns( @@ -99,7 +99,7 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look val aliases = view.schema.fields.zipWithIndex.map { case (expected, pos) => val attr = GetColumnByOrdinal(pos, expected.dataType) Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = Some(expected.metadata)) - } + }.toIndexedSeq SubqueryAlias(nameParts, Project(aliases, rewritten)) } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index 3e854dfcb966..c7f09b2f5a07 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -68,7 +68,7 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case view @ ShowViews(UnresolvedNamespace(Seq()), pattern, output) => if (ViewUtil.isViewCatalog(catalogManager.currentCatalog)) { - ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), + ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace.toIndexedSeq), pattern, output) } else { view diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala index d2db0f4992fe..eb4cc7de48c4 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.types.StructType -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class CreateV2ViewExec( @@ -52,8 +52,8 @@ case class CreateV2ViewExec( val engineVersion = "Spark " + org.apache.spark.SPARK_VERSION val newProperties = properties ++ - comment.map(ViewCatalog.PROP_COMMENT -> _) + - (ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion, + comment.map(ViewCatalog.PROP_COMMENT -> _) ++ + Map(ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion, ViewCatalog.PROP_ENGINE_VERSION -> engineVersion) if (replace) { diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala index 2a5566a15f21..a9c7312d4a90 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.escapeSingleQuotedString import org.apache.spark.sql.connector.catalog.View import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class DescribeV2ViewExec( output: Seq[Attribute], diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala index 3be0f150313b..ecd3e0d3c9ba 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.escapeSingleQuotedString import org.apache.spark.sql.connector.catalog.View import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View) extends V2CommandExec with LeafExecNode { diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala index 89fafe99efc8..553b80833087 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.View import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class ShowV2ViewPropertiesExec( output: Seq[Attribute], diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala index b8cd1020298d..2a0e573b18e3 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -41,7 +41,7 @@ object CheckViews extends (LogicalPlan => Unit) { case CreateIcebergView(resolvedIdent@ResolvedIdentifier(_: ViewCatalog, _), _, query, columnAliases, _, _, _, _, _, replace, _) => verifyColumnCount(resolvedIdent, columnAliases, query) - SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames, SQLConf.get.resolver) + SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames.toIndexedSeq, SQLConf.get.resolver) if (replace) { val viewIdent: Seq[String] = resolvedIdent.catalog.name() +: resolvedIdent.identifier.asMultipartIdentifier checkCyclicViewReference(viewIdent, query, Seq(viewIdent)) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala index c80ce448e051..0fb5d30a91d5 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala @@ -64,7 +64,7 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look case c@CreateIcebergView(ResolvedIdentifier(_, _), _, query, columnAliases, columnComments, _, _, _, _, _, _) if query.resolved && !c.rewritten => val aliased = aliasColumns(query, columnAliases, columnComments) - c.copy(query = aliased, queryColumnNames = query.schema.fieldNames, rewritten = true) + c.copy(query = aliased, queryColumnNames = query.schema.fieldNames.toIndexedSeq, rewritten = true) } private def aliasColumns( @@ -100,7 +100,7 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look val aliases = view.schema.fields.zipWithIndex.map { case (expected, pos) => val attr = GetColumnByOrdinal(pos, expected.dataType) Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = Some(expected.metadata)) - } + }.toIndexedSeq SubqueryAlias(nameParts, Project(aliases, rewritten)) } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index c5bd648f2e4e..800ad1f66abc 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -68,7 +68,7 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case view @ ShowViews(UnresolvedNamespace(Seq()), pattern, output) => if (ViewUtil.isViewCatalog(catalogManager.currentCatalog)) { - ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), + ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace.toIndexedSeq), pattern, output) } else { view diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala index d2db0f4992fe..eb4cc7de48c4 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.types.StructType -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class CreateV2ViewExec( @@ -52,8 +52,8 @@ case class CreateV2ViewExec( val engineVersion = "Spark " + org.apache.spark.SPARK_VERSION val newProperties = properties ++ - comment.map(ViewCatalog.PROP_COMMENT -> _) + - (ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion, + comment.map(ViewCatalog.PROP_COMMENT -> _) ++ + Map(ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion, ViewCatalog.PROP_ENGINE_VERSION -> engineVersion) if (replace) { diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala index 2a5566a15f21..a9c7312d4a90 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.escapeSingleQuotedString import org.apache.spark.sql.connector.catalog.View import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class DescribeV2ViewExec( output: Seq[Attribute], diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala index 3be0f150313b..ecd3e0d3c9ba 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.escapeSingleQuotedString import org.apache.spark.sql.connector.catalog.View import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View) extends V2CommandExec with LeafExecNode { diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala index 89fafe99efc8..553b80833087 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.View import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class ShowV2ViewPropertiesExec( output: Seq[Attribute], diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala index 109aa5360e84..12c1fab36bd4 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -41,7 +41,7 @@ object CheckViews extends (LogicalPlan => Unit) { case CreateIcebergView(resolvedIdent@ResolvedIdentifier(_: ViewCatalog, _), _, query, columnAliases, _, _, _, _, _, replace, _) => verifyColumnCount(resolvedIdent, columnAliases, query) - SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames, SQLConf.get.resolver) + SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames.toIndexedSeq, SQLConf.get.resolver) if (replace) { val viewIdent: Seq[String] = resolvedIdent.catalog.name() +: resolvedIdent.identifier.asMultipartIdentifier checkCyclicViewReference(viewIdent, query, Seq(viewIdent)) diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala index e69db3fce509..fe9462523405 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala @@ -64,7 +64,7 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look case c@CreateIcebergView(ResolvedIdentifier(_, _), _, query, columnAliases, columnComments, _, _, _, _, _, _) if query.resolved && !c.rewritten => val aliased = aliasColumns(query, columnAliases, columnComments) - c.copy(query = aliased, queryColumnNames = query.schema.fieldNames, rewritten = true) + c.copy(query = aliased, queryColumnNames = query.schema.fieldNames.toIndexedSeq, rewritten = true) } private def aliasColumns( @@ -100,7 +100,7 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look val aliases = view.schema.fields.zipWithIndex.map { case (expected, pos) => val attr = GetColumnByOrdinal(pos, expected.dataType) Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = Some(expected.metadata)) - } + }.toIndexedSeq SubqueryAlias(nameParts, Project(aliases, rewritten)) } diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index db9ecfcd9e45..1eeb50df8d11 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -67,7 +67,7 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case view @ ShowViews(CurrentNamespace, pattern, output) => if (ViewUtil.isViewCatalog(catalogManager.currentCatalog)) { - ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), + ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace.toIndexedSeq), pattern, output) } else { view diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala index 9015fb338ea5..97077a5ecf37 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.connector.catalog.ViewInfo import org.apache.spark.sql.types.StructType -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class CreateV2ViewExec( @@ -53,8 +53,8 @@ case class CreateV2ViewExec( val engineVersion = "Spark " + org.apache.spark.SPARK_VERSION val newProperties = properties ++ - comment.map(ViewCatalog.PROP_COMMENT -> _) + - (ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion, + comment.map(ViewCatalog.PROP_COMMENT -> _) ++ + Map(ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion, ViewCatalog.PROP_ENGINE_VERSION -> engineVersion) if (replace) { diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala index 2a5566a15f21..38385d2979b7 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.escapeSingleQuotedString import org.apache.spark.sql.connector.catalog.View import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class DescribeV2ViewExec( output: Seq[Attribute], @@ -55,7 +55,7 @@ case class DescribeV2ViewExec( private def describeExtended: Seq[InternalRow] = { val outputColumns = view.queryColumnNames.mkString("[", ", ", "]") val properties: Map[String, String] = view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala - val viewCatalogAndNamespace: Seq[String] = view.name.split("\\.").take(2) + val viewCatalogAndNamespace: Seq[String] = view.name.split("\\.").take(2).toIndexedSeq val viewProperties = properties.toSeq.sortBy(_._1).map { case (key, value) => s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala index 3be0f150313b..ecd3e0d3c9ba 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.escapeSingleQuotedString import org.apache.spark.sql.connector.catalog.View import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View) extends V2CommandExec with LeafExecNode { diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala index 89fafe99efc8..553b80833087 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.View import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode -import scala.collection.JavaConverters._ +import scala.jdk.CollectionConverters._ case class ShowV2ViewPropertiesExec( output: Seq[Attribute], From d2d4135724b844107b6a0cc117c1776abbcf69d9 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Fri, 28 Nov 2025 14:15:55 -0800 Subject: [PATCH 014/201] infra: notify on github workflow failure (#14609) Notify ci-jobs@iceberg.apache.org when Github Workflow fails --- .asf.yaml | 1 + 1 file changed, 1 insertion(+) diff --git a/.asf.yaml b/.asf.yaml index d9734f1cd804..e7c1e4cab2e5 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -64,6 +64,7 @@ notifications: commits: commits@iceberg.apache.org issues: issues@iceberg.apache.org pullrequests: issues@iceberg.apache.org + jobs: ci-jobs@iceberg.apache.org jira_options: link label link label publish: From 784f1f4693713b922ebfe43e055135fb3a8441c0 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Sat, 29 Nov 2025 15:55:15 +0900 Subject: [PATCH 015/201] Docs: Fix package of iceberg.catalog.io-impl (#14711) --- docs/docs/kafka-connect.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/docs/kafka-connect.md b/docs/docs/kafka-connect.md index 5fa03a957b8e..9c7d3d83f2e4 100644 --- a/docs/docs/kafka-connect.md +++ b/docs/docs/kafka-connect.md @@ -218,7 +218,7 @@ be found in the [Google Cloud documentation](https://cloud.google.com/docs/authe "iceberg.catalog.type": "rest", "iceberg.catalog.uri": "https://catalog:8181", "iceberg.catalog.warehouse": "gs://bucket-name/warehouse", -"iceberg.catalog.io-impl": "org.apache.iceberg.google.gcs.GCSFileIO" +"iceberg.catalog.io-impl": "org.apache.iceberg.gcp.gcs.GCSFileIO" ``` ### Hadoop configuration From 8a38d6e389758d372af63c4d56619c5a18832d2f Mon Sep 17 00:00:00 2001 From: Alessandro Nori Date: Sat, 29 Nov 2025 18:01:58 +0100 Subject: [PATCH 016/201] Spark 4.0: expire-snapshots with cleanupLevel=None (#14695) --- .../iceberg/spark/actions/ExpireSnapshotsSparkAction.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java index 5f4d4ec15184..e49e7326736f 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java @@ -27,6 +27,7 @@ import java.util.concurrent.ExecutorService; import java.util.function.Consumer; import java.util.stream.Collectors; +import org.apache.iceberg.ExpireSnapshots.CleanupLevel; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; @@ -169,7 +170,7 @@ public Dataset expireFiles() { expireSnapshots.cleanExpiredMetadata(cleanExpiredMetadata); } - expireSnapshots.cleanExpiredFiles(false).commit(); + expireSnapshots.cleanupLevel(CleanupLevel.NONE).commit(); // fetch valid files after expiration TableMetadata updatedMetadata = ops.refresh(); From 52c176df681cd6381ccf4d7f58ef1815ef5db19e Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Sat, 29 Nov 2025 09:13:21 -0800 Subject: [PATCH 017/201] Use non-deprecated del_branch_on_merge (#14710) --- .asf.yaml | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/.asf.yaml b/.asf.yaml index e7c1e4cab2e5..8d43adeff39f 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -33,14 +33,16 @@ github: squash: true rebase: true + pull_requests: + # auto-delete head branches after being merged + del_branch_on_merge: true + protected_branches: main: required_pull_request_reviews: required_approving_review_count: 1 required_linear_history: true - - del_branch_on_merge: true features: wiki: true From b9a8c31c048a03a29f2014ecea01cac4b5078c33 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 29 Nov 2025 23:41:16 -0800 Subject: [PATCH 018/201] Build: Bump software.amazon.awssdk:bom from 2.39.4 to 2.39.5 (#14718) Bumps software.amazon.awssdk:bom from 2.39.4 to 2.39.5. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-version: 2.39.5 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 034138681b06..b26f2651494e 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -31,7 +31,7 @@ arrow = "15.0.2" avro = "1.12.1" assertj-core = "3.27.6" awaitility = "4.3.0" -awssdk-bom = "2.39.4" +awssdk-bom = "2.39.5" azuresdk-bom = "1.3.2" awssdk-s3accessgrants = "2.3.0" bson-ver = "4.11.5" From 3a1177645af8f75c9ed5e3a33a229f1dd5a49ade Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 29 Nov 2025 23:42:05 -0800 Subject: [PATCH 019/201] Build: Bump datamodel-code-generator from 0.35.0 to 0.36.0 (#14716) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.35.0 to 0.36.0. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.35.0...0.36.0) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-version: 0.36.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index 9c06c4168dd8..c76cb3485572 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,4 +16,4 @@ # under the License. openapi-spec-validator==0.7.2 -datamodel-code-generator==0.35.0 +datamodel-code-generator==0.36.0 From 46d766ae48597d09cfdc26d3f502b2bbfaf6edcc Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 29 Nov 2025 23:42:29 -0800 Subject: [PATCH 020/201] Build: Bump com.google.errorprone:error_prone_annotations (#14715) Bumps [com.google.errorprone:error_prone_annotations](https://github.com/google/error-prone) from 2.44.0 to 2.45.0. - [Release notes](https://github.com/google/error-prone/releases) - [Commits](https://github.com/google/error-prone/compare/v2.44.0...v2.45.0) --- updated-dependencies: - dependency-name: com.google.errorprone:error_prone_annotations dependency-version: 2.45.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index b26f2651494e..6ecf0eec2f46 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -43,7 +43,7 @@ delta-standalone = "3.3.2" delta-spark = "3.3.2" derby = "10.15.2.0" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.44.0" +errorprone-annotations = "2.45.0" failsafe = "3.3.2" findbugs-jsr305 = "3.0.2" flink120 = { strictly = "1.20.1"} From 65c667da2f7231bdfa571864e0436868bfbd4918 Mon Sep 17 00:00:00 2001 From: Sreesh Maheshwar Date: Sun, 30 Nov 2025 08:04:45 +0000 Subject: [PATCH 021/201] Nit: Move unchecked suppression down to violating assignment in `ParquetMetricsRowGroupFilter` (#14013) Co-authored-by: Sreesh Maheshwar --- .../apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java index 89ded2a8acdb..cae9447513c0 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java @@ -438,7 +438,6 @@ public Boolean notIn(BoundReference ref, Set literalSet) { } @Override - @SuppressWarnings("unchecked") public Boolean startsWith(BoundReference ref, Literal lit) { int id = ref.fieldId(); @@ -448,6 +447,7 @@ public Boolean startsWith(BoundReference ref, Literal lit) { return ROWS_CANNOT_MATCH; } + @SuppressWarnings("unchecked") Statistics colStats = (Statistics) stats.get(id); if (colStats != null && !colStats.isEmpty()) { if (allNulls(colStats, valueCount)) { @@ -487,7 +487,6 @@ public Boolean startsWith(BoundReference ref, Literal lit) { } @Override - @SuppressWarnings("unchecked") public Boolean notStartsWith(BoundReference ref, Literal lit) { int id = ref.fieldId(); Long valueCount = valueCounts.get(id); @@ -497,6 +496,7 @@ public Boolean notStartsWith(BoundReference ref, Literal lit) { return ROWS_MIGHT_MATCH; } + @SuppressWarnings("unchecked") Statistics colStats = (Statistics) stats.get(id); if (colStats != null && !colStats.isEmpty()) { if (mayContainNull(colStats)) { From f3949cea069abd9339a150e6e8df8b90ee9f6675 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 30 Nov 2025 09:11:23 -0800 Subject: [PATCH 022/201] Build: Bump com.azure:azure-sdk-bom from 1.3.2 to 1.3.3 (#14717) Bumps [com.azure:azure-sdk-bom](https://github.com/azure/azure-sdk-for-java) from 1.3.2 to 1.3.3. - [Release notes](https://github.com/azure/azure-sdk-for-java/releases) - [Commits](https://github.com/azure/azure-sdk-for-java/compare/azure-sdk-bom_1.3.2...azure-sdk-bom_1.3.3) --- updated-dependencies: - dependency-name: com.azure:azure-sdk-bom dependency-version: 1.3.3 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 6ecf0eec2f46..3ba051c8b897 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -32,7 +32,7 @@ avro = "1.12.1" assertj-core = "3.27.6" awaitility = "4.3.0" awssdk-bom = "2.39.5" -azuresdk-bom = "1.3.2" +azuresdk-bom = "1.3.3" awssdk-s3accessgrants = "2.3.0" bson-ver = "4.11.5" caffeine = "2.9.3" From b35c7ec1b03e3897da68960cd556d635b2f5ae54 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 1 Dec 2025 00:45:09 -0800 Subject: [PATCH 023/201] Core: Fix NAN_VALUE_COUNTS serialization for ContentFile (#14721) --- .../org/apache/iceberg/ContentFileParser.java | 2 +- .../apache/iceberg/TestContentFileParser.java | 39 +++++++++++++++++++ 2 files changed, 40 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/ContentFileParser.java b/core/src/main/java/org/apache/iceberg/ContentFileParser.java index b48334d8222d..3d0deba3dfcb 100644 --- a/core/src/main/java/org/apache/iceberg/ContentFileParser.java +++ b/core/src/main/java/org/apache/iceberg/ContentFileParser.java @@ -228,7 +228,7 @@ private static void metricsToJson(ContentFile contentFile, JsonGenerator gene DataFile.NULL_VALUE_COUNTS.type(), contentFile.nullValueCounts(), generator); } - if (contentFile.nullValueCounts() != null) { + if (contentFile.nanValueCounts() != null) { generator.writeFieldName(NAN_VALUE_COUNTS); SingleValueParser.toJson( DataFile.NAN_VALUE_COUNTS.type(), contentFile.nanValueCounts(), generator); diff --git a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java index 3f463f722e9e..f8f0d77b0f0e 100644 --- a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java +++ b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java @@ -64,6 +64,21 @@ public void testNullArguments() throws Exception { .hasMessage("Invalid partition spec: null"); } + @Test + public void testNanCountsOnlyWritesNanValueCounts() throws Exception { + PartitionSpec spec = PartitionSpec.unpartitioned(); + DataFile dataFile = dataFileWithOnlyNanCounts(spec); + String jsonStr = ContentFileParser.toJson(dataFile, spec); + // ensure nan counts are present and null counts are not emitted + assertThat(jsonStr).contains("\"nan-value-counts\""); + assertThat(jsonStr).doesNotContain("\"null-value-counts\""); + JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); + ContentFile deserialized = + ContentFileParser.fromJson(jsonNode, Map.of(TestBase.SPEC.specId(), spec)); + assertThat(deserialized).isInstanceOf(DataFile.class); + assertContentFileEquals(dataFile, deserialized, spec); + } + @ParameterizedTest @MethodSource("provideSpecAndDataFile") public void testDataFile(PartitionSpec spec, DataFile dataFile, String expectedJson) @@ -125,6 +140,30 @@ private static DataFile dataFileWithRequiredOnly(PartitionSpec spec) { return builder.build(); } + private static DataFile dataFileWithOnlyNanCounts(PartitionSpec spec) { + DataFiles.Builder builder = + DataFiles.builder(spec) + .withPath("/path/to/data-nan-only.parquet") + .withMetrics( + new Metrics( + 1L, // record count + null, // column sizes + null, // value counts + null, // null value counts (intentionally null) + ImmutableMap.of(3, 0L), // nan value counts present + null, // lower bounds + null // upper bounds + )) + .withFileSizeInBytes(10) + .withRecordCount(1); + + if (spec.isPartitioned()) { + builder.withPartitionPath("data_bucket=1"); + } + + return builder.build(); + } + private static String dataFileJsonWithRequiredOnly(PartitionSpec spec) { if (spec.isUnpartitioned()) { return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\",\"file-format\":\"PARQUET\"," From 3747965e9d750cdf0cba65e5367a75dbc3d132e1 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Tue, 2 Dec 2025 15:11:25 +0900 Subject: [PATCH 024/201] Core: Use `@TempDir` in TestTableMetadataParser (#14732) --- .../apache/iceberg/TestTableMetadataParser.java | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadataParser.java b/core/src/test/java/org/apache/iceberg/TestTableMetadataParser.java index 87c618b9adcd..45333a4dde83 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadataParser.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadataParser.java @@ -29,7 +29,6 @@ import java.io.IOException; import java.io.InputStream; import java.nio.file.Path; -import java.nio.file.Paths; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -39,9 +38,9 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types.BooleanType; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; @ExtendWith(ParameterizedTestExtension.class) public class TestTableMetadataParser { @@ -56,10 +55,10 @@ private static List parameters() { @Parameter private String codecName; @TestTemplate - public void testGzipCompressionProperty() throws IOException { + public void testGzipCompressionProperty(@TempDir Path tempDir) throws IOException { Codec codec = Codec.fromName(codecName); String fileExtension = getFileExtension(codec); - String fileName = "v3" + fileExtension; + String fileName = tempDir + "/v3" + fileExtension; OutputFile outputFile = Files.localOutput(fileName); Map properties = Maps.newHashMap(); properties.put(TableProperties.METADATA_COMPRESSION, codecName); @@ -71,13 +70,6 @@ public void testGzipCompressionProperty() throws IOException { verifyMetadata(metadata, actualMetadata); } - @AfterEach - public void cleanup() throws IOException { - Codec codec = Codec.fromName(codecName); - Path metadataFilePath = Paths.get("v3" + getFileExtension(codec)); - java.nio.file.Files.deleteIfExists(metadataFilePath); - } - private void verifyMetadata(TableMetadata expected, TableMetadata actual) { assertThat(actual.schema().asStruct()).isEqualTo(expected.schema().asStruct()); assertThat(actual.location()).isEqualTo(expected.location()); From 16e84356dae1975fa04d8c3ecce30a90df18ca9f Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 2 Dec 2025 00:46:37 -0800 Subject: [PATCH 025/201] Core: Add UUIDv7 generator (#14700) --- .../org/apache/iceberg/util/UUIDUtil.java | 31 +++++++++++++++++ .../org/apache/iceberg/util/TestUUIDUtil.java | 34 +++++++++++++++++++ 2 files changed, 65 insertions(+) create mode 100644 api/src/test/java/org/apache/iceberg/util/TestUUIDUtil.java diff --git a/api/src/main/java/org/apache/iceberg/util/UUIDUtil.java b/api/src/main/java/org/apache/iceberg/util/UUIDUtil.java index b72feec00b2c..3146a3763cce 100644 --- a/api/src/main/java/org/apache/iceberg/util/UUIDUtil.java +++ b/api/src/main/java/org/apache/iceberg/util/UUIDUtil.java @@ -20,10 +20,13 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.security.SecureRandom; import java.util.UUID; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; public class UUIDUtil { + private static final SecureRandom SECURE_RANDOM = new SecureRandom(); + private UUIDUtil() {} public static UUID convert(byte[] buf) { @@ -78,4 +81,32 @@ public static ByteBuffer convertToByteBuffer(UUID value, ByteBuffer reuse) { buffer.putLong(8, value.getLeastSignificantBits()); return buffer; } + + /** + * Generate a RFC 9562 UUIDv7. + * + *

Layout: - 48-bit Unix epoch milliseconds - 4-bit version (0b0111) - 12-bit random (rand_a) - + * 2-bit variant (RFC 4122, 0b10) - 62-bit random (rand_b) + */ + public static UUID generateUuidV7() { + long epochMs = System.currentTimeMillis(); + Preconditions.checkState( + (epochMs >>> 48) == 0, "Invalid timestamp: does not fit within 48 bits: %s", epochMs); + + // Draw 10 random bytes once: 2 bytes for rand_a (12 bits) and 8 bytes for rand_b (62 bits) + byte[] randomBytes = new byte[10]; + SECURE_RANDOM.nextBytes(randomBytes); + ByteBuffer rb = ByteBuffer.wrap(randomBytes).order(ByteOrder.BIG_ENDIAN); + long randMSB = ((long) rb.getShort()) & 0x0FFFL; // 12 bits + long randLSB = rb.getLong() & 0x3FFFFFFFFFFFFFFFL; // 62 bits + + long msb = (epochMs << 16); // place timestamp in the top 48 bits + msb |= 0x7000L; // version 7 (UUID bits 48..51) + msb |= randMSB; // low 12 bits of MSB + + long lsb = 0x8000000000000000L; // RFC 4122 variant '10' + lsb |= randLSB; + + return new UUID(msb, lsb); + } } diff --git a/api/src/test/java/org/apache/iceberg/util/TestUUIDUtil.java b/api/src/test/java/org/apache/iceberg/util/TestUUIDUtil.java new file mode 100644 index 000000000000..c5f85c2f20b3 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/util/TestUUIDUtil.java @@ -0,0 +1,34 @@ +/* + * 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.util; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.UUID; +import org.junit.jupiter.api.Test; + +public class TestUUIDUtil { + + @Test + public void uuidV7HasVersionAndVariant() { + UUID uuid = UUIDUtil.generateUuidV7(); + assertThat(uuid.version()).isEqualTo(7); + assertThat(uuid.variant()).isEqualTo(2); + } +} From 166a6ebb343cbad85e5cc892c4fe9e880bdc0fb7 Mon Sep 17 00:00:00 2001 From: Xianyang Liu Date: Wed, 3 Dec 2025 00:49:58 +0800 Subject: [PATCH 026/201] Build: Apply spotless for scala code (#8023) --- .baseline/scala/.scala212fmt.conf | 32 +++ .baseline/scala/.scala213fmt.conf | 32 +++ baseline.gradle | 15 ++ site/docs/contribute.md | 2 + .../IcebergSparkSessionExtensions.scala | 7 +- .../AlignRowLevelCommandAssignments.scala | 10 +- .../AlignedRowLevelIcebergCommandCheck.scala | 1 - .../analysis/AssignmentAlignmentSupport.scala | 22 +- .../CheckMergeIntoTableConditions.scala | 9 +- .../sql/catalyst/analysis/CheckViews.scala | 80 ++++--- ...MergeIntoIcebergTableResolutionCheck.scala | 1 - .../analysis/ProcedureArgumentCoercion.scala | 1 - .../ResolveMergeIntoTableReferences.scala | 15 +- .../catalyst/analysis/ResolveProcedures.scala | 10 +- .../sql/catalyst/analysis/ResolveViews.scala | 76 ++++--- .../analysis/RewriteMergeIntoTable.scala | 42 +++- .../RewriteMergeIntoTableForRowLineage.scala | 22 +- .../RewriteOperationForRowLineage.scala | 16 +- .../RewriteRowLevelIcebergCommand.scala | 6 +- .../analysis/RewriteUpdateTable.scala | 6 +- .../RewriteUpdateTableForRowLineage.scala | 7 +- .../analysis/RewriteViewCommands.scala | 69 ++++-- .../sql/catalyst/analysis/ViewUtil.scala | 1 - .../expressions/AssignmentUtils.scala | 24 +- ...endedReplaceNullWithFalseInPredicate.scala | 21 +- ...endedSimplifyConditionalsInPredicate.scala | 31 +-- ...oveRowLineageOutputFromOriginalTable.scala | 12 +- .../optimizer/ReplaceStaticInvoke.scala | 21 +- .../IcebergSparkSqlExtensionsParser.scala | 85 +++---- .../IcebergSqlExtensionsAstBuilder.scala | 215 ++++++++++-------- .../planning/RewrittenRowLevelCommand.scala | 1 - .../plans/logical/AddPartitionField.scala | 4 +- .../plans/logical/BranchOptions.scala | 8 +- .../sql/catalyst/plans/logical/Call.scala | 1 - .../plans/logical/CreateOrReplaceBranch.scala | 4 +- .../plans/logical/CreateOrReplaceTag.scala | 4 +- .../catalyst/plans/logical/DropBranch.scala | 1 - .../plans/logical/DropIdentifierFields.scala | 5 +- .../plans/logical/DropPartitionField.scala | 1 - .../sql/catalyst/plans/logical/DropTag.scala | 1 - .../plans/logical/MergeIntoIcebergTable.scala | 7 +- .../catalyst/plans/logical/MergeRows.scala | 4 +- .../plans/logical/NoStatsUnaryNode.scala | 1 - .../plans/logical/ReplaceIcebergData.scala | 11 +- .../plans/logical/ReplacePartitionField.scala | 6 +- .../plans/logical/RowLevelCommand.scala | 1 - .../plans/logical/SetIdentifierFields.scala | 5 +- .../catalyst/plans/logical/TagOptions.scala | 1 - .../UnresolvedMergeIntoIcebergTable.scala | 8 +- .../plans/logical/UpdateIcebergTable.scala | 4 +- .../catalyst/plans/logical/UpdateRows.scala | 4 +- .../plans/logical/V2WriteCommandLike.scala | 1 - .../plans/logical/WriteIcebergDelta.scala | 22 +- .../catalyst/plans/logical/statements.scala | 1 - .../logical/views/CreateIcebergView.scala | 27 +-- .../plans/logical/views/DropIcebergView.scala | 5 +- .../plans/logical/views/ResolvedV2View.scala | 6 +- .../logical/views/ShowIcebergViews.scala | 9 +- .../v2/AddPartitionFieldExec.scala | 14 +- .../v2/AlterV2ViewSetPropertiesExec.scala | 13 +- .../v2/AlterV2ViewUnsetPropertiesExec.scala | 11 +- .../execution/datasources/v2/CallExec.scala | 7 +- .../v2/CreateOrReplaceBranchExec.scala | 25 +- .../v2/CreateOrReplaceTagExec.scala | 10 +- .../datasources/v2/CreateV2ViewExec.scala | 45 ++-- .../datasources/v2/DescribeV2ViewExec.scala | 30 +-- .../datasources/v2/DropBranchExec.scala | 4 +- .../v2/DropIdentifierFieldsExec.scala | 25 +- .../v2/DropPartitionFieldExec.scala | 27 +-- .../datasources/v2/DropTagExec.scala | 8 +- .../datasources/v2/DropV2ViewExec.scala | 8 +- .../v2/ExtendedDataSourceV2Implicits.scala | 1 - .../v2/ExtendedDataSourceV2Strategy.scala | 89 ++++++-- .../datasources/v2/ExtendedV2Writes.scala | 1 - .../datasources/v2/MergeRowsExec.scala | 30 ++- .../datasources/v2/RenameV2ViewExec.scala | 9 +- .../datasources/v2/ReplaceDataExec.scala | 7 +- .../v2/ReplacePartitionFieldExec.scala | 30 +-- .../v2/ReplaceRewrittenRowLevelCommand.scala | 1 - .../RowLevelCommandScanRelationPushDown.scala | 30 ++- .../v2/SetIdentifierFieldsExec.scala | 13 +- .../SetWriteDistributionAndOrderingExec.scala | 18 +- .../datasources/v2/ShowCreateV2ViewExec.scala | 17 +- .../v2/ShowV2ViewPropertiesExec.scala | 16 +- .../datasources/v2/ShowV2ViewsExec.scala | 17 +- .../datasources/v2/UpdateRowsExec.scala | 6 +- .../RowLevelCommandDynamicPruning.scala | 30 +-- .../plans/logical/OrderAwareCoalesce.scala | 7 +- .../SetWriteDistributionAndOrdering.scala | 12 +- .../plans/logical/SortOrderParserUtil.scala | 5 +- .../spark/sql/catalyst/utils/PlanUtils.scala | 1 - .../execution/OrderAwareCoalesceExec.scala | 4 +- .../SparkExpressionConverter.scala | 27 ++- .../spark/sql/stats/ThetaSketchAgg.scala | 13 +- .../IcebergSparkSessionExtensions.scala | 7 +- .../sql/catalyst/analysis/CheckViews.scala | 54 +++-- .../analysis/ProcedureArgumentCoercion.scala | 1 - .../catalyst/analysis/ResolveProcedures.scala | 10 +- .../sql/catalyst/analysis/ResolveViews.scala | 77 ++++--- .../RewriteMergeIntoTableForRowLineage.scala | 21 +- .../RewriteOperationForRowLineage.scala | 16 +- .../RewriteUpdateTableForRowLineage.scala | 7 +- .../analysis/RewriteViewCommands.scala | 69 ++++-- .../sql/catalyst/analysis/ViewUtil.scala | 1 - ...oveRowLineageOutputFromOriginalTable.scala | 12 +- .../optimizer/ReplaceStaticInvoke.scala | 21 +- .../IcebergSparkSqlExtensionsParser.scala | 72 +++--- .../IcebergSqlExtensionsAstBuilder.scala | 215 ++++++++++-------- .../plans/logical/AddPartitionField.scala | 4 +- .../plans/logical/BranchOptions.scala | 8 +- .../sql/catalyst/plans/logical/Call.scala | 1 - .../plans/logical/CreateOrReplaceBranch.scala | 4 +- .../plans/logical/CreateOrReplaceTag.scala | 4 +- .../catalyst/plans/logical/DropBranch.scala | 1 - .../plans/logical/DropIdentifierFields.scala | 5 +- .../plans/logical/DropPartitionField.scala | 1 - .../sql/catalyst/plans/logical/DropTag.scala | 1 - .../plans/logical/ReplacePartitionField.scala | 6 +- .../plans/logical/SetIdentifierFields.scala | 5 +- .../catalyst/plans/logical/TagOptions.scala | 1 - .../catalyst/plans/logical/statements.scala | 1 - .../logical/views/CreateIcebergView.scala | 27 +-- .../plans/logical/views/DropIcebergView.scala | 5 +- .../plans/logical/views/ResolvedV2View.scala | 6 +- .../logical/views/ShowIcebergViews.scala | 9 +- .../v2/AddPartitionFieldExec.scala | 14 +- .../v2/AlterV2ViewSetPropertiesExec.scala | 13 +- .../v2/AlterV2ViewUnsetPropertiesExec.scala | 11 +- .../execution/datasources/v2/CallExec.scala | 7 +- .../v2/CreateOrReplaceBranchExec.scala | 25 +- .../v2/CreateOrReplaceTagExec.scala | 10 +- .../datasources/v2/CreateV2ViewExec.scala | 45 ++-- .../datasources/v2/DescribeV2ViewExec.scala | 30 +-- .../datasources/v2/DropBranchExec.scala | 4 +- .../v2/DropIdentifierFieldsExec.scala | 25 +- .../v2/DropPartitionFieldExec.scala | 27 +-- .../datasources/v2/DropTagExec.scala | 8 +- .../datasources/v2/DropV2ViewExec.scala | 8 +- .../v2/ExtendedDataSourceV2Strategy.scala | 50 +++- .../datasources/v2/RenameV2ViewExec.scala | 9 +- .../v2/ReplacePartitionFieldExec.scala | 30 +-- .../v2/SetIdentifierFieldsExec.scala | 13 +- .../SetWriteDistributionAndOrderingExec.scala | 18 +- .../datasources/v2/ShowCreateV2ViewExec.scala | 17 +- .../v2/ShowV2ViewPropertiesExec.scala | 16 +- .../datasources/v2/ShowV2ViewsExec.scala | 17 +- .../plans/logical/OrderAwareCoalesce.scala | 7 +- .../SetWriteDistributionAndOrdering.scala | 12 +- .../plans/logical/SortOrderParserUtil.scala | 5 +- .../spark/sql/catalyst/utils/PlanUtils.scala | 1 - .../execution/OrderAwareCoalesceExec.scala | 4 +- .../SparkExpressionConverter.scala | 27 ++- .../spark/sql/stats/ThetaSketchAgg.scala | 13 +- .../IcebergSparkSessionExtensions.scala | 1 - .../sql/catalyst/analysis/CheckViews.scala | 51 +++-- .../sql/catalyst/analysis/ResolveViews.scala | 77 ++++--- .../analysis/RewriteViewCommands.scala | 71 ++++-- .../sql/catalyst/analysis/ViewUtil.scala | 1 - .../optimizer/ReplaceStaticInvoke.scala | 21 +- .../IcebergSparkSqlExtensionsParser.scala | 73 +++--- .../IcebergSqlExtensionsAstBuilder.scala | 206 +++++++++-------- .../plans/logical/AddPartitionField.scala | 4 +- .../plans/logical/BranchOptions.scala | 8 +- .../plans/logical/CreateOrReplaceBranch.scala | 4 +- .../plans/logical/CreateOrReplaceTag.scala | 4 +- .../catalyst/plans/logical/DropBranch.scala | 1 - .../plans/logical/DropIdentifierFields.scala | 5 +- .../plans/logical/DropPartitionField.scala | 1 - .../sql/catalyst/plans/logical/DropTag.scala | 1 - .../plans/logical/ReplacePartitionField.scala | 6 +- .../plans/logical/SetIdentifierFields.scala | 5 +- .../catalyst/plans/logical/TagOptions.scala | 1 - .../logical/views/CreateIcebergView.scala | 27 +-- .../plans/logical/views/DropIcebergView.scala | 5 +- .../plans/logical/views/ResolvedV2View.scala | 6 +- .../logical/views/ShowIcebergViews.scala | 9 +- .../v2/AddPartitionFieldExec.scala | 14 +- .../v2/AlterV2ViewSetPropertiesExec.scala | 13 +- .../v2/AlterV2ViewUnsetPropertiesExec.scala | 11 +- .../v2/CreateOrReplaceBranchExec.scala | 25 +- .../v2/CreateOrReplaceTagExec.scala | 10 +- .../datasources/v2/CreateV2ViewExec.scala | 45 ++-- .../datasources/v2/DescribeV2ViewExec.scala | 30 +-- .../datasources/v2/DropBranchExec.scala | 4 +- .../v2/DropIdentifierFieldsExec.scala | 25 +- .../v2/DropPartitionFieldExec.scala | 27 +-- .../datasources/v2/DropTagExec.scala | 8 +- .../datasources/v2/DropV2ViewExec.scala | 8 +- .../v2/ExtendedDataSourceV2Strategy.scala | 50 +++- .../datasources/v2/RenameV2ViewExec.scala | 9 +- .../v2/ReplacePartitionFieldExec.scala | 30 +-- .../v2/SetIdentifierFieldsExec.scala | 13 +- .../SetWriteDistributionAndOrderingExec.scala | 18 +- .../datasources/v2/ShowCreateV2ViewExec.scala | 17 +- .../v2/ShowV2ViewPropertiesExec.scala | 16 +- .../datasources/v2/ShowV2ViewsExec.scala | 17 +- .../plans/logical/OrderAwareCoalesce.scala | 7 +- .../SetWriteDistributionAndOrdering.scala | 12 +- .../plans/logical/SortOrderParserUtil.scala | 5 +- .../spark/sql/catalyst/utils/PlanUtils.scala | 1 - .../execution/OrderAwareCoalesceExec.scala | 4 +- .../SparkExpressionConverter.scala | 27 ++- .../spark/sql/stats/ThetaSketchAgg.scala | 13 +- 203 files changed, 2168 insertions(+), 1672 deletions(-) create mode 100644 .baseline/scala/.scala212fmt.conf create mode 100644 .baseline/scala/.scala213fmt.conf diff --git a/.baseline/scala/.scala212fmt.conf b/.baseline/scala/.scala212fmt.conf new file mode 100644 index 000000000000..485a11d6ff17 --- /dev/null +++ b/.baseline/scala/.scala212fmt.conf @@ -0,0 +1,32 @@ +# 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. + +version = 3.9.7 + +align = none +align.openParenDefnSite = false +align.openParenCallSite = false +align.tokens = [] +importSelectors = "singleLine" +optIn = { + configStyleArguments = false +} +danglingParentheses.preset = false +docstrings.style = Asterisk +docstrings.wrap = false +maxColumn = 100 +runner.dialect = scala212 diff --git a/.baseline/scala/.scala213fmt.conf b/.baseline/scala/.scala213fmt.conf new file mode 100644 index 000000000000..57c44a61c46e --- /dev/null +++ b/.baseline/scala/.scala213fmt.conf @@ -0,0 +1,32 @@ +# 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. + +version = 3.9.7 + +align = none +align.openParenDefnSite = false +align.openParenCallSite = false +align.tokens = [] +importSelectors = "singleLine" +optIn = { + configStyleArguments = false +} +danglingParentheses.preset = false +docstrings.style = Asterisk +docstrings.wrap = false +maxColumn = 100 +runner.dialect = scala213 diff --git a/baseline.gradle b/baseline.gradle index 151fa40a1a28..54e67b84f853 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -62,6 +62,21 @@ subprojects { removeUnusedImports() licenseHeaderFile "$rootDir/.baseline/copyright/copyright-header-java.txt" } + + // Configure different scalafmt rules for specific Scala version + if (project.name.startsWith("iceberg-spark") && project.name.endsWith("2.13")) { + scala { + target 'src/**/*.scala' + scalafmt("3.9.7").configFile("$rootDir/.baseline/scala/.scala213fmt.conf") + licenseHeaderFile "$rootDir/.baseline/copyright/copyright-header-java.txt", "package" + } + } else if (project.name.startsWith("iceberg-spark") && project.name.endsWith("2.12")) { + scala { + target 'src/**/*.scala' + scalafmt("3.9.7").configFile("$rootDir/.baseline/scala/.scala212fmt.conf") + licenseHeaderFile "$rootDir/.baseline/copyright/copyright-header-java.txt", "package" + } + } } } diff --git a/site/docs/contribute.md b/site/docs/contribute.md index d38cc28c366c..54aa076639ce 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -125,6 +125,8 @@ This project Iceberg also has modules for adding Iceberg support to processing e Follow the instructions for [Eclipse](https://github.com/google/google-java-format#eclipse) or [IntelliJ](https://github.com/google/google-java-format#intellij-android-studio-and-other-jetbrains-ides) to install the **google-java-format** plugin (note the required manual actions for IntelliJ). +Follow the [instructions](https://scalameta.org/scalafmt/docs/installation.html) to install **scalafmt** plugin +and configure it to point to the configuration file located under the directory `.baseline/scala/`. ## Semantic Versioning diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala index 350ed6988f2d..c1f77da66f9f 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.iceberg.spark.extensions import org.apache.spark.sql.SparkSessionExtensions @@ -57,8 +56,8 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) { extensions.injectResolutionRule { _ => CheckMergeIntoTableConditions } extensions.injectResolutionRule { _ => ProcedureArgumentCoercion } extensions.injectResolutionRule { _ => AlignRowLevelCommandAssignments } - extensions.injectResolutionRule { _ => RewriteUpdateTableForRowLineage} - extensions.injectResolutionRule { _ => RewriteMergeIntoTableForRowLineage} + extensions.injectResolutionRule { _ => RewriteUpdateTableForRowLineage } + extensions.injectResolutionRule { _ => RewriteMergeIntoTableForRowLineage } extensions.injectResolutionRule { _ => RewriteUpdateTable } extensions.injectResolutionRule { _ => RewriteMergeIntoTable } extensions.injectCheckRule { _ => CheckViews } @@ -69,7 +68,7 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) { extensions.injectOptimizerRule { _ => ExtendedSimplifyConditionalsInPredicate } extensions.injectOptimizerRule { _ => ExtendedReplaceNullWithFalseInPredicate } extensions.injectOptimizerRule { _ => ReplaceStaticInvoke } - extensions.injectOptimizerRule { _ => RemoveRowLineageOutputFromOriginalTable} + extensions.injectOptimizerRule { _ => RemoveRowLineageOutputFromOriginalTable } // pre-CBO rules run only once and the order of the rules is important // - dynamic filters should be added before replacing commands with rewrite plans // - scans must be planned before building writes diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignRowLevelCommandAssignments.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignRowLevelCommandAssignments.scala index 23ba50bdfd06..c70ded5ed283 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignRowLevelCommandAssignments.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignRowLevelCommandAssignments.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException @@ -35,8 +34,7 @@ import org.apache.spark.sql.catalyst.rules.Rule * * Note that this rule must be run before rewriting row-level commands. */ -object AlignRowLevelCommandAssignments - extends Rule[LogicalPlan] with AssignmentAlignmentSupport { +object AlignRowLevelCommandAssignments extends Rule[LogicalPlan] with AssignmentAlignmentSupport { override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case u: UpdateIcebergTable if u.resolved && !u.aligned => @@ -60,7 +58,7 @@ object AlignRowLevelCommandAssignments if (ref.size > 1) { throw new AnalysisException( "Nested fields are not supported inside INSERT clauses of MERGE operations: " + - s"${ref.mkString("`", "`.`", "`")}") + s"${ref.mkString("`", "`.`", "`")}") } } @@ -101,8 +99,8 @@ object AlignRowLevelCommandAssignments if (assignment.isEmpty) { throw new AnalysisException( s"Cannot find column '${targetAttr.name}' of the target table among " + - s"the INSERT columns: ${assignmentMap.keys.mkString(", ")}. " + - "INSERT clauses must provide values for all columns of the target table.") + s"the INSERT columns: ${assignmentMap.keys.mkString(", ")}. " + + "INSERT clauses must provide values for all columns of the target table.") } val key = assignment.get.key diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignedRowLevelIcebergCommandCheck.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignedRowLevelIcebergCommandCheck.scala index d915e4f10949..38021fda29f4 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignedRowLevelIcebergCommandCheck.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlignedRowLevelIcebergCommandCheck.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala index 76aec46a23b5..0b53075a73ed 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/AssignmentAlignmentSupport.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException @@ -69,8 +68,8 @@ trait AssignmentAlignmentSupport extends CastSupport { val columnUpdates = assignments.map(a => ColumnUpdate(toAssignmentRef(a.key), a.value)) val outputExprs = applyUpdates(table.output, columnUpdates) - outputExprs.zip(table.output).map { - case (expr, attr) => handleCharVarcharLimits(Assignment(attr, expr)) + outputExprs.zip(table.output).map { case (expr, attr) => + handleCharVarcharLimits(Assignment(attr, expr)) } } @@ -117,8 +116,7 @@ trait AssignmentAlignmentSupport extends CastSupport { val colName = (namePrefix :+ col.name).mkString(".") throw new AnalysisException( "Updating nested fields is only supported for StructType " + - s"but $colName is of type $otherType" - ) + s"but $colName is of type $otherType") } // if there are conflicting updates, throw an exception @@ -129,7 +127,7 @@ trait AssignmentAlignmentSupport extends CastSupport { val conflictingCols = updates.map(u => (namePrefix ++ u.ref).mkString(".")) throw new AnalysisException( "Updates are in conflict for these columns: " + - conflictingCols.distinct.mkString(", ")) + conflictingCols.distinct.mkString(", ")) } } } @@ -180,8 +178,13 @@ trait AssignmentAlignmentSupport extends CastSupport { // e.g. a struct with fields (a, b) is assigned as a struct with fields (a, c) or (b, a) val errors = new mutable.ArrayBuffer[String]() val canWrite = DataType.canWrite( - expr.dataType, tableAttr.dataType, byName = true, resolver, tableAttr.name, - storeAssignmentPolicy, err => errors += err) + expr.dataType, + tableAttr.dataType, + byName = true, + resolver, + tableAttr.name, + storeAssignmentPolicy, + err => errors += err) if (!canWrite) { throw new AnalysisException( @@ -195,7 +198,8 @@ trait AssignmentAlignmentSupport extends CastSupport { case _ if tableAttr.dataType.sameType(expr.dataType) => expr case StoreAssignmentPolicy.ANSI => - val cast = Cast(expr, tableAttr.dataType, Option(conf.sessionLocalTimeZone), ansiEnabled = true) + val cast = + Cast(expr, tableAttr.dataType, Option(conf.sessionLocalTimeZone), ansiEnabled = true) cast.setTagValue(Cast.BY_TABLE_INSERTION, ()) TableOutputResolver.checkCastOverflowInTableInsert(cast, colPath.quoted) case _ => diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckMergeIntoTableConditions.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckMergeIntoTableConditions.scala index 70f6694af60b..079a0eee234d 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckMergeIntoTableConditions.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckMergeIntoTableConditions.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException @@ -57,18 +56,20 @@ object CheckMergeIntoTableConditions extends Rule[LogicalPlan] { if (!cond.deterministic) { throw new AnalysisException( s"Non-deterministic functions are not supported in $condName conditions of " + - s"MERGE operations: ${cond.sql}") + s"MERGE operations: ${cond.sql}") } if (SubqueryExpression.hasSubquery(cond)) { throw new AnalysisException( s"Subqueries are not supported in conditions of MERGE operations. " + - s"Found a subquery in the $condName condition: ${cond.sql}") + s"Found a subquery in the $condName condition: ${cond.sql}") } if (cond.find(_.isInstanceOf[AggregateExpression]).isDefined) { throw new AnalysisException( - s"Agg functions are not supported in $condName conditions of MERGE operations: " + {cond.sql}) + s"Agg functions are not supported in $condName conditions of MERGE operations: " + { + cond.sql + }) } } } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala index 44b22e68d076..a7c903dfdbc4 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException @@ -38,44 +37,71 @@ object CheckViews extends (LogicalPlan => Unit) { override def apply(plan: LogicalPlan): Unit = { plan foreach { - case CreateIcebergView(resolvedIdent@ResolvedIdentifier(_: ViewCatalog, _), _, query, columnAliases, _, - _, _, _, _, replace, _) => + case CreateIcebergView( + resolvedIdent @ ResolvedIdentifier(_: ViewCatalog, _), + _, + query, + columnAliases, + _, + _, + _, + _, + _, + replace, + _) => verifyColumnCount(resolvedIdent, columnAliases, query) - SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames.toIndexedSeq, SQLConf.get.resolver) + SchemaUtils.checkColumnNameDuplication( + query.schema.fieldNames.toIndexedSeq, + SQLConf.get.resolver) if (replace) { - val viewIdent: Seq[String] = resolvedIdent.catalog.name() +: resolvedIdent.identifier.asMultipartIdentifier + val viewIdent: Seq[String] = + resolvedIdent.catalog.name() +: resolvedIdent.identifier.asMultipartIdentifier checkCyclicViewReference(viewIdent, query, Seq(viewIdent)) } case AlterViewAs(ResolvedV2View(_, _), _, _) => - throw new AnalysisException("ALTER VIEW AS is not supported. Use CREATE OR REPLACE VIEW instead") + throw new AnalysisException( + "ALTER VIEW AS is not supported. Use CREATE OR REPLACE VIEW instead") case _ => // OK } } - private def verifyColumnCount(ident: ResolvedIdentifier, columns: Seq[String], query: LogicalPlan): Unit = { + private def verifyColumnCount( + ident: ResolvedIdentifier, + columns: Seq[String], + query: LogicalPlan): Unit = { if (columns.nonEmpty) { if (columns.length > query.output.length) { - throw new AnalysisException(String.format("Cannot create view %s.%s, the reason is not enough data columns:\n" + - "View columns: %s\n" + - "Data columns: %s", ident.catalog.name(), ident.identifier, columns.mkString(", "), - query.output.map(c => c.name).mkString(", "))) + throw new AnalysisException( + String.format( + "Cannot create view %s.%s, the reason is not enough data columns:\n" + + "View columns: %s\n" + + "Data columns: %s", + ident.catalog.name(), + ident.identifier, + columns.mkString(", "), + query.output.map(c => c.name).mkString(", "))) } else if (columns.length < query.output.length) { - throw new AnalysisException(String.format("Cannot create view %s.%s, the reason is too many data columns:\n" + - "View columns: %s\n" + - "Data columns: %s", ident.catalog.name(), ident.identifier, columns.mkString(", "), - query.output.map(c => c.name).mkString(", "))) + throw new AnalysisException( + String.format( + "Cannot create view %s.%s, the reason is too many data columns:\n" + + "View columns: %s\n" + + "Data columns: %s", + ident.catalog.name(), + ident.identifier, + columns.mkString(", "), + query.output.map(c => c.name).mkString(", "))) } } } private def checkCyclicViewReference( - viewIdent: Seq[String], - plan: LogicalPlan, - cyclePath: Seq[Seq[String]]): Unit = { + viewIdent: Seq[String], + plan: LogicalPlan, + cyclePath: Seq[Seq[String]]): Unit = { plan match { - case sub@SubqueryAlias(_, Project(_, _)) => + case sub @ SubqueryAlias(_, Project(_, _)) => val currentViewIdent: Seq[String] = sub.identifier.qualifier :+ sub.identifier.name checkIfRecursiveView(viewIdent, currentViewIdent, cyclePath, sub.children) case v1View: View => @@ -94,15 +120,17 @@ object CheckViews extends (LogicalPlan => Unit) { } private def checkIfRecursiveView( - viewIdent: Seq[String], - currentViewIdent: Seq[String], - cyclePath: Seq[Seq[String]], - children: Seq[LogicalPlan] - ): Unit = { + viewIdent: Seq[String], + currentViewIdent: Seq[String], + cyclePath: Seq[Seq[String]], + children: Seq[LogicalPlan]): Unit = { val newCyclePath = cyclePath :+ currentViewIdent if (currentViewIdent == viewIdent) { - throw new AnalysisException(String.format("Recursive cycle in view detected: %s (cycle: %s)", - viewIdent.asIdentifier, newCyclePath.map(p => p.mkString(".")).mkString(" -> "))) + throw new AnalysisException( + String.format( + "Recursive cycle in view detected: %s (cycle: %s)", + viewIdent.asIdentifier, + newCyclePath.map(p => p.mkString(".")).mkString(" -> "))) } else { children.foreach { c => checkCyclicViewReference(viewIdent, c, newCyclePath) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/MergeIntoIcebergTableResolutionCheck.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/MergeIntoIcebergTableResolutionCheck.scala index b3a9bda280d2..3cbf27e6453b 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/MergeIntoIcebergTableResolutionCheck.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/MergeIntoIcebergTableResolutionCheck.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala index 7f0ca8fadded..aadc332181e2 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoTableReferences.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoTableReferences.scala index bb270391a170..0c64636a000d 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoTableReferences.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveMergeIntoTableReferences.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException @@ -53,7 +52,8 @@ case class ResolveMergeIntoTableReferences(spark: SparkSession) extends Rule[Log case UpdateAction(cond, assignments) => val resolvedCond = cond.map(resolveCond("UPDATE", _, m)) // the update action can access columns from both target and source tables - val resolvedAssignments = resolveAssignments(assignments, m, resolveValuesWithSourceOnly = false) + val resolvedAssignments = + resolveAssignments(assignments, m, resolveValuesWithSourceOnly = false) UpdateAction(resolvedCond, resolvedAssignments) case UpdateStarAction(updateCondition) => @@ -62,7 +62,8 @@ case class ResolveMergeIntoTableReferences(spark: SparkSession) extends Rule[Log Assignment(attr, UnresolvedAttribute(Seq(attr.name))) } // for UPDATE *, the value must be from the source table - val resolvedAssignments = resolveAssignments(assignments, m, resolveValuesWithSourceOnly = true) + val resolvedAssignments = + resolveAssignments(assignments, m, resolveValuesWithSourceOnly = true) UpdateAction(resolvedUpdateCondition, resolvedAssignments) case _ => @@ -74,7 +75,8 @@ case class ResolveMergeIntoTableReferences(spark: SparkSession) extends Rule[Log // the insert action is used when not matched, so its condition and value can only // access columns from the source table val resolvedCond = cond.map(resolveCond("INSERT", _, Project(Nil, m.sourceTable))) - val resolvedAssignments = resolveAssignments(assignments, m, resolveValuesWithSourceOnly = true) + val resolvedAssignments = + resolveAssignments(assignments, m, resolveValuesWithSourceOnly = true) InsertAction(resolvedCond, resolvedAssignments) case InsertStarAction(cond) => @@ -84,7 +86,8 @@ case class ResolveMergeIntoTableReferences(spark: SparkSession) extends Rule[Log val assignments = targetTable.output.map { attr => Assignment(attr, UnresolvedAttribute(Seq(attr.name))) } - val resolvedAssignments = resolveAssignments(assignments, m, resolveValuesWithSourceOnly = true) + val resolvedAssignments = + resolveAssignments(assignments, m, resolveValuesWithSourceOnly = true) InsertAction(resolvedCond, resolvedAssignments) case _ => @@ -108,7 +111,7 @@ case class ResolveMergeIntoTableReferences(spark: SparkSession) extends Rule[Log if (unresolvedAttrs.nonEmpty) { throw new AnalysisException( s"Cannot resolve ${unresolvedAttrs.map(_.sql).mkString("[", ",", "]")} in $condName condition " + - s"of MERGE operation given input columns: ${plan.inputSet.toSeq.map(_.sql).mkString("[", ",", "]")}") + s"of MERGE operation given input columns: ${plan.inputSet.toSeq.map(_.sql).mkString("[", ",", "]")}") } resolvedCond diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala index ee69b5e344f0..c2d13bfa021e 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import java.util.Locale @@ -61,7 +60,8 @@ case class ResolveProcedures(spark: SparkSession) extends Rule[LogicalPlan] with } if (duplicateParamNames.nonEmpty) { - throw new AnalysisException(s"Duplicate parameter names: ${duplicateParamNames.mkString("[", ",", "]")}") + throw new AnalysisException( + s"Duplicate parameter names: ${duplicateParamNames.mkString("[", ",", "]")}") } // optional params should be at the end @@ -89,7 +89,8 @@ case class ResolveProcedures(spark: SparkSession) extends Rule[LogicalPlan] with } if (missingParamNames.nonEmpty) { - throw new AnalysisException(s"Missing required parameters: ${missingParamNames.mkString("[", ",", "]")}") + throw new AnalysisException( + s"Missing required parameters: ${missingParamNames.mkString("[", ",", "]")}") } val argExprs = new Array[Expression](params.size) @@ -141,7 +142,8 @@ case class ResolveProcedures(spark: SparkSession) extends Rule[LogicalPlan] with } if (validationErrors.nonEmpty) { - throw new AnalysisException(s"Could not build name to arg map: ${validationErrors.mkString(", ")}") + throw new AnalysisException( + s"Could not build name to arg map: ${validationErrors.mkString(", ")}") } namedArgs.map(arg => arg.name -> arg).toMap diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala index b77cc033268d..1e0e5e8951ba 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.SparkSession @@ -47,36 +46,52 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case u@UnresolvedRelation(nameParts, _, _) - if catalogManager.v1SessionCatalog.isTempView(nameParts) => + case u @ UnresolvedRelation(nameParts, _, _) + if catalogManager.v1SessionCatalog.isTempView(nameParts) => u - case u@UnresolvedRelation(parts@CatalogAndIdentifier(catalog, ident), _, _) => - ViewUtil.loadView(catalog, ident) + case u @ UnresolvedRelation(parts @ CatalogAndIdentifier(catalog, ident), _, _) => + ViewUtil + .loadView(catalog, ident) .map(createViewRelation(parts, _)) .getOrElse(u) - case u@UnresolvedTableOrView(CatalogAndIdentifier(catalog, ident), _, _) => - ViewUtil.loadView(catalog, ident) + case u @ UnresolvedTableOrView(CatalogAndIdentifier(catalog, ident), _, _) => + ViewUtil + .loadView(catalog, ident) .map(_ => ResolvedV2View(catalog.asViewCatalog, ident)) .getOrElse(u) - case c@CreateIcebergView(ResolvedIdentifier(_, _), _, query, columnAliases, columnComments, _, _, _, _, _, _) - if query.resolved && !c.rewritten => + case c @ CreateIcebergView( + ResolvedIdentifier(_, _), + _, + query, + columnAliases, + columnComments, + _, + _, + _, + _, + _, + _) if query.resolved && !c.rewritten => val aliased = aliasColumns(query, columnAliases, columnComments) - c.copy(query = aliased, queryColumnNames = query.schema.fieldNames.toIndexedSeq, rewritten = true) + c.copy( + query = aliased, + queryColumnNames = query.schema.fieldNames.toIndexedSeq, + rewritten = true) } private def aliasColumns( - plan: LogicalPlan, - columnAliases: Seq[String], - columnComments: Seq[Option[String]]): LogicalPlan = { + plan: LogicalPlan, + columnAliases: Seq[String], + columnComments: Seq[Option[String]]): LogicalPlan = { if (columnAliases.isEmpty || columnAliases.length != plan.output.length) { plan } else { val projectList = plan.output.zipWithIndex.map { case (attr, pos) => if (columnComments.apply(pos).isDefined) { - val meta = new MetadataBuilder().putString("comment", columnComments.apply(pos).get).build() + val meta = + new MetadataBuilder().putString("comment", columnComments.apply(pos).get).build() Alias(attr, columnAliases.apply(pos))(explicitMetadata = Some(meta)) } else { Alias(attr, columnAliases.apply(pos))() @@ -98,17 +113,15 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look // This is more strict because it doesn't allow resolution by field name. val aliases = view.schema.fields.zipWithIndex.map { case (expected, pos) => val attr = GetColumnByOrdinal(pos, expected.dataType) - Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = Some(expected.metadata)) + Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = + Some(expected.metadata)) }.toIndexedSeq SubqueryAlias(nameParts, Project(aliases, rewritten)) } private def parseViewText(name: String, viewText: String): LogicalPlan = { - val origin = Origin( - objectType = Some("VIEW"), - objectName = Some(name) - ) + val origin = Origin(objectType = Some("VIEW"), objectName = Some(name)) try { CurrentOrigin.withOrigin(origin) { @@ -121,8 +134,8 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look } private def rewriteIdentifiers( - plan: LogicalPlan, - catalogAndNamespace: Seq[String]): LogicalPlan = { + plan: LogicalPlan, + catalogAndNamespace: Seq[String]): LogicalPlan = { // Substitute CTEs and Unresolved Ordinals within the view, then rewrite unresolved functions and relations qualifyTableIdentifiers( qualifyFunctionIdentifiers( @@ -132,15 +145,15 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look } private def qualifyFunctionIdentifiers( - plan: LogicalPlan, - catalogAndNamespace: Seq[String]): LogicalPlan = plan transformExpressions { - case u@UnresolvedFunction(Seq(name), _, _, _, _) => + plan: LogicalPlan, + catalogAndNamespace: Seq[String]): LogicalPlan = plan transformExpressions { + case u @ UnresolvedFunction(Seq(name), _, _, _, _) => if (!isBuiltinFunction(name)) { u.copy(nameParts = catalogAndNamespace :+ name) } else { u } - case u@UnresolvedFunction(parts, _, _, _, _) if !isCatalog(parts.head) => + case u @ UnresolvedFunction(parts, _, _, _, _) if !isCatalog(parts.head) => u.copy(nameParts = catalogAndNamespace.head +: parts) } @@ -148,17 +161,16 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look * Qualify table identifiers with default catalog and namespace if necessary. */ private def qualifyTableIdentifiers( - child: LogicalPlan, - catalogAndNamespace: Seq[String]): LogicalPlan = + child: LogicalPlan, + catalogAndNamespace: Seq[String]): LogicalPlan = child transform { - case u@UnresolvedRelation(Seq(table), _, _) => + case u @ UnresolvedRelation(Seq(table), _, _) => u.copy(multipartIdentifier = catalogAndNamespace :+ table) - case u@UnresolvedRelation(parts, _, _) if !isCatalog(parts.head) => + case u @ UnresolvedRelation(parts, _, _) if !isCatalog(parts.head) => u.copy(multipartIdentifier = catalogAndNamespace.head +: parts) case other => - other.transformExpressions { - case subquery: SubqueryExpression => - subquery.withNewPlan(qualifyTableIdentifiers(subquery.plan, catalogAndNamespace)) + other.transformExpressions { case subquery: SubqueryExpression => + subquery.withNewPlan(qualifyTableIdentifiers(subquery.plan, catalogAndNamespace)) } } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala index 4aae44ecf16c..7ac9a1c6c856 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException @@ -82,7 +81,13 @@ object RewriteMergeIntoTable extends RewriteRowLevelIcebergCommand with Predicat private final val ROW_FROM_TARGET_REF = FieldReference(ROW_FROM_TARGET) override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case m @ MergeIntoIcebergTable(aliasedTable, source, cond, matchedActions, notMatchedActions, None) + case m @ MergeIntoIcebergTable( + aliasedTable, + source, + cond, + matchedActions, + notMatchedActions, + None) if m.resolved && m.aligned && matchedActions.isEmpty && notMatchedActions.size == 1 => EliminateSubqueryAliases(aliasedTable) match { @@ -112,8 +117,13 @@ object RewriteMergeIntoTable extends RewriteRowLevelIcebergCommand with Predicat throw new AnalysisException(s"$p is not an Iceberg table") } - case m @ MergeIntoIcebergTable(aliasedTable, source, cond, matchedActions, notMatchedActions, None) - if m.resolved && m.aligned && matchedActions.isEmpty => + case m @ MergeIntoIcebergTable( + aliasedTable, + source, + cond, + matchedActions, + notMatchedActions, + None) if m.resolved && m.aligned && matchedActions.isEmpty => EliminateSubqueryAliases(aliasedTable) match { case r: DataSourceV2Relation => @@ -145,8 +155,13 @@ object RewriteMergeIntoTable extends RewriteRowLevelIcebergCommand with Predicat throw new AnalysisException(s"$p is not an Iceberg table") } - case m @ MergeIntoIcebergTable(aliasedTable, source, cond, matchedActions, notMatchedActions, None) - if m.resolved && m.aligned => + case m @ MergeIntoIcebergTable( + aliasedTable, + source, + cond, + matchedActions, + notMatchedActions, + None) if m.resolved && m.aligned => EliminateSubqueryAliases(aliasedTable) match { case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, _, _) => @@ -204,7 +219,8 @@ object RewriteMergeIntoTable extends RewriteRowLevelIcebergCommand with Predicat // disable broadcasts for the target table to perform the cardinality check val joinType = if (notMatchedActions.isEmpty) LeftOuter else FullOuter val joinHint = JoinHint(leftHint = Some(HintInfo(Some(NO_BROADCAST_HASH))), rightHint = None) - val joinPlan = Join(NoStatsUnaryNode(targetTableProj), sourceTableProj, joinType, Some(cond), joinHint) + val joinPlan = + Join(NoStatsUnaryNode(targetTableProj), sourceTableProj, joinType, Some(cond), joinHint) val matchedConditions = matchedActions.map(actionCondition) val matchedOutputs = matchedActions.map(matchedActionOutput(_, metadataAttrs)) @@ -217,7 +233,8 @@ object RewriteMergeIntoTable extends RewriteRowLevelIcebergCommand with Predicat val mergeRows = MergeRows( isSourceRowPresent = IsNotNull(rowFromSourceAttr), - isTargetRowPresent = if (notMatchedActions.isEmpty) TrueLiteral else IsNotNull(rowFromTargetAttr), + isTargetRowPresent = + if (notMatchedActions.isEmpty) TrueLiteral else IsNotNull(rowFromTargetAttr), matchedConditions = matchedConditions, matchedOutputs = matchedOutputs, notMatchedConditions = notMatchedConditions, @@ -275,7 +292,8 @@ object RewriteMergeIntoTable extends RewriteRowLevelIcebergCommand with Predicat // also disable broadcasts for the target table to perform the cardinality check val joinType = if (notMatchedActions.isEmpty) Inner else RightOuter val joinHint = JoinHint(leftHint = Some(HintInfo(Some(NO_BROADCAST_HASH))), rightHint = None) - val joinPlan = Join(NoStatsUnaryNode(targetTableProj), sourceTableProj, joinType, Some(joinCond), joinHint) + val joinPlan = + Join(NoStatsUnaryNode(targetTableProj), sourceTableProj, joinType, Some(joinCond), joinHint) val metadataReadAttrs = readAttrs.filterNot(relation.outputSet.contains) @@ -294,11 +312,13 @@ object RewriteMergeIntoTable extends RewriteRowLevelIcebergCommand with Predicat val rowFromTargetAttr = resolveAttrRef(ROW_FROM_TARGET_REF, joinPlan) // merged rows must contain values for the operation type and all read attrs - val mergeRowsOutput = buildMergeRowsOutput(matchedOutputs, notMatchedOutputs, operationTypeAttr +: readAttrs) + val mergeRowsOutput = + buildMergeRowsOutput(matchedOutputs, notMatchedOutputs, operationTypeAttr +: readAttrs) val mergeRows = MergeRows( isSourceRowPresent = IsNotNull(rowFromSourceAttr), - isTargetRowPresent = if (notMatchedActions.isEmpty) TrueLiteral else IsNotNull(rowFromTargetAttr), + isTargetRowPresent = + if (notMatchedActions.isEmpty) TrueLiteral else IsNotNull(rowFromTargetAttr), matchedConditions = matchedConditions, matchedOutputs = matchedOutputs, notMatchedConditions = notMatchedConditions, diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTableForRowLineage.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTableForRowLineage.scala index cebb360a294f..1dbc171f5dbd 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTableForRowLineage.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTableForRowLineage.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Literal @@ -32,9 +31,9 @@ object RewriteMergeIntoTableForRowLineage extends RewriteOperationForRowLineage override def apply(plan: LogicalPlan): LogicalPlan = { plan.resolveOperators { case m @ MergeIntoIcebergTable(_, _, _, matchedActions, _, _) - if m.resolved && m.aligned && - matchedActions.nonEmpty && - shouldUpdatePlan(m.targetTable) => + if m.resolved && m.aligned && + matchedActions.nonEmpty && + shouldUpdatePlan(m.targetTable) => updateMergeIntoForRowLineage(m) } } @@ -48,17 +47,22 @@ object RewriteMergeIntoTableForRowLineage extends RewriteOperationForRowLineage val matchedAssignmentsForLineage = matchedActions.map { case UpdateAction(cond, assignments) => - UpdateAction(cond, assignments ++ Seq(Assignment(rowId, rowId), - Assignment(lastUpdatedSequenceNumber, Literal(null)))) + UpdateAction( + cond, + assignments ++ Seq( + Assignment(rowId, rowId), + Assignment(lastUpdatedSequenceNumber, Literal(null)))) case deleteAction => deleteAction } val notMatchedActionsForLineage = notMatchedActions.map { case InsertAction(cond, assignments) => - InsertAction(cond, assignments ++ Seq( - Assignment(rowId, Literal(null)), - Assignment(lastUpdatedSequenceNumber, Literal(null)))) + InsertAction( + cond, + assignments ++ Seq( + Assignment(rowId, Literal(null)), + Assignment(lastUpdatedSequenceNumber, Literal(null)))) } val tableWithLineage = r.copy(output = r.output ++ Seq(rowId, lastUpdatedSequenceNumber)) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteOperationForRowLineage.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteOperationForRowLineage.scala index 611e62b75e07..538e04264778 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteOperationForRowLineage.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteOperationForRowLineage.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.iceberg.MetadataColumns @@ -32,7 +31,8 @@ import org.apache.spark.sql.types.MetadataBuilder trait RewriteOperationForRowLineage extends RewriteRowLevelIcebergCommand { protected val ROW_ID_ATTRIBUTE_NAME = MetadataColumns.ROW_ID.name() - protected val LAST_UPDATED_SEQUENCE_NUMBER_ATTRIBUTE_NAME = MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name() + protected val LAST_UPDATED_SEQUENCE_NUMBER_ATTRIBUTE_NAME = + MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name() // The plan should only be updated if the underlying Iceberg table supports row lineage AND // lineage attributes are not already on the output of operation which indicates the rule already ran @@ -52,16 +52,17 @@ trait RewriteOperationForRowLineage extends RewriteRowLevelIcebergCommand { } protected def findRowLineageAttributes( - expressions: Seq[Expression] - ): Option[(AttributeReference, AttributeReference)] = { + expressions: Seq[Expression]): Option[(AttributeReference, AttributeReference)] = { val rowIdAttr = expressions.collectFirst { case attr: AttributeReference - if isMetadataColumn(attr) && attr.name == ROW_ID_ATTRIBUTE_NAME => attr + if isMetadataColumn(attr) && attr.name == ROW_ID_ATTRIBUTE_NAME => + attr } val lastUpdatedAttr = expressions.collectFirst { case attr: AttributeReference - if isMetadataColumn(attr) && attr.name == LAST_UPDATED_SEQUENCE_NUMBER_ATTRIBUTE_NAME => attr + if isMetadataColumn(attr) && attr.name == LAST_UPDATED_SEQUENCE_NUMBER_ATTRIBUTE_NAME => + attr } // Treat row lineage columns as data columns by removing the metadata attribute @@ -78,7 +79,8 @@ trait RewriteOperationForRowLineage extends RewriteRowLevelIcebergCommand { attr.withMetadata( new MetadataBuilder() .withMetadata(attr.metadata) - .remove(METADATA_COL_ATTR_KEY).build()) + .remove(METADATA_COL_ATTR_KEY) + .build()) } private def isMetadataColumn(attributeReference: AttributeReference): Boolean = { diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelIcebergCommand.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelIcebergCommand.scala index 0b1871038fe7..f40fc16e8ce1 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelIcebergCommand.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteRowLevelIcebergCommand.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException @@ -169,9 +168,8 @@ trait RewriteRowLevelIcebergCommand extends RewriteRowLevelCommand { operation match { case supportsDelta: SupportsDelta => - val rowIdAttrs = V2ExpressionUtils.resolveRefs[AttributeReference]( - supportsDelta.rowId.toSeq, - relation) + val rowIdAttrs = + V2ExpressionUtils.resolveRefs[AttributeReference](supportsDelta.rowId.toSeq, relation) val nullableRowIdAttrs = rowIdAttrs.filter(_.nullable) if (nullableRowIdAttrs.nonEmpty) { diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteUpdateTable.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteUpdateTable.scala index bbe55321855f..664f4d8da6cb 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteUpdateTable.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteUpdateTable.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException @@ -152,9 +151,8 @@ object RewriteUpdateTable extends RewriteRowLevelIcebergCommand { // build a plan for updated records that match the cond val matchedRowsPlan = Filter(cond, readRelation) - val updatedRowsPlan = updateRows( - matchedRowsPlan, assignments, readAttrs, - rowAttrs, rowIdAttrs, metadataReadAttrs) + val updatedRowsPlan = + updateRows(matchedRowsPlan, assignments, readAttrs, rowAttrs, rowIdAttrs, metadataReadAttrs) // build a plan to write the row delta to the table val writeRelation = relation.copy(table = operationTable) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteUpdateTableForRowLineage.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteUpdateTableForRowLineage.scala index 00513d89fcdb..34f4de6bb3e3 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteUpdateTableForRowLineage.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteUpdateTableForRowLineage.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Literal @@ -30,17 +29,17 @@ object RewriteUpdateTableForRowLineage extends RewriteOperationForRowLineage { override def apply(plan: LogicalPlan): LogicalPlan = { plan resolveOperators { - case updateTable@UpdateIcebergTable(_, _, _, _) if shouldUpdatePlan(updateTable.table) => + case updateTable @ UpdateIcebergTable(_, _, _, _) if shouldUpdatePlan(updateTable.table) => updatePlanWithRowLineage(updateTable) } } - private def updatePlanWithRowLineage(updateTable: UpdateIcebergTable): LogicalPlan = { EliminateSubqueryAliases(updateTable.table) match { case r @ DataSourceV2Relation(_: SupportsRowLevelOperations, _, _, _, _) => val lineageAttributes = findRowLineageAttributes(r.metadataOutput).get - val (rowId, lastUpdatedSequence) = (removeMetadataColumnAttribute(lineageAttributes._1), + val (rowId, lastUpdatedSequence) = ( + removeMetadataColumnAttribute(lineageAttributes._1), removeMetadataColumnAttribute(lineageAttributes._2)) val lineageAssignments = updateTable.assignments ++ diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index c7f09b2f5a07..e5cad5ee10bc 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException @@ -52,11 +51,19 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case DropView(ResolvedIdent(resolved), ifExists) => DropIcebergView(resolved, ifExists) - case CreateView(ResolvedIdent(resolved), userSpecifiedColumns, comment, properties, - Some(queryText), query, allowExisting, replace) => + case CreateView( + ResolvedIdent(resolved), + userSpecifiedColumns, + comment, + properties, + Some(queryText), + query, + allowExisting, + replace) => val q = CTESubstitution.apply(query) verifyTemporaryObjectsDontExist(resolved, q) - CreateIcebergView(child = resolved, + CreateIcebergView( + child = resolved, queryText = queryText, query = q, columnAliases = userSpecifiedColumns.map(_._1), @@ -68,20 +75,25 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case view @ ShowViews(UnresolvedNamespace(Seq()), pattern, output) => if (ViewUtil.isViewCatalog(catalogManager.currentCatalog)) { - ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace.toIndexedSeq), - pattern, output) + ShowIcebergViews( + ResolvedNamespace( + catalogManager.currentCatalog, + catalogManager.currentNamespace.toIndexedSeq), + pattern, + output) } else { view } case ShowViews(UnresolvedNamespace(CatalogAndNamespace(catalog, ns)), pattern, output) - if ViewUtil.isViewCatalog(catalog) => + if ViewUtil.isViewCatalog(catalog) => ShowIcebergViews(ResolvedNamespace(catalog, ns), pattern, output) // needs to be done here instead of in ResolveViews, so that a V2 view can be resolved before the Analyzer // tries to resolve it, which would result in an error, saying that V2 views aren't supported - case u@UnresolvedView(ResolvedView(resolved), _, _, _) => - ViewUtil.loadView(resolved.catalog, resolved.identifier) + case u @ UnresolvedView(ResolvedView(resolved), _, _, _) => + ViewUtil + .loadView(resolved.catalog, resolved.identifier) .map(_ => ResolvedV2View(resolved.catalog.asViewCatalog, resolved.identifier)) .getOrElse(u) } @@ -102,7 +114,8 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case UnresolvedIdentifier(nameParts, true) if isTempView(nameParts) => None - case UnresolvedIdentifier(CatalogAndIdentifier(catalog, ident), _) if ViewUtil.isViewCatalog(catalog) => + case UnresolvedIdentifier(CatalogAndIdentifier(catalog, ident), _) + if ViewUtil.isViewCatalog(catalog) => Some(ResolvedIdentifier(catalog, ident)) case _ => @@ -114,11 +127,14 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi * Permanent views are not allowed to reference temp objects */ private def verifyTemporaryObjectsDontExist( - identifier: ResolvedIdentifier, - child: LogicalPlan): Unit = { + identifier: ResolvedIdentifier, + child: LogicalPlan): Unit = { val tempViews = collectTemporaryViews(child) if (tempViews.nonEmpty) { - throw invalidRefToTempObject(identifier, tempViews.map(v => v.quoted).mkString("[", ", ", "]"), "view") + throw invalidRefToTempObject( + identifier, + tempViews.map(v => v.quoted).mkString("[", ", ", "]"), + "view") } val tempFunctions = collectTemporaryFunctions(child) @@ -127,9 +143,17 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi } } - private def invalidRefToTempObject(ident: ResolvedIdentifier, tempObjectNames: String, tempObjectType: String) = { - new AnalysisException(String.format("Cannot create view %s.%s that references temporary %s: %s", - ident.catalog.name(), ident.identifier, tempObjectType, tempObjectNames)) + private def invalidRefToTempObject( + ident: ResolvedIdentifier, + tempObjectNames: String, + tempObjectType: String) = { + new AnalysisException( + String.format( + "Cannot create view %s.%s that references temporary %s: %s", + ident.catalog.name(), + ident.identifier, + tempObjectType, + tempObjectNames)) } /** @@ -141,10 +165,11 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case unresolved: UnresolvedRelation if isTempView(unresolved.multipartIdentifier) => Seq(unresolved.multipartIdentifier) case view: View if view.isTempView => Seq(view.desc.identifier.nameParts) - case plan => plan.expressions.flatMap(_.flatMap { - case e: SubqueryExpression => collectTempViews(e.plan) - case _ => Seq.empty - }) + case plan => + plan.expressions.flatMap(_.flatMap { + case e: SubqueryExpression => collectTempViews(e.plan) + case _ => Seq.empty + }) }.distinct } @@ -157,7 +182,9 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi None case CatalogAndIdentifier(catalog, ident) if ViewUtil.isViewCatalog(catalog) => - ViewUtil.loadView(catalog, ident).flatMap(_ => Some(ResolvedV2View(catalog.asViewCatalog, ident))) + ViewUtil + .loadView(catalog, ident) + .flatMap(_ => Some(ResolvedV2View(catalog.asViewCatalog, ident))) case _ => None diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala index d46f10b7f5a2..a7188837c51e 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.connector.catalog.CatalogPlugin diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/expressions/AssignmentUtils.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/expressions/AssignmentUtils.scala index ce3818922c78..89c0630556b3 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/expressions/AssignmentUtils.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/expressions/AssignmentUtils.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.AnalysisException @@ -40,12 +39,13 @@ object AssignmentUtils extends SQLConfHelper { sameSize && table.output.zip(assignments).forall { case (attr, assignment) => val key = assignment.key val value = assignment.value - val refsEqual = toAssignmentRef(attr).zip(toAssignmentRef(key)) - .forall{ case (attrRef, keyRef) => conf.resolver(attrRef, keyRef)} + val refsEqual = toAssignmentRef(attr) + .zip(toAssignmentRef(key)) + .forall { case (attrRef, keyRef) => conf.resolver(attrRef, keyRef) } refsEqual && - DataType.equalsIgnoreCompatibleNullability(value.dataType, attr.dataType) && - (attr.nullable || !value.nullable) + DataType.equalsIgnoreCompatibleNullability(value.dataType, attr.dataType) && + (attr.nullable || !value.nullable) } } @@ -66,16 +66,16 @@ object AssignmentUtils extends SQLConfHelper { val key = assignment.key val value = assignment.value - val rawKeyType = key.transform { - case attr: AttributeReference => - CharVarcharUtils.getRawType(attr.metadata) - .map(attr.withDataType) - .getOrElse(attr) + val rawKeyType = key.transform { case attr: AttributeReference => + CharVarcharUtils + .getRawType(attr.metadata) + .map(attr.withDataType) + .getOrElse(attr) }.dataType if (CharVarcharUtils.hasCharVarchar(rawKeyType)) { - val newKey = key.transform { - case attr: AttributeReference => CharVarcharUtils.cleanAttrMetadata(attr) + val newKey = key.transform { case attr: AttributeReference => + CharVarcharUtils.cleanAttrMetadata(attr) } val newValue = CharVarcharUtils.stringLengthCheck(value, rawKeyType) Assignment(newKey, newValue) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedReplaceNullWithFalseInPredicate.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedReplaceNullWithFalseInPredicate.scala index 803eb5d25749..ad11f927f142 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedReplaceNullWithFalseInPredicate.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedReplaceNullWithFalseInPredicate.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.expressions.And @@ -50,18 +49,18 @@ import org.apache.spark.util.Utils */ object ExtendedReplaceNullWithFalseInPredicate extends Rule[LogicalPlan] { - override def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning( - _.containsAnyPattern(NULL_LITERAL, TRUE_OR_FALSE_LITERAL, INSET)) { + override def apply(plan: LogicalPlan): LogicalPlan = + plan.transformWithPruning(_.containsAnyPattern(NULL_LITERAL, TRUE_OR_FALSE_LITERAL, INSET)) { - case u @ UpdateIcebergTable(_, _, Some(cond), _) => - u.copy(condition = Some(replaceNullWithFalse(cond))) + case u @ UpdateIcebergTable(_, _, Some(cond), _) => + u.copy(condition = Some(replaceNullWithFalse(cond))) - case m @ MergeIntoIcebergTable(_, _, mergeCond, matchedActions, notMatchedActions, _) => - m.copy( - mergeCondition = replaceNullWithFalse(mergeCond), - matchedActions = replaceNullWithFalse(matchedActions), - notMatchedActions = replaceNullWithFalse(notMatchedActions)) - } + case m @ MergeIntoIcebergTable(_, _, mergeCond, matchedActions, notMatchedActions, _) => + m.copy( + mergeCondition = replaceNullWithFalse(mergeCond), + matchedActions = replaceNullWithFalse(matchedActions), + notMatchedActions = replaceNullWithFalse(notMatchedActions)) + } /** * Recursively traverse the Boolean-type expression to replace diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedSimplifyConditionalsInPredicate.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedSimplifyConditionalsInPredicate.scala index ac6df514d3d1..ba5f6c8dfbbf 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedSimplifyConditionalsInPredicate.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ExtendedSimplifyConditionalsInPredicate.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.expressions.And @@ -40,18 +39,18 @@ import org.apache.spark.sql.types.BooleanType */ object ExtendedSimplifyConditionalsInPredicate extends Rule[LogicalPlan] { - override def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning( - _.containsAnyPattern(CASE_WHEN, IF)) { + override def apply(plan: LogicalPlan): LogicalPlan = + plan.transformWithPruning(_.containsAnyPattern(CASE_WHEN, IF)) { - case u @ UpdateIcebergTable(_, _, Some(cond), _) => - u.copy(condition = Some(simplifyConditional(cond))) + case u @ UpdateIcebergTable(_, _, Some(cond), _) => + u.copy(condition = Some(simplifyConditional(cond))) - case m @ MergeIntoIcebergTable(_, _, mergeCond, matchedActions, notMatchedActions, _) => - m.copy( - mergeCondition = simplifyConditional(mergeCond), - matchedActions = simplifyConditional(matchedActions), - notMatchedActions = simplifyConditional(notMatchedActions)) - } + case m @ MergeIntoIcebergTable(_, _, mergeCond, matchedActions, notMatchedActions, _) => + m.copy( + mergeCondition = simplifyConditional(mergeCond), + matchedActions = simplifyConditional(matchedActions), + notMatchedActions = simplifyConditional(notMatchedActions)) + } private def simplifyConditional(e: Expression): Expression = e match { case And(left, right) => And(simplifyConditional(left), simplifyConditional(right)) @@ -61,8 +60,9 @@ object ExtendedSimplifyConditionalsInPredicate extends Rule[LogicalPlan] { case If(cond, FalseLiteral, falseValue) => And(Not(Coalesce(Seq(cond, FalseLiteral))), falseValue) case If(cond, TrueLiteral, falseValue) => Or(cond, falseValue) - case CaseWhen(Seq((cond, trueValue)), - Some(FalseLiteral) | Some(Literal(null, BooleanType)) | None) => + case CaseWhen( + Seq((cond, trueValue)), + Some(FalseLiteral) | Some(Literal(null, BooleanType)) | None) => And(cond, trueValue) case CaseWhen(Seq((cond, trueValue)), Some(TrueLiteral)) => Or(Not(Coalesce(Seq(cond, FalseLiteral))), trueValue) @@ -72,9 +72,10 @@ object ExtendedSimplifyConditionalsInPredicate extends Rule[LogicalPlan] { Or(cond, elseValue) case e if e.dataType == BooleanType => e case e => - assert(e.dataType != BooleanType, + assert( + e.dataType != BooleanType, "Expected a Boolean type expression in ExtendedSimplifyConditionalsInPredicate, " + - s"but got the type `${e.dataType.catalogString}` in `${e.sql}`.") + s"but got the type `${e.dataType.catalogString}` in `${e.sql}`.") e } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRowLineageOutputFromOriginalTable.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRowLineageOutputFromOriginalTable.scala index a3e9ad372bac..b6af8ae799e3 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRowLineageOutputFromOriginalTable.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRowLineageOutputFromOriginalTable.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.optimizer import org.apache.iceberg.MetadataColumns @@ -35,20 +34,19 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation object RemoveRowLineageOutputFromOriginalTable extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = { plan.transform { - case writeDelta@WriteIcebergDelta(_, _, originalTable, _, _) => + case writeDelta @ WriteIcebergDelta(_, _, originalTable, _, _) => writeDelta.copy(originalTable = removeRowLineageOutput(originalTable)) - case replaceData@ReplaceIcebergData(_, _, originalTable, _) => + case replaceData @ ReplaceIcebergData(_, _, originalTable, _) => replaceData.copy(originalTable = removeRowLineageOutput(originalTable)) } } private def removeRowLineageOutput(table: NamedRelation): DataSourceV2Relation = { table match { - case dsv2Relation@DataSourceV2Relation(_, _, _, _, _) => - dsv2Relation.copy(output = dsv2Relation.output.filterNot( - attr => attr.name == MetadataColumns.ROW_ID.name() || + case dsv2Relation @ DataSourceV2Relation(_, _, _, _, _) => + dsv2Relation.copy(output = dsv2Relation.output.filterNot(attr => + attr.name == MetadataColumns.ROW_ID.name() || attr.name == MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name())) } } } - diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala index 5ad26bcaae21..a2a68d864b65 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala @@ -47,13 +47,13 @@ import org.apache.spark.sql.types.StructType object ReplaceStaticInvoke extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = - plan.transformWithPruning (_.containsAnyPattern(COMMAND, FILTER, JOIN)) { + plan.transformWithPruning(_.containsAnyPattern(COMMAND, FILTER, JOIN)) { case join @ Join(_, _, _, Some(cond), _) => replaceStaticInvoke(join, cond, newCond => join.copy(condition = Some(newCond))) case filter @ Filter(cond, _) => replaceStaticInvoke(filter, cond, newCond => filter.copy(condition = newCond)) - } + } private def replaceStaticInvoke[T <: LogicalPlan]( node: T, @@ -86,16 +86,17 @@ object ReplaceStaticInvoke extends Rule[LogicalPlan] { return invoke } - val inputType = StructType(invoke.arguments.zipWithIndex.map { - case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable) + val inputType = StructType(invoke.arguments.zipWithIndex.map { case (exp, pos) => + StructField(s"_$pos", exp.dataType, exp.nullable) }) - val bound = try { - unbound.bind(inputType) - } catch { - case _: Exception => - return invoke - } + val bound = + try { + unbound.bind(inputType) + } catch { + case _: Exception => + return invoke + } if (bound.inputTypes().length != invoke.arguments.length) { return invoke diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala index e3f42c12fa78..1fb0641c00c8 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.parser.extensions import java.util.Locale @@ -59,7 +58,9 @@ import org.apache.spark.sql.types.StructType import scala.jdk.CollectionConverters._ import scala.util.Try -class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserInterface with ExtendedParser { +class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) + extends ParserInterface + with ExtendedParser { import IcebergSparkSqlExtensionsParser._ @@ -128,7 +129,8 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI override def parsePlan(sqlText: String): LogicalPlan = { val sqlTextAfterSubstitution = substitutor.substitute(sqlText) if (isIcebergCommand(sqlTextAfterSubstitution)) { - parse(sqlTextAfterSubstitution) { parser => astBuilder.visit(parser.singleStatement()) }.asInstanceOf[LogicalPlan] + parse(sqlTextAfterSubstitution) { parser => astBuilder.visit(parser.singleStatement()) } + .asInstanceOf[LogicalPlan] } else { val parsedPlan = RewriteViewCommands(SparkSession.active).apply(delegate.parsePlan(sqlText)) parsedPlan match { @@ -144,7 +146,13 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI case UpdateTable(UnresolvedIcebergTable(aliasedTable), assignments, condition) => UpdateIcebergTable(aliasedTable, assignments, condition) - case MergeIntoTable(UnresolvedIcebergTable(aliasedTable), source, cond, matchedActions, notMatchedActions, Nil) => + case MergeIntoTable( + UnresolvedIcebergTable(aliasedTable), + source, + cond, + matchedActions, + notMatchedActions, + Nil) => // cannot construct MergeIntoIcebergTable right away as MERGE operations require special resolution // that's why the condition and actions must be hidden from the regular resolution rules in Spark // see ResolveMergeIntoTableReferences for details @@ -168,7 +176,8 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI } private def isIcebergTable(multipartIdent: Seq[String]): Boolean = { - val catalogAndIdentifier = Spark3Util.catalogAndIdentifier(SparkSession.active, multipartIdent.asJava) + val catalogAndIdentifier = + Spark3Util.catalogAndIdentifier(SparkSession.active, multipartIdent.asJava) catalogAndIdentifier.catalog match { case tableCatalog: TableCatalog => Try(tableCatalog.loadTable(catalogAndIdentifier.identifier)) @@ -187,7 +196,9 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI } private def isIcebergCommand(sqlText: String): Boolean = { - val normalized = sqlText.toLowerCase(Locale.ROOT).trim() + val normalized = sqlText + .toLowerCase(Locale.ROOT) + .trim() // Strip simple SQL comments that terminate a line, e.g. comments starting with `--` . .replaceAll("--.*?\\n", " ") // Strip newlines. @@ -198,37 +209,37 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI // Strip backtick then `system`.`ancestors_of` changes to system.ancestors_of .replaceAll("`", "") .trim() - isIcebergProcedure(normalized) || ( - normalized.startsWith("alter table") && ( - normalized.contains("add partition field") || - normalized.contains("drop partition field") || - normalized.contains("replace partition field") || - normalized.contains("write ordered by") || - normalized.contains("write locally ordered by") || - normalized.contains("write distributed by") || - normalized.contains("write unordered") || - normalized.contains("set identifier fields") || - normalized.contains("drop identifier fields") || - isSnapshotRefDdl(normalized))) + isIcebergProcedure(normalized) || (normalized + .startsWith("alter table") && (normalized.contains("add partition field") || + normalized.contains("drop partition field") || + normalized.contains("replace partition field") || + normalized.contains("write ordered by") || + normalized.contains("write locally ordered by") || + normalized.contains("write distributed by") || + normalized.contains("write unordered") || + normalized.contains("set identifier fields") || + normalized.contains("drop identifier fields") || + isSnapshotRefDdl(normalized))) } // All builtin Iceberg procedures are under the 'system' namespace private def isIcebergProcedure(normalized: String): Boolean = { normalized.startsWith("call") && - SparkProcedures.names().asScala.map("system." + _).exists(normalized.contains) + SparkProcedures.names().asScala.map("system." + _).exists(normalized.contains) } private def isSnapshotRefDdl(normalized: String): Boolean = { normalized.contains("create branch") || - normalized.contains("replace branch") || - normalized.contains("create tag") || - normalized.contains("replace tag") || - normalized.contains("drop branch") || - normalized.contains("drop tag") + normalized.contains("replace branch") || + normalized.contains("create tag") || + normalized.contains("replace tag") || + normalized.contains("drop branch") || + normalized.contains("drop tag") } protected def parse[T](command: String)(toResult: IcebergSqlExtensionsParser => T): T = { - val lexer = new IcebergSqlExtensionsLexer(new UpperCaseCharStream(CharStreams.fromString(command))) + val lexer = new IcebergSqlExtensionsLexer( + new UpperCaseCharStream(CharStreams.fromString(command))) lexer.removeErrorListeners() lexer.addErrorListener(IcebergParseErrorListener) @@ -246,8 +257,7 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI parser.setErrorHandler(new BailErrorStrategy) parser.getInterpreter.setPredictionMode(PredictionMode.SLL) toResult(parser) - } - catch { + } catch { case _: ParseCancellationException => // if we fail, parse with LL mode with DefaultErrorStrategy tokenStream.seek(0) // rewind input stream @@ -258,8 +268,7 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI parser.getInterpreter.setPredictionMode(PredictionMode.LL) toResult(parser) } - } - catch { + } catch { case e: IcebergParseException if e.command.isDefined => throw e case e: IcebergParseException => @@ -277,7 +286,8 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI object IcebergSparkSqlExtensionsParser { private val substitutorCtor: DynConstructors.Ctor[VariableSubstitution] = - DynConstructors.builder() + DynConstructors + .builder() .impl(classOf[VariableSubstitution]) .impl(classOf[VariableSubstitution], classOf[SQLConf]) .build() @@ -323,9 +333,7 @@ case object IcebergSqlExtensionsPostProcessor extends IcebergSqlExtensionsBaseLi replaceTokenByIdentifier(ctx, 0)(identity) } - private def replaceTokenByIdentifier( - ctx: ParserRuleContext, - stripMargins: Int)( + private def replaceTokenByIdentifier(ctx: ParserRuleContext, stripMargins: Int)( f: CommonToken => CommonToken = identity): Unit = { val parent = ctx.getParent parent.removeLastChild() @@ -372,10 +380,12 @@ class IcebergParseException( val command: Option[String], message: String, val start: Origin, - val stop: Origin) extends AnalysisException(message, start.line, start.startPosition) { + val stop: Origin) + extends AnalysisException(message, start.line, start.startPosition) { def this(message: String, ctx: ParserRuleContext) = { - this(Option(IcebergParserUtils.command(ctx)), + this( + Option(IcebergParserUtils.command(ctx)), message, IcebergParserUtils.position(ctx.getStart), IcebergParserUtils.position(ctx.getStop)) @@ -385,8 +395,7 @@ class IcebergParseException( val builder = new StringBuilder builder ++= "\n" ++= message start match { - case Origin( - Some(l), Some(p), Some(_), Some(_), Some(_), Some(_), Some(_)) => + case Origin(Some(l), Some(p), Some(_), Some(_), Some(_), Some(_), Some(_)) => builder ++= s"(line $l, pos $p)\n" command.foreach { cmd => val (above, below) = cmd.split("\n").splitAt(l) @@ -406,4 +415,4 @@ class IcebergParseException( def withCommand(cmd: String): IcebergParseException = { new IcebergParseException(Option(cmd), message, start, stop) } -} \ No newline at end of file +} diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala index 6b1cc41da04c..f87733e6b79b 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.parser.extensions import java.util.Locale @@ -63,9 +62,11 @@ import org.apache.spark.sql.connector.expressions.LiteralValue import org.apache.spark.sql.connector.expressions.Transform import scala.jdk.CollectionConverters._ -class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergSqlExtensionsBaseVisitor[AnyRef] { +class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) + extends IcebergSqlExtensionsBaseVisitor[AnyRef] { - private def toBuffer[T](list: java.util.List[T]): scala.collection.mutable.Buffer[T] = list.asScala + private def toBuffer[T](list: java.util.List[T]): scala.collection.mutable.Buffer[T] = + list.asScala private def toSeq[T](list: java.util.List[T]): Seq[T] = toBuffer(list).toSeq /** @@ -80,52 +81,59 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS /** * Create an ADD PARTITION FIELD logical command. */ - override def visitAddPartitionField(ctx: AddPartitionFieldContext): AddPartitionField = withOrigin(ctx) { - AddPartitionField( - typedVisit[Seq[String]](ctx.multipartIdentifier), - typedVisit[Transform](ctx.transform), - Option(ctx.name).map(_.getText)) - } + override def visitAddPartitionField(ctx: AddPartitionFieldContext): AddPartitionField = + withOrigin(ctx) { + AddPartitionField( + typedVisit[Seq[String]](ctx.multipartIdentifier), + typedVisit[Transform](ctx.transform), + Option(ctx.name).map(_.getText)) + } /** * Create a DROP PARTITION FIELD logical command. */ - override def visitDropPartitionField(ctx: DropPartitionFieldContext): DropPartitionField = withOrigin(ctx) { - DropPartitionField( - typedVisit[Seq[String]](ctx.multipartIdentifier), - typedVisit[Transform](ctx.transform)) - } + override def visitDropPartitionField(ctx: DropPartitionFieldContext): DropPartitionField = + withOrigin(ctx) { + DropPartitionField( + typedVisit[Seq[String]](ctx.multipartIdentifier), + typedVisit[Transform](ctx.transform)) + } /** * Create a CREATE OR REPLACE BRANCH logical command. */ - override def visitCreateOrReplaceBranch(ctx: CreateOrReplaceBranchContext): CreateOrReplaceBranch = withOrigin(ctx) { + override def visitCreateOrReplaceBranch( + ctx: CreateOrReplaceBranchContext): CreateOrReplaceBranch = withOrigin(ctx) { val createOrReplaceBranchClause = ctx.createReplaceBranchClause() val branchName = createOrReplaceBranchClause.identifier() val branchOptionsContext = Option(createOrReplaceBranchClause.branchOptions()) - val snapshotId = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.snapshotId())) + val snapshotId = branchOptionsContext + .flatMap(branchOptions => Option(branchOptions.snapshotId())) .map(_.getText.toLong) - val snapshotRetention = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.snapshotRetention())) - val minSnapshotsToKeep = snapshotRetention.flatMap(retention => Option(retention.minSnapshotsToKeep())) + val snapshotRetention = + branchOptionsContext.flatMap(branchOptions => Option(branchOptions.snapshotRetention())) + val minSnapshotsToKeep = snapshotRetention + .flatMap(retention => Option(retention.minSnapshotsToKeep())) .map(minSnapshots => minSnapshots.number().getText.toLong) val maxSnapshotAgeMs = snapshotRetention .flatMap(retention => Option(retention.maxSnapshotAge())) - .map(retention => TimeUnit.valueOf(retention.timeUnit().getText.toUpperCase(Locale.ENGLISH)) - .toMillis(retention.number().getText.toLong)) - val branchRetention = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.refRetain())) + .map(retention => + TimeUnit + .valueOf(retention.timeUnit().getText.toUpperCase(Locale.ENGLISH)) + .toMillis(retention.number().getText.toLong)) + val branchRetention = + branchOptionsContext.flatMap(branchOptions => Option(branchOptions.refRetain())) val branchRefAgeMs = branchRetention.map(retain => - TimeUnit.valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)).toMillis(retain.number().getText.toLong)) + TimeUnit + .valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)) + .toMillis(retain.number().getText.toLong)) val create = createOrReplaceBranchClause.CREATE() != null val replace = ctx.createReplaceBranchClause().REPLACE() != null val ifNotExists = createOrReplaceBranchClause.EXISTS() != null - val branchOptions = BranchOptions( - snapshotId, - minSnapshotsToKeep, - maxSnapshotAgeMs, - branchRefAgeMs - ) + val branchOptions = + BranchOptions(snapshotId, minSnapshotsToKeep, maxSnapshotAgeMs, branchRefAgeMs) CreateOrReplaceBranch( typedVisit[Seq[String]](ctx.multipartIdentifier), @@ -139,52 +147,61 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS /** * Create an CREATE OR REPLACE TAG logical command. */ - override def visitCreateOrReplaceTag(ctx: CreateOrReplaceTagContext): CreateOrReplaceTag = withOrigin(ctx) { - val createTagClause = ctx.createReplaceTagClause() - - val tagName = createTagClause.identifier().getText - - val tagOptionsContext = Option(createTagClause.tagOptions()) - val snapshotId = tagOptionsContext.flatMap(tagOptions => Option(tagOptions.snapshotId())) - .map(_.getText.toLong) - val tagRetain = tagOptionsContext.flatMap(tagOptions => Option(tagOptions.refRetain())) - val tagRefAgeMs = tagRetain.map(retain => - TimeUnit.valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)).toMillis(retain.number().getText.toLong)) - val tagOptions = TagOptions( - snapshotId, - tagRefAgeMs - ) - - val create = createTagClause.CREATE() != null - val replace = createTagClause.REPLACE() != null - val ifNotExists = createTagClause.EXISTS() != null - - CreateOrReplaceTag(typedVisit[Seq[String]](ctx.multipartIdentifier), - tagName, - tagOptions, - create, - replace, - ifNotExists) - } + override def visitCreateOrReplaceTag(ctx: CreateOrReplaceTagContext): CreateOrReplaceTag = + withOrigin(ctx) { + val createTagClause = ctx.createReplaceTagClause() + + val tagName = createTagClause.identifier().getText + + val tagOptionsContext = Option(createTagClause.tagOptions()) + val snapshotId = tagOptionsContext + .flatMap(tagOptions => Option(tagOptions.snapshotId())) + .map(_.getText.toLong) + val tagRetain = tagOptionsContext.flatMap(tagOptions => Option(tagOptions.refRetain())) + val tagRefAgeMs = tagRetain.map(retain => + TimeUnit + .valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)) + .toMillis(retain.number().getText.toLong)) + val tagOptions = TagOptions(snapshotId, tagRefAgeMs) + + val create = createTagClause.CREATE() != null + val replace = createTagClause.REPLACE() != null + val ifNotExists = createTagClause.EXISTS() != null + + CreateOrReplaceTag( + typedVisit[Seq[String]](ctx.multipartIdentifier), + tagName, + tagOptions, + create, + replace, + ifNotExists) + } /** * Create an DROP BRANCH logical command. */ override def visitDropBranch(ctx: DropBranchContext): DropBranch = withOrigin(ctx) { - DropBranch(typedVisit[Seq[String]](ctx.multipartIdentifier), ctx.identifier().getText, ctx.EXISTS() != null) + DropBranch( + typedVisit[Seq[String]](ctx.multipartIdentifier), + ctx.identifier().getText, + ctx.EXISTS() != null) } /** * Create an DROP TAG logical command. */ override def visitDropTag(ctx: DropTagContext): DropTag = withOrigin(ctx) { - DropTag(typedVisit[Seq[String]](ctx.multipartIdentifier), ctx.identifier().getText, ctx.EXISTS() != null) + DropTag( + typedVisit[Seq[String]](ctx.multipartIdentifier), + ctx.identifier().getText, + ctx.EXISTS() != null) } /** * Create an REPLACE PARTITION FIELD logical command. */ - override def visitReplacePartitionField(ctx: ReplacePartitionFieldContext): ReplacePartitionField = withOrigin(ctx) { + override def visitReplacePartitionField( + ctx: ReplacePartitionFieldContext): ReplacePartitionField = withOrigin(ctx) { ReplacePartitionField( typedVisit[Seq[String]](ctx.multipartIdentifier), typedVisit[Transform](ctx.transform(0)), @@ -195,20 +212,22 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS /** * Create an SET IDENTIFIER FIELDS logical command. */ - override def visitSetIdentifierFields(ctx: SetIdentifierFieldsContext): SetIdentifierFields = withOrigin(ctx) { - SetIdentifierFields( - typedVisit[Seq[String]](ctx.multipartIdentifier), - toSeq(ctx.fieldList.fields).map(_.getText)) - } + override def visitSetIdentifierFields(ctx: SetIdentifierFieldsContext): SetIdentifierFields = + withOrigin(ctx) { + SetIdentifierFields( + typedVisit[Seq[String]](ctx.multipartIdentifier), + toSeq(ctx.fieldList.fields).map(_.getText)) + } /** * Create an DROP IDENTIFIER FIELDS logical command. */ - override def visitDropIdentifierFields(ctx: DropIdentifierFieldsContext): DropIdentifierFields = withOrigin(ctx) { - DropIdentifierFields( - typedVisit[Seq[String]](ctx.multipartIdentifier), - toSeq(ctx.fieldList.fields).map(_.getText)) - } + override def visitDropIdentifierFields(ctx: DropIdentifierFieldsContext): DropIdentifierFields = + withOrigin(ctx) { + DropIdentifierFields( + typedVisit[Seq[String]](ctx.multipartIdentifier), + toSeq(ctx.fieldList.fields).map(_.getText)) + } /** * Create a [[SetWriteDistributionAndOrdering]] for changing the write distribution and ordering. @@ -266,12 +285,15 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS */ override def visitOrderField(ctx: OrderFieldContext): (Term, SortDirection, NullOrder) = { val term = Spark3Util.toIcebergTerm(typedVisit[Transform](ctx.transform)) - val direction = Option(ctx.ASC).map(_ => SortDirection.ASC) - .orElse(Option(ctx.DESC).map(_ => SortDirection.DESC)) - .getOrElse(SortDirection.ASC) - val nullOrder = Option(ctx.FIRST).map(_ => NullOrder.NULLS_FIRST) - .orElse(Option(ctx.LAST).map(_ => NullOrder.NULLS_LAST)) - .getOrElse(if (direction == SortDirection.ASC) NullOrder.NULLS_FIRST else NullOrder.NULLS_LAST) + val direction = Option(ctx.ASC) + .map(_ => SortDirection.ASC) + .orElse(Option(ctx.DESC).map(_ => SortDirection.DESC)) + .getOrElse(SortDirection.ASC) + val nullOrder = Option(ctx.FIRST) + .map(_ => NullOrder.NULLS_FIRST) + .orElse(Option(ctx.LAST).map(_ => NullOrder.NULLS_LAST)) + .getOrElse( + if (direction == SortDirection.ASC) NullOrder.NULLS_FIRST else NullOrder.NULLS_LAST) (term, direction, nullOrder) } @@ -293,35 +315,40 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS /** * Create a transform argument from a column reference or a constant. */ - override def visitTransformArgument(ctx: TransformArgumentContext): expressions.Expression = withOrigin(ctx) { - val reference = Option(ctx.multipartIdentifier()) + override def visitTransformArgument(ctx: TransformArgumentContext): expressions.Expression = + withOrigin(ctx) { + val reference = Option(ctx.multipartIdentifier()) .map(typedVisit[Seq[String]]) .map(FieldReference(_)) - val literal = Option(ctx.constant) + val literal = Option(ctx.constant) .map(visitConstant) .map(lit => LiteralValue(lit.value, lit.dataType)) - reference.orElse(literal) + reference + .orElse(literal) .getOrElse(throw new IcebergParseException(s"Invalid transform argument", ctx)) - } + } /** * Return a multi-part identifier as Seq[String]. */ - override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = withOrigin(ctx) { - toSeq(ctx.parts).map(_.getText) - } + override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = + withOrigin(ctx) { + toSeq(ctx.parts).map(_.getText) + } - override def visitSingleOrder(ctx: SingleOrderContext): Seq[(Term, SortDirection, NullOrder)] = withOrigin(ctx) { - toSeq(ctx.order.fields).map(typedVisit[(Term, SortDirection, NullOrder)]) - } + override def visitSingleOrder(ctx: SingleOrderContext): Seq[(Term, SortDirection, NullOrder)] = + withOrigin(ctx) { + toSeq(ctx.order.fields).map(typedVisit[(Term, SortDirection, NullOrder)]) + } /** * Create a positional argument in a stored procedure call. */ - override def visitPositionalArgument(ctx: PositionalArgumentContext): CallArgument = withOrigin(ctx) { - val expr = typedVisit[Expression](ctx.expression) - PositionalArgument(expr) - } + override def visitPositionalArgument(ctx: PositionalArgumentContext): CallArgument = + withOrigin(ctx) { + val expr = typedVisit[Expression](ctx.expression) + PositionalArgument(expr) + } /** * Create a named argument in a stored procedure call. @@ -350,10 +377,12 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS } private def reconstructSqlString(ctx: ParserRuleContext): String = { - toBuffer(ctx.children).map { - case c: ParserRuleContext => reconstructSqlString(c) - case t: TerminalNode => t.getText - }.mkString(" ") + toBuffer(ctx.children) + .map { + case c: ParserRuleContext => reconstructSqlString(c) + case t: TerminalNode => t.getText + } + .mkString(" ") } private def typedVisit[T](ctx: ParseTree): T = { diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala index b6b39fd77eab..623251d1222c 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/planning/RewrittenRowLevelCommand.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.planning import org.apache.spark.sql.AnalysisException diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala index e8b1b2941161..0a830dbd4f6a 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala @@ -16,13 +16,13 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.expressions.Transform -case class AddPartitionField(table: Seq[String], transform: Transform, name: Option[String]) extends LeafCommand { +case class AddPartitionField(table: Seq[String], transform: Transform, name: Option[String]) + extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala index 4d7e0a086bda..15b908300213 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala @@ -16,8 +16,10 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical -case class BranchOptions (snapshotId: Option[Long], numSnapshots: Option[Long], - snapshotRetain: Option[Long], snapshotRefRetain: Option[Long]) +case class BranchOptions( + snapshotId: Option[Long], + numSnapshots: Option[Long], + snapshotRetain: Option[Long], + snapshotRefRetain: Option[Long]) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala index 9616dae5a8d3..e80715e57ba1 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala index b7981a3c7a0d..6900f6e8cc50 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute @@ -27,7 +26,8 @@ case class CreateOrReplaceBranch( branchOptions: BranchOptions, create: Boolean, replace: Boolean, - ifNotExists: Boolean) extends LeafCommand { + ifNotExists: Boolean) + extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala index 6e7db84a90fb..957c68e7a540 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute @@ -27,7 +26,8 @@ case class CreateOrReplaceTag( tagOptions: TagOptions, create: Boolean, replace: Boolean, - ifNotExists: Boolean) extends LeafCommand { + ifNotExists: Boolean) + extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala index bee0b0fae688..ed4f1f512b85 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala index 29dd686a0fba..1a91806280b3 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala @@ -16,14 +16,11 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute -case class DropIdentifierFields( - table: Seq[String], - fields: Seq[String]) extends LeafCommand { +case class DropIdentifierFields(table: Seq[String], fields: Seq[String]) extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala index fb1451324182..ec952c8c7118 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala index 7e4b38e74d2f..da69ca0383a1 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeIntoIcebergTable.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeIntoIcebergTable.scala index 8f84851dcda2..9e2734ab1f68 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeIntoIcebergTable.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeIntoIcebergTable.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.AssignmentUtils @@ -28,7 +27,8 @@ case class MergeIntoIcebergTable( mergeCondition: Expression, matchedActions: Seq[MergeAction], notMatchedActions: Seq[MergeAction], - rewritePlan: Option[LogicalPlan] = None) extends RowLevelCommand { + rewritePlan: Option[LogicalPlan] = None) + extends RowLevelCommand { lazy val aligned: Boolean = { val matchedActionsAligned = matchedActions.forall { @@ -71,7 +71,8 @@ case class MergeIntoIcebergTable( case Seq(newTarget, newSource, newRewritePlan) => copy(targetTable = newTarget, sourceTable = newSource, rewritePlan = Some(newRewritePlan)) case _ => - throw new IllegalArgumentException("MergeIntoIcebergTable expects either two or three children") + throw new IllegalArgumentException( + "MergeIntoIcebergTable expects either two or three children") } } } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeRows.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeRows.scala index 57f8bf358330..d808737d1048 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeRows.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/MergeRows.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute @@ -35,7 +34,8 @@ case class MergeRows( performCardinalityCheck: Boolean, emitNotMatchedTargetRows: Boolean, output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode { + child: LogicalPlan) + extends UnaryNode { require(targetOutput.nonEmpty || !emitNotMatchedTargetRows) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/NoStatsUnaryNode.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/NoStatsUnaryNode.scala index c21df71f069d..936873466721 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/NoStatsUnaryNode.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/NoStatsUnaryNode.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceIcebergData.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceIcebergData.scala index 2b741bef121a..6648193229df 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceIcebergData.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplaceIcebergData.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.analysis.NamedRelation @@ -33,7 +32,8 @@ case class ReplaceIcebergData( table: NamedRelation, query: LogicalPlan, originalTable: NamedRelation, - write: Option[Write] = None) extends V2WriteCommandLike { + write: Option[Write] = None) + extends V2WriteCommandLike { override lazy val references: AttributeSet = query.outputSet override lazy val stringArgs: Iterator[Any] = Iterator(table, query, write) @@ -45,7 +45,8 @@ case class ReplaceIcebergData( } override def outputResolved: Boolean = { - assert(table.resolved && query.resolved, + assert( + table.resolved && query.resolved, "`outputResolved` can only be called when `table` and `query` are both resolved.") // take into account only incoming data columns and ignore metadata columns in the query @@ -58,8 +59,8 @@ case class ReplaceIcebergData( val outType = CharVarcharUtils.getRawType(outAttr.metadata).getOrElse(outAttr.dataType) // names and types must match, nullability must be compatible inAttr.name == outAttr.name && - DataType.equalsIgnoreCompatibleNullability(inAttr.dataType, outType) && - (outAttr.nullable || !inAttr.nullable) + DataType.equalsIgnoreCompatibleNullability(inAttr.dataType, outType) && + (outAttr.nullable || !inAttr.nullable) }) } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala index 8c660c6f37b1..c2525369e7c7 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute @@ -26,13 +25,14 @@ case class ReplacePartitionField( table: Seq[String], transformFrom: Transform, transformTo: Transform, - name: Option[String]) extends LeafCommand { + name: Option[String]) + extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil override def simpleString(maxFields: Int): String = { s"ReplacePartitionField ${table.quoted} ${transformFrom.describe} " + - s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" + s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" } } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/RowLevelCommand.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/RowLevelCommand.scala index 837ee963bcea..ffddce9f3e0e 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/RowLevelCommand.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/RowLevelCommand.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Expression diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala index 1e5622e9c374..8cd2c0ddad05 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala @@ -16,14 +16,11 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute -case class SetIdentifierFields( - table: Seq[String], - fields: Seq[String]) extends LeafCommand { +case class SetIdentifierFields(table: Seq[String], fields: Seq[String]) extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala index 85e3b95f4aba..6afe1478d747 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical case class TagOptions(snapshotId: Option[Long], snapshotRefRetain: Option[Long]) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/UnresolvedMergeIntoIcebergTable.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/UnresolvedMergeIntoIcebergTable.scala index 895aa733ff20..1ecb727007db 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/UnresolvedMergeIntoIcebergTable.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/UnresolvedMergeIntoIcebergTable.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Expression @@ -27,14 +26,17 @@ import org.apache.spark.sql.catalyst.expressions.Expression case class UnresolvedMergeIntoIcebergTable( targetTable: LogicalPlan, sourceTable: LogicalPlan, - context: MergeIntoContext) extends BinaryCommand { + context: MergeIntoContext) + extends BinaryCommand { def duplicateResolved: Boolean = targetTable.outputSet.intersect(sourceTable.outputSet).isEmpty override def left: LogicalPlan = targetTable override def right: LogicalPlan = sourceTable - override protected def withNewChildrenInternal(newLeft: LogicalPlan, newRight: LogicalPlan): LogicalPlan = { + override protected def withNewChildrenInternal( + newLeft: LogicalPlan, + newRight: LogicalPlan): LogicalPlan = { copy(targetTable = newLeft, sourceTable = newRight) } } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/UpdateIcebergTable.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/UpdateIcebergTable.scala index 790eb9380e3d..58aef899d639 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/UpdateIcebergTable.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/UpdateIcebergTable.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.AssignmentUtils @@ -26,7 +25,8 @@ case class UpdateIcebergTable( table: LogicalPlan, assignments: Seq[Assignment], condition: Option[Expression], - rewritePlan: Option[LogicalPlan] = None) extends RowLevelCommand { + rewritePlan: Option[LogicalPlan] = None) + extends RowLevelCommand { lazy val aligned: Boolean = AssignmentUtils.aligned(table, assignments) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/UpdateRows.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/UpdateRows.scala index 0446f3040481..268150e6b086 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/UpdateRows.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/UpdateRows.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute @@ -28,7 +27,8 @@ case class UpdateRows( deleteOutput: Seq[Expression], insertOutput: Seq[Expression], output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode { + child: LogicalPlan) + extends UnaryNode { override lazy val producedAttributes: AttributeSet = { AttributeSet(output.filterNot(attr => inputSet.contains(attr))) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/V2WriteCommandLike.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/V2WriteCommandLike.scala index 9192d74b7caf..e34d1e2daa05 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/V2WriteCommandLike.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/V2WriteCommandLike.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.analysis.NamedRelation diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteIcebergDelta.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteIcebergDelta.scala index 8495856fb6b0..66ece84f883e 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteIcebergDelta.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/WriteIcebergDelta.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.AnalysisException @@ -44,7 +43,8 @@ case class WriteIcebergDelta( query: LogicalPlan, originalTable: NamedRelation, projections: WriteDeltaProjections, - write: Option[DeltaWrite] = None) extends V2WriteCommandLike { + write: Option[DeltaWrite] = None) + extends V2WriteCommandLike { override protected lazy val stringArgs: Iterator[Any] = Iterator(table, query, write) @@ -71,18 +71,17 @@ case class WriteIcebergDelta( table.skipSchemaResolution || (projections.rowProjection match { case Some(projection) => table.output.size == projection.schema.size && - projection.schema.zip(table.output).forall { case (field, outAttr) => - isCompatible(field, outAttr) - } + projection.schema.zip(table.output).forall { case (field, outAttr) => + isCompatible(field, outAttr) + } case None => true }) } private def rowIdAttrsResolved: Boolean = { - val rowIdAttrs = V2ExpressionUtils.resolveRefs[AttributeReference]( - operation.rowId.toSeq, - originalTable) + val rowIdAttrs = + V2ExpressionUtils.resolveRefs[AttributeReference](operation.rowId.toSeq, originalTable) projections.rowIdProjection.schema.forall { field => rowIdAttrs.exists(rowIdAttr => isCompatible(field, rowIdAttr)) @@ -109,12 +108,13 @@ case class WriteIcebergDelta( val outType = CharVarcharUtils.getRawType(outAttr.metadata).getOrElse(outAttr.dataType) // names and types must match, nullability must be compatible projectionField.name == outAttr.name && - DataType.equalsIgnoreCompatibleNullability(inType, outType) && - (outAttr.nullable || !projectionField.nullable) + DataType.equalsIgnoreCompatibleNullability(inType, outType) && + (outAttr.nullable || !projectionField.nullable) } override def outputResolved: Boolean = { - assert(table.resolved && query.resolved, + assert( + table.resolved && query.resolved, "`outputResolved` can only be called when `table` and `query` are both resolved.") operationResolved && rowAttrsResolved && rowIdAttrsResolved && metadataAttrsResolved diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index be15f32bc1b8..9986b30c45ed 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Expression diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala index 9366d5efe163..c5dadae70110 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala @@ -16,29 +16,30 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical.views import org.apache.spark.sql.catalyst.plans.logical.BinaryCommand import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan case class CreateIcebergView( - child: LogicalPlan, - queryText: String, - query: LogicalPlan, - columnAliases: Seq[String], - columnComments: Seq[Option[String]], - queryColumnNames: Seq[String] = Seq.empty, - comment: Option[String], - properties: Map[String, String], - allowExisting: Boolean, - replace: Boolean, - rewritten: Boolean = false) extends BinaryCommand { + child: LogicalPlan, + queryText: String, + query: LogicalPlan, + columnAliases: Seq[String], + columnComments: Seq[Option[String]], + queryColumnNames: Seq[String] = Seq.empty, + comment: Option[String], + properties: Map[String, String], + allowExisting: Boolean, + replace: Boolean, + rewritten: Boolean = false) + extends BinaryCommand { override def left: LogicalPlan = child override def right: LogicalPlan = query override protected def withNewChildrenInternal( - newLeft: LogicalPlan, newRight: LogicalPlan): LogicalPlan = + newLeft: LogicalPlan, + newRight: LogicalPlan): LogicalPlan = copy(child = newLeft, query = newRight) } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala index 275dba6fbf5e..092b6b33fb0c 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala @@ -16,15 +16,12 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical.views import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.UnaryCommand -case class DropIcebergView( - child: LogicalPlan, - ifExists: Boolean) extends UnaryCommand { +case class DropIcebergView(child: LogicalPlan, ifExists: Boolean) extends UnaryCommand { override protected def withNewChildInternal(newChild: LogicalPlan): DropIcebergView = copy(child = newChild) } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala index b9c05ff0061d..4d384e857703 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical.views import org.apache.spark.sql.catalyst.analysis.LeafNodeWithoutStats @@ -24,8 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog -case class ResolvedV2View( - catalog: ViewCatalog, - identifier: Identifier) extends LeafNodeWithoutStats { +case class ResolvedV2View(catalog: ViewCatalog, identifier: Identifier) + extends LeafNodeWithoutStats { override def output: Seq[Attribute] = Nil } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala index b09c27acdc16..cbfe23d94cbe 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala @@ -16,8 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - - package org.apache.spark.sql.catalyst.plans.logical.views import org.apache.spark.sql.catalyst.expressions.Attribute @@ -26,9 +24,10 @@ import org.apache.spark.sql.catalyst.plans.logical.ShowViews import org.apache.spark.sql.catalyst.plans.logical.UnaryCommand case class ShowIcebergViews( - namespace: LogicalPlan, - pattern: Option[String], - override val output: Seq[Attribute] = ShowViews.getOutputAttrs) extends UnaryCommand { + namespace: LogicalPlan, + pattern: Option[String], + override val output: Seq[Attribute] = ShowViews.getOutputAttrs) + extends UnaryCommand { override def child: LogicalPlan = namespace override protected def withNewChildInternal(newChild: LogicalPlan): ShowIcebergViews = diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala index 55f327f7e45e..e28dcfb194b6 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.Spark3Util @@ -31,7 +30,8 @@ case class AddPartitionFieldExec( catalog: TableCatalog, ident: Identifier, transform: Transform, - name: Option[String]) extends LeafV2CommandExec { + name: Option[String]) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -39,12 +39,14 @@ case class AddPartitionFieldExec( override protected def run(): Seq[InternalRow] = { catalog.loadTable(ident) match { case iceberg: SparkTable => - iceberg.table.updateSpec() - .addField(name.orNull, Spark3Util.toIcebergTerm(transform)) - .commit() + iceberg.table + .updateSpec() + .addField(name.orNull, Spark3Util.toIcebergTerm(transform)) + .commit() case table => - throw new UnsupportedOperationException(s"Cannot add partition field to non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot add partition field to non-Iceberg table: $table") } Nil diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala index b103d1ee2c58..d6630e51ff5a 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -25,17 +24,17 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.connector.catalog.ViewChange - case class AlterV2ViewSetPropertiesExec( - catalog: ViewCatalog, - ident: Identifier, - properties: Map[String, String]) extends LeafV2CommandExec { + catalog: ViewCatalog, + ident: Identifier, + properties: Map[String, String]) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil override protected def run(): Seq[InternalRow] = { - val changes = properties.map { - case (property, value) => ViewChange.setProperty(property, value) + val changes = properties.map { case (property, value) => + ViewChange.setProperty(property, value) }.toSeq catalog.alterView(ident, changes: _*) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala index 91abd95f574c..7ebf0b9f7085 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.AnalysisException @@ -26,12 +25,12 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.connector.catalog.ViewChange - case class AlterV2ViewUnsetPropertiesExec( - catalog: ViewCatalog, - ident: Identifier, - propertyKeys: Seq[String], - ifExists: Boolean) extends LeafV2CommandExec { + catalog: ViewCatalog, + ident: Identifier, + propertyKeys: Seq[String], + ifExists: Boolean) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala index f66962a8c453..a12277502521 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -25,10 +24,8 @@ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.connector.iceberg.catalog.Procedure import scala.collection.compat.immutable.ArraySeq -case class CallExec( - output: Seq[Attribute], - procedure: Procedure, - input: InternalRow) extends LeafV2CommandExec { +case class CallExec(output: Seq[Attribute], procedure: Procedure, input: InternalRow) + extends LeafV2CommandExec { override protected def run(): Seq[InternalRow] = { ArraySeq.unsafeWrapArray(procedure.call(input)) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala index 2be406e7f344..baf985f53a22 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.relocated.com.google.common.base.Preconditions @@ -28,13 +27,14 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog case class CreateOrReplaceBranchExec( - catalog: TableCatalog, - ident: Identifier, - branch: String, - branchOptions: BranchOptions, - create: Boolean, - replace: Boolean, - ifNotExists: Boolean) extends LeafV2CommandExec { + catalog: TableCatalog, + ident: Identifier, + branch: String, + branchOptions: BranchOptions, + create: Boolean, + replace: Boolean, + ifNotExists: Boolean) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -62,8 +62,10 @@ case class CreateOrReplaceBranchExec( if (create && replace && !refExists) { safeCreateBranch() } else if (replace) { - Preconditions.checkArgument(snapshotId != null, - "Cannot complete replace branch operation on %s, main has no snapshot", ident) + Preconditions.checkArgument( + snapshotId != null, + "Cannot complete replace branch operation on %s, main has no snapshot", + ident) manageSnapshots.replaceBranch(branch, snapshotId) } else { if (refExists && ifNotExists) { @@ -88,7 +90,8 @@ case class CreateOrReplaceBranchExec( manageSnapshots.commit() case table => - throw new UnsupportedOperationException(s"Cannot create or replace branch on non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot create or replace branch on non-Iceberg table: $table") } Nil diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala index 372cd7548632..e486892614cb 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.relocated.com.google.common.base.Preconditions @@ -33,7 +32,8 @@ case class CreateOrReplaceTagExec( tagOptions: TagOptions, create: Boolean, replace: Boolean, - ifNotExists: Boolean) extends LeafV2CommandExec { + ifNotExists: Boolean) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -47,8 +47,10 @@ case class CreateOrReplaceTagExec( .map(java.lang.Long.valueOf) .orNull - Preconditions.checkArgument(snapshotId != null, - "Cannot complete create or replace tag operation on %s, main has no snapshot", ident) + Preconditions.checkArgument( + snapshotId != null, + "Cannot complete create or replace tag operation on %s, main has no snapshot", + ident) val manageSnapshot = iceberg.table.manageSnapshots() val refExists = null != iceberg.table().refs().get(tag) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala index eb4cc7de48c4..c9d404765845 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.SupportsReplaceView @@ -29,31 +28,33 @@ import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.types.StructType import scala.jdk.CollectionConverters._ - case class CreateV2ViewExec( - catalog: ViewCatalog, - ident: Identifier, - queryText: String, - viewSchema: StructType, - columnAliases: Seq[String], - columnComments: Seq[Option[String]], - queryColumnNames: Seq[String], - comment: Option[String], - properties: Map[String, String], - allowExisting: Boolean, - replace: Boolean) extends LeafV2CommandExec { + catalog: ViewCatalog, + ident: Identifier, + queryText: String, + viewSchema: StructType, + columnAliases: Seq[String], + columnComments: Seq[Option[String]], + queryColumnNames: Seq[String], + comment: Option[String], + properties: Map[String, String], + allowExisting: Boolean, + replace: Boolean) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil override protected def run(): Seq[InternalRow] = { val currentCatalogName = session.sessionState.catalogManager.currentCatalog.name - val currentCatalog = if (!catalog.name().equals(currentCatalogName)) currentCatalogName else null + val currentCatalog = + if (!catalog.name().equals(currentCatalogName)) currentCatalogName else null val currentNamespace = session.sessionState.catalogManager.currentNamespace val engineVersion = "Spark " + org.apache.spark.SPARK_VERSION val newProperties = properties ++ comment.map(ViewCatalog.PROP_COMMENT -> _) ++ - Map(ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion, + Map( + ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion, ViewCatalog.PROP_ENGINE_VERSION -> engineVersion) if (replace) { @@ -87,10 +88,10 @@ case class CreateV2ViewExec( } private def replaceView( - supportsReplaceView: SupportsReplaceView, - currentCatalog: String, - currentNamespace: Array[String], - newProperties: Map[String, String]) = { + supportsReplaceView: SupportsReplaceView, + currentCatalog: String, + currentNamespace: Array[String], + newProperties: Map[String, String]) = { supportsReplaceView.replaceView( ident, queryText, @@ -104,9 +105,9 @@ case class CreateV2ViewExec( } private def createView( - currentCatalog: String, - currentNamespace: Array[String], - newProperties: Map[String, String]) = { + currentCatalog: String, + currentNamespace: Array[String], + newProperties: Map[String, String]) = { catalog.createView( ident, queryText, diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala index a9c7312d4a90..f25ac8e3c3dc 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -27,10 +26,9 @@ import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode import scala.jdk.CollectionConverters._ -case class DescribeV2ViewExec( - output: Seq[Attribute], - view: View, - isExtended: Boolean) extends V2CommandExec with LeafExecNode { +case class DescribeV2ViewExec(output: Seq[Attribute], view: View, isExtended: Boolean) + extends V2CommandExec + with LeafExecNode { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -44,23 +42,22 @@ case class DescribeV2ViewExec( private def describeSchema: Seq[InternalRow] = view.schema().map { column => - toCatalystRow( - column.name, - column.dataType.simpleString, - column.getComment().getOrElse("")) + toCatalystRow(column.name, column.dataType.simpleString, column.getComment().getOrElse("")) } private def emptyRow: InternalRow = toCatalystRow("", "", "") private def describeExtended: Seq[InternalRow] = { val outputColumns = view.queryColumnNames.mkString("[", ", ", "]") - val properties: Map[String, String] = view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala + val properties: Map[String, String] = + view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala val viewCatalogAndNamespace: Seq[String] = view.name.split("\\.").take(2) - val viewProperties = properties.toSeq.sortBy(_._1).map { - case (key, value) => + val viewProperties = properties.toSeq + .sortBy(_._1) + .map { case (key, value) => s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" - }.mkString("[", ", ", "]") - + } + .mkString("[", ", ", "]") // omitting view text here because it is shown as // part of SHOW CREATE TABLE and can result in weird formatting in the DESCRIBE output @@ -69,7 +66,10 @@ case class DescribeV2ViewExec( toCatalystRow("View Catalog and Namespace", viewCatalogAndNamespace.quoted, "") :: toCatalystRow("View Query Output Columns", outputColumns, "") :: toCatalystRow("View Properties", viewProperties, "") :: - toCatalystRow("Created By", view.properties.getOrDefault(ViewCatalog.PROP_CREATE_ENGINE_VERSION, ""), "") :: + toCatalystRow( + "Created By", + view.properties.getOrDefault(ViewCatalog.PROP_CREATE_ENGINE_VERSION, ""), + "") :: Nil } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala index ff8f1820099a..e7d9c7b70d82 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.source.SparkTable @@ -29,7 +28,8 @@ case class DropBranchExec( catalog: TableCatalog, ident: Identifier, branch: String, - ifExists: Boolean) extends LeafV2CommandExec { + ifExists: Boolean) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala index dee778b474f9..87b18594d573 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.relocated.com.google.common.base.Preconditions @@ -27,10 +26,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog -case class DropIdentifierFieldsExec( - catalog: TableCatalog, - ident: Identifier, - fields: Seq[String]) extends LeafV2CommandExec { +case class DropIdentifierFieldsExec(catalog: TableCatalog, ident: Identifier, fields: Seq[String]) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -42,18 +39,24 @@ case class DropIdentifierFieldsExec( val identifierFieldNames = Sets.newHashSet(schema.identifierFieldNames) for (name <- fields) { - Preconditions.checkArgument(schema.findField(name) != null, - "Cannot complete drop identifier fields operation: field %s not found", name) - Preconditions.checkArgument(identifierFieldNames.contains(name), - "Cannot complete drop identifier fields operation: %s is not an identifier field", name) + Preconditions.checkArgument( + schema.findField(name) != null, + "Cannot complete drop identifier fields operation: field %s not found", + name) + Preconditions.checkArgument( + identifierFieldNames.contains(name), + "Cannot complete drop identifier fields operation: %s is not an identifier field", + name) identifierFieldNames.remove(name) } - iceberg.table.updateSchema() + iceberg.table + .updateSchema() .setIdentifierFields(identifierFieldNames) .commit(); case table => - throw new UnsupportedOperationException(s"Cannot drop identifier fields in non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot drop identifier fields in non-Iceberg table: $table") } Nil diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala index 9a153f0c004e..db43263e0e66 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.Spark3Util @@ -29,10 +28,8 @@ import org.apache.spark.sql.connector.expressions.FieldReference import org.apache.spark.sql.connector.expressions.IdentityTransform import org.apache.spark.sql.connector.expressions.Transform -case class DropPartitionFieldExec( - catalog: TableCatalog, - ident: Identifier, - transform: Transform) extends LeafV2CommandExec { +case class DropPartitionFieldExec(catalog: TableCatalog, ident: Identifier, transform: Transform) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -42,20 +39,24 @@ case class DropPartitionFieldExec( case iceberg: SparkTable => val schema = iceberg.table.schema transform match { - case IdentityTransform(FieldReference(parts)) if parts.size == 1 && schema.findField(parts.head) == null => + case IdentityTransform(FieldReference(parts)) + if parts.size == 1 && schema.findField(parts.head) == null => // the name is not present in the Iceberg schema, so it must be a partition field name, not a column name - iceberg.table.updateSpec() - .removeField(parts.head) - .commit() + iceberg.table + .updateSpec() + .removeField(parts.head) + .commit() case _ => - iceberg.table.updateSpec() - .removeField(Spark3Util.toIcebergTerm(transform)) - .commit() + iceberg.table + .updateSpec() + .removeField(Spark3Util.toIcebergTerm(transform)) + .commit() } case table => - throw new UnsupportedOperationException(s"Cannot drop partition field in non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot drop partition field in non-Iceberg table: $table") } Nil diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala index 8df88765a986..79b4a1525591 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.source.SparkTable @@ -25,11 +24,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog -case class DropTagExec( - catalog: TableCatalog, - ident: Identifier, - tag: String, - ifExists: Boolean) extends LeafV2CommandExec { +case class DropTagExec(catalog: TableCatalog, ident: Identifier, tag: String, ifExists: Boolean) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala index c35af1486fc7..6dd1188b78e8 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -25,11 +24,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog - -case class DropV2ViewExec( - catalog: ViewCatalog, - ident: Identifier, - ifExists: Boolean) extends LeafV2CommandExec { +case class DropV2ViewExec(catalog: ViewCatalog, ident: Identifier, ifExists: Boolean) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Implicits.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Implicits.scala index 85bda0b08d46..c960505a519e 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Implicits.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Implicits.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.AnalysisException diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala index a87b215ce695..afdddb7fab4e 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.Spark3Util @@ -77,11 +76,28 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi AddPartitionFieldExec(catalog, ident, transform, name) :: Nil case CreateOrReplaceBranch( - IcebergCatalogAndIdentifier(catalog, ident), branch, branchOptions, create, replace, ifNotExists) => - CreateOrReplaceBranchExec(catalog, ident, branch, branchOptions, create, replace, ifNotExists) :: Nil + IcebergCatalogAndIdentifier(catalog, ident), + branch, + branchOptions, + create, + replace, + ifNotExists) => + CreateOrReplaceBranchExec( + catalog, + ident, + branch, + branchOptions, + create, + replace, + ifNotExists) :: Nil case CreateOrReplaceTag( - IcebergCatalogAndIdentifier(catalog, ident), tag, tagOptions, create, replace, ifNotExists) => + IcebergCatalogAndIdentifier(catalog, ident), + tag, + tagOptions, + create, + replace, + ifNotExists) => CreateOrReplaceTagExec(catalog, ident, tag, tagOptions, create, replace, ifNotExists) :: Nil case DropBranch(IcebergCatalogAndIdentifier(catalog, ident), branch, ifExists) => @@ -93,7 +109,11 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi case DropPartitionField(IcebergCatalogAndIdentifier(catalog, ident), transform) => DropPartitionFieldExec(catalog, ident, transform) :: Nil - case ReplacePartitionField(IcebergCatalogAndIdentifier(catalog, ident), transformFrom, transformTo, name) => + case ReplacePartitionField( + IcebergCatalogAndIdentifier(catalog, ident), + transformFrom, + transformTo, + name) => ReplacePartitionFieldExec(catalog, ident, transformFrom, transformTo, name) :: Nil case SetIdentifierFields(IcebergCatalogAndIdentifier(catalog, ident), fields) => @@ -103,24 +123,49 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi DropIdentifierFieldsExec(catalog, ident, fields) :: Nil case SetWriteDistributionAndOrdering( - IcebergCatalogAndIdentifier(catalog, ident), distributionMode, ordering) => + IcebergCatalogAndIdentifier(catalog, ident), + distributionMode, + ordering) => SetWriteDistributionAndOrderingExec(catalog, ident, distributionMode, ordering) :: Nil case ReplaceIcebergData(_: DataSourceV2Relation, query, r: DataSourceV2Relation, Some(write)) => // refresh the cache using the original relation ReplaceDataExec(planLater(query), refreshCache(r), write) :: Nil - case WriteIcebergDelta(_: DataSourceV2Relation, query, r: DataSourceV2Relation, projs, Some(write)) => + case WriteIcebergDelta( + _: DataSourceV2Relation, + query, + r: DataSourceV2Relation, + projs, + Some(write)) => // refresh the cache using the original relation WriteDeltaExec(planLater(query), refreshCache(r), projs, write) :: Nil - case MergeRows(isSourceRowPresent, isTargetRowPresent, matchedConditions, matchedOutputs, notMatchedConditions, - notMatchedOutputs, targetOutput, performCardinalityCheck, emitNotMatchedTargetRows, - output, child) => - - MergeRowsExec(isSourceRowPresent, isTargetRowPresent, matchedConditions, matchedOutputs, notMatchedConditions, - notMatchedOutputs, targetOutput, performCardinalityCheck, emitNotMatchedTargetRows, - output, planLater(child)) :: Nil + case MergeRows( + isSourceRowPresent, + isTargetRowPresent, + matchedConditions, + matchedOutputs, + notMatchedConditions, + notMatchedOutputs, + targetOutput, + performCardinalityCheck, + emitNotMatchedTargetRows, + output, + child) => + + MergeRowsExec( + isSourceRowPresent, + isTargetRowPresent, + matchedConditions, + matchedOutputs, + notMatchedConditions, + notMatchedOutputs, + targetOutput, + performCardinalityCheck, + emitNotMatchedTargetRows, + output, + planLater(child)) :: Nil case UpdateRows(deleteOutput, insertOutput, output, child) => UpdateRowsExec(deleteOutput, insertOutput, output, planLater(child)) :: Nil @@ -131,7 +176,7 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi case OrderAwareCoalesce(numPartitions, coalescer, child) => OrderAwareCoalesceExec(numPartitions, coalescer, planLater(child)) :: Nil - case RenameTable(ResolvedV2View(oldCatalog: ViewCatalog, oldIdent), newName, isView@true) => + case RenameTable(ResolvedV2View(oldCatalog: ViewCatalog, oldIdent), newName, isView @ true) => val newIdent = Spark3Util.catalogAndIdentifier(spark, newName.toList.asJava) if (oldCatalog.name != newIdent.catalog().name()) { throw new AnalysisException( @@ -142,8 +187,18 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi case DropIcebergView(ResolvedIdentifier(viewCatalog: ViewCatalog, ident), ifExists) => DropV2ViewExec(viewCatalog, ident, ifExists) :: Nil - case CreateIcebergView(ResolvedIdentifier(viewCatalog: ViewCatalog, ident), queryText, query, - columnAliases, columnComments, queryColumnNames, comment, properties, allowExisting, replace, _) => + case CreateIcebergView( + ResolvedIdentifier(viewCatalog: ViewCatalog, ident), + queryText, + query, + columnAliases, + columnComments, + queryColumnNames, + comment, + properties, + allowExisting, + replace, + _) => CreateV2ViewExec( catalog = viewCatalog, ident = ident, diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala index 0d13f6a5230b..98b93b476bc9 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedV2Writes.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import java.util.Optional diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeRowsExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeRowsExec.scala index 474a417d3328..a183b1903c38 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeRowsExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MergeRowsExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.SparkException @@ -45,7 +44,8 @@ case class MergeRowsExec( performCardinalityCheck: Boolean, emitNotMatchedTargetRows: Boolean, output: Seq[Attribute], - child: SparkPlan) extends UnaryExecNode { + child: SparkPlan) + extends UnaryExecNode { private final val ROW_ID = "__row_id" @@ -115,12 +115,20 @@ case class MergeRowsExec( val mergeIterator = if (matchedActions.exists(_.isInstanceOf[Split])) { new SplittingMergeRowIterator( - rowIterator, cardinalityCheck, isTargetRowPresentPred, - matchedActions, notMatchedActions) + rowIterator, + cardinalityCheck, + isTargetRowPresentPred, + matchedActions, + notMatchedActions) } else { new MergeRowIterator( - rowIterator, cardinalityCheck, isTargetRowPresentPred, isSourceRowPresentPred, - projectTargetCols, matchedActions.asInstanceOf[Seq[Project]], notMatchedActions) + rowIterator, + cardinalityCheck, + isTargetRowPresentPred, + isSourceRowPresentPred, + projectTargetCols, + matchedActions.asInstanceOf[Seq[Project]], + notMatchedActions) } // null indicates a record must be discarded @@ -152,7 +160,7 @@ case class MergeRowsExec( private val targetTableProj: Projection, private val matchedActions: Seq[Project], private val notMatchedActions: Seq[Project]) - extends Iterator[InternalRow] { + extends Iterator[InternalRow] { override def hasNext: Boolean = rowIterator.hasNext @@ -206,7 +214,7 @@ case class MergeRowsExec( private val isTargetRowPresentPred: BasePredicate, private val matchedActions: Seq[Action], private val notMatchedActions: Seq[Project]) - extends Iterator[InternalRow] { + extends Iterator[InternalRow] { var cachedExtraRow: InternalRow = _ @@ -266,9 +274,9 @@ case class MergeRowsExec( protected def fail(): Unit = { throw new SparkException( "The ON search condition of the MERGE statement matched a single row from " + - "the target table with multiple rows of the source table. This could result " + - "in the target row being operated on more than once with an update or delete " + - "operation and is not allowed.") + "the target table with multiple rows of the source table. This could result " + + "in the target row being operated on more than once with an update or delete " + + "operation and is not allowed.") } } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala index 61d362044c3c..5dada1cab0bb 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -24,11 +23,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog - -case class RenameV2ViewExec( - catalog: ViewCatalog, - oldIdent: Identifier, - newIdent: Identifier) extends LeafV2CommandExec { +case class RenameV2ViewExec(catalog: ViewCatalog, oldIdent: Identifier, newIdent: Identifier) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil @@ -38,7 +34,6 @@ case class RenameV2ViewExec( Seq.empty } - override def simpleString(maxFields: Int): String = { s"RenameV2View ${oldIdent} to {newIdent}" } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceDataExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceDataExec.scala index 26c652469ac4..135c194a60eb 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceDataExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceDataExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.expressions.AttributeSet @@ -26,10 +25,8 @@ import org.apache.spark.sql.execution.SparkPlan /** * Physical plan node to replace data in existing tables. */ -case class ReplaceDataExec( - query: SparkPlan, - refreshCache: () => Unit, - write: Write) extends V2ExistingTableWriteExec { +case class ReplaceDataExec(query: SparkPlan, refreshCache: () => Unit, write: Write) + extends V2ExistingTableWriteExec { override lazy val references: AttributeSet = query.outputSet override lazy val stringArgs: Iterator[Any] = Iterator(query, write) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala index fcae0a5defc4..00b998c49e83 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.Spark3Util @@ -34,7 +33,8 @@ case class ReplacePartitionFieldExec( ident: Identifier, transformFrom: Transform, transformTo: Transform, - name: Option[String]) extends LeafV2CommandExec { + name: Option[String]) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -44,22 +44,26 @@ case class ReplacePartitionFieldExec( case iceberg: SparkTable => val schema = iceberg.table.schema transformFrom match { - case IdentityTransform(FieldReference(parts)) if parts.size == 1 && schema.findField(parts.head) == null => + case IdentityTransform(FieldReference(parts)) + if parts.size == 1 && schema.findField(parts.head) == null => // the name is not present in the Iceberg schema, so it must be a partition field name, not a column name - iceberg.table.updateSpec() - .removeField(parts.head) - .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) - .commit() + iceberg.table + .updateSpec() + .removeField(parts.head) + .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) + .commit() case _ => - iceberg.table.updateSpec() - .removeField(Spark3Util.toIcebergTerm(transformFrom)) - .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) - .commit() + iceberg.table + .updateSpec() + .removeField(Spark3Util.toIcebergTerm(transformFrom)) + .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) + .commit() } case table => - throw new UnsupportedOperationException(s"Cannot replace partition field in non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot replace partition field in non-Iceberg table: $table") } Nil @@ -67,6 +71,6 @@ case class ReplacePartitionFieldExec( override def simpleString(maxFields: Int): String = { s"ReplacePartitionField ${catalog.name}.${ident.quoted} ${transformFrom.describe} " + - s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" + s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" } } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceRewrittenRowLevelCommand.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceRewrittenRowLevelCommand.scala index 414d4c0ec305..8025f1a072be 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceRewrittenRowLevelCommand.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplaceRewrittenRowLevelCommand.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala index 8839e730e708..130d1e1bcd3c 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RowLevelCommandScanRelationPushDown.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.expressions.And @@ -48,15 +47,21 @@ object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with Predic override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { // use native Spark planning for delta-based plans // unlike other commands, these plans have filters that can be pushed down directly - case RewrittenRowLevelCommand(command, _: DataSourceV2Relation, rewritePlan: WriteIcebergDelta) => + case RewrittenRowLevelCommand( + command, + _: DataSourceV2Relation, + rewritePlan: WriteIcebergDelta) => val newRewritePlan = V2ScanRelationPushDown.apply(rewritePlan) command.withNewRewritePlan(newRewritePlan) // group-based MERGE operations are rewritten as joins and may be planned in a special way // the join condition is the MERGE condition and can be pushed into the source // this allows us to remove completely pushed down predicates from the join condition - case UnplannedGroupBasedMergeOperation(command, rd: ReplaceIcebergData, - join @ Join(_, _, _, Some(joinCond), _), relation: DataSourceV2Relation) => + case UnplannedGroupBasedMergeOperation( + command, + rd: ReplaceIcebergData, + join @ Join(_, _, _, Some(joinCond), _), + relation: DataSourceV2Relation) => val table = relation.table.asRowLevelOperationTable val scanBuilder = table.newScanBuilder(relation.options) @@ -70,8 +75,7 @@ object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with Predic val (scan, output) = PushDownUtils.pruneColumns(scanBuilder, relation, relation.output, Nil) - logInfo( - s""" + logInfo(s""" |Pushing MERGE operators to ${relation.name} |Pushed filters: $pushedFiltersStr |Original JOIN condition: $joinCond @@ -107,8 +111,7 @@ object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with Predic val (scan, output) = PushDownUtils.pruneColumns(scanBuilder, relation, relation.output, Nil) - logInfo( - s""" + logInfo(s""" |Pushing operators to ${relation.name} |Pushed filters: $pushedFiltersStr |Filters that were not pushed: ${remainingFilters.mkString(",")} @@ -128,7 +131,8 @@ object RowLevelCommandScanRelationPushDown extends Rule[LogicalPlan] with Predic private def pushFilters( cond: Expression, scanBuilder: ScanBuilder, - tableAttrs: Seq[AttributeReference]): (Either[Seq[Filter], Seq[Predicate]], Seq[Expression]) = { + tableAttrs: Seq[AttributeReference]) + : (Either[Seq[Filter], Seq[Predicate]], Seq[Expression]) = { val tableAttrSet = AttributeSet(tableAttrs) val filters = splitConjunctivePredicates(cond).filter(_.references.subsetOf(tableAttrSet)) @@ -178,8 +182,12 @@ object UnplannedGroupBasedMergeOperation { case rd @ ReplaceIcebergData(DataSourceV2Relation(table, _, _, _, _), query, _, _) => val joinsAndRelations = query.collect { case j @ Join( - NoStatsUnaryNode(ScanOperation(_, pushDownFilters, pushUpFilters, r: DataSourceV2Relation)), _, _, _, _) - if pushUpFilters.isEmpty && pushDownFilters.isEmpty && r.table.eq(table) => + NoStatsUnaryNode( + ScanOperation(_, pushDownFilters, pushUpFilters, r: DataSourceV2Relation)), + _, + _, + _, + _) if pushUpFilters.isEmpty && pushDownFilters.isEmpty && r.table.eq(table) => j -> r } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala index b50550ad38ef..50c53473ab60 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.source.SparkTable @@ -26,10 +25,8 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog import scala.jdk.CollectionConverters._ -case class SetIdentifierFieldsExec( - catalog: TableCatalog, - ident: Identifier, - fields: Seq[String]) extends LeafV2CommandExec { +case class SetIdentifierFieldsExec(catalog: TableCatalog, ident: Identifier, fields: Seq[String]) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -37,11 +34,13 @@ case class SetIdentifierFieldsExec( override protected def run(): Seq[InternalRow] = { catalog.loadTable(ident) match { case iceberg: SparkTable => - iceberg.table.updateSchema() + iceberg.table + .updateSchema() .setIdentifierFields(fields.asJava) .commit(); case table => - throw new UnsupportedOperationException(s"Cannot set identifier fields in non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot set identifier fields in non-Iceberg table: $table") } Nil diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala index c9004ddc5bda..9a10949d5e9e 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.DistributionMode @@ -36,7 +35,8 @@ case class SetWriteDistributionAndOrderingExec( catalog: TableCatalog, ident: Identifier, distributionMode: Option[DistributionMode], - sortOrder: Seq[(Term, SortDirection, NullOrder)]) extends LeafV2CommandExec { + sortOrder: Seq[(Term, SortDirection, NullOrder)]) + extends LeafV2CommandExec { import CatalogV2Implicits._ @@ -57,7 +57,8 @@ case class SetWriteDistributionAndOrderingExec( orderBuilder.commit() distributionMode.foreach { mode => - txn.updateProperties() + txn + .updateProperties() .set(WRITE_DISTRIBUTION_MODE, mode.modeName()) .commit() } @@ -65,7 +66,8 @@ case class SetWriteDistributionAndOrderingExec( txn.commitTransaction() case table => - throw new UnsupportedOperationException(s"Cannot set write order of non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot set write order of non-Iceberg table: $table") } Nil @@ -73,9 +75,11 @@ case class SetWriteDistributionAndOrderingExec( override def simpleString(maxFields: Int): String = { val tableIdent = s"${catalog.name}.${ident.quoted}" - val order = sortOrder.map { - case (term, direction, nullOrder) => s"$term $direction $nullOrder" - }.mkString(", ") + val order = sortOrder + .map { case (term, direction, nullOrder) => + s"$term $direction $nullOrder" + } + .mkString(", ") s"SetWriteDistributionAndOrdering $tableIdent $distributionMode $order" } } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala index ecd3e0d3c9ba..07ac4aeda8fb 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -28,7 +27,8 @@ import org.apache.spark.sql.execution.LeafExecNode import scala.jdk.CollectionConverters._ case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View) - extends V2CommandExec with LeafExecNode { + extends V2CommandExec + with LeafExecNode { override protected def run(): Seq[InternalRow] = { val builder = new StringBuilder @@ -43,7 +43,9 @@ case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View) private def showColumns(view: View, builder: StringBuilder): Unit = { val columns = concatByMultiLines( - view.schema().fields + view + .schema() + .fields .map(x => s"${x.name}${x.getComment().map(c => s" COMMENT '$c'").getOrElse("")}")) builder ++= columns } @@ -54,14 +56,11 @@ case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View) .foreach(builder.append) } - private def showProperties( - view: View, - builder: StringBuilder): Unit = { + private def showProperties(view: View, builder: StringBuilder): Unit = { val showProps = view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala if (showProps.nonEmpty) { - val props = conf.redactOptions(showProps).toSeq.sortBy(_._1).map { - case (key, value) => - s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" + val props = conf.redactOptions(showProps).toSeq.sortBy(_._1).map { case (key, value) => + s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" } builder ++= "TBLPROPERTIES " diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala index 553b80833087..ace43eb6c07b 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -26,25 +25,22 @@ import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode import scala.jdk.CollectionConverters._ -case class ShowV2ViewPropertiesExec( - output: Seq[Attribute], - view: View, - propertyKey: Option[String]) extends V2CommandExec with LeafExecNode { +case class ShowV2ViewPropertiesExec(output: Seq[Attribute], view: View, propertyKey: Option[String]) + extends V2CommandExec + with LeafExecNode { override protected def run(): Seq[InternalRow] = { propertyKey match { case Some(p) => - val propValue = properties.getOrElse(p, - s"View ${view.name()} does not have property: $p") + val propValue = properties.getOrElse(p, s"View ${view.name()} does not have property: $p") Seq(toCatalystRow(p, propValue)) case None => - properties.map { - case (k, v) => toCatalystRow(k, v) + properties.map { case (k, v) => + toCatalystRow(k, v) }.toSeq } } - private def properties = { view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala index 3aa85c3db54a..83a1f84aecba 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -27,10 +26,12 @@ import org.apache.spark.sql.execution.LeafExecNode import scala.collection.mutable.ArrayBuffer case class ShowV2ViewsExec( - output: Seq[Attribute], - catalog: ViewCatalog, - namespace: Seq[String], - pattern: Option[String]) extends V2CommandExec with LeafExecNode { + output: Seq[Attribute], + catalog: ViewCatalog, + namespace: Seq[String], + pattern: Option[String]) + extends V2CommandExec + with LeafExecNode { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -40,7 +41,8 @@ case class ShowV2ViewsExec( // handle GLOBAL VIEWS val globalTemp = session.sessionState.catalog.globalTempViewManager.database if (namespace.nonEmpty && globalTemp == namespace.head) { - pattern.map(p => session.sessionState.catalog.globalTempViewManager.listViewNames(p)) + pattern + .map(p => session.sessionState.catalog.globalTempViewManager.listViewNames(p)) .getOrElse(session.sessionState.catalog.globalTempViewManager.listViewNames("*")) .map(name => rows += toCatalystRow(globalTemp, name, true)) } else { @@ -53,7 +55,8 @@ case class ShowV2ViewsExec( } // include TEMP VIEWS - pattern.map(p => session.sessionState.catalog.listLocalTempViews(p)) + pattern + .map(p => session.sessionState.catalog.listLocalTempViews(p)) .getOrElse(session.sessionState.catalog.listLocalTempViews("*")) .map(v => rows += toCatalystRow(v.database.toArray.quoted, v.table, true)) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/UpdateRowsExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/UpdateRowsExec.scala index ef4ad3733931..9fd0dcd8c067 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/UpdateRowsExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/UpdateRowsExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.rdd.RDD @@ -33,7 +32,8 @@ case class UpdateRowsExec( deleteOutput: Seq[Expression], insertOutput: Seq[Expression], output: Seq[Attribute], - child: SparkPlan) extends UnaryExecNode { + child: SparkPlan) + extends UnaryExecNode { @transient override lazy val producedAttributes: AttributeSet = { AttributeSet(output.filterNot(attr => inputSet.contains(attr))) @@ -65,7 +65,7 @@ case class UpdateRowsExec( private val inputRows: Iterator[InternalRow], private val deleteProj: UnsafeProjection, private val insertProj: UnsafeProjection) - extends Iterator[InternalRow] { + extends Iterator[InternalRow] { var cachedInsertRow: InternalRow = _ diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandDynamicPruning.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandDynamicPruning.scala index f84bf6bd7561..f8acef9fe355 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandDynamicPruning.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/dynamicpruning/RowLevelCommandDynamicPruning.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.dynamicpruning import org.apache.spark.sql.SparkSession @@ -58,16 +57,19 @@ import scala.collection.compat.immutable.ArraySeq * Note that only group-based rewrite plans (i.e. ReplaceData) are taken into account. * Row-based rewrite plans are subject to usual runtime filtering. */ -case class RowLevelCommandDynamicPruning(spark: SparkSession) extends Rule[LogicalPlan] with PredicateHelper { +case class RowLevelCommandDynamicPruning(spark: SparkSession) + extends Rule[LogicalPlan] + with PredicateHelper { import ExtendedDataSourceV2Implicits._ override def apply(plan: LogicalPlan): LogicalPlan = plan transformDown { // apply special dynamic filtering only for plans that don't support deltas case RewrittenRowLevelCommand( - command: RowLevelCommand, - DataSourceV2ScanRelation(_, scan: SupportsRuntimeFiltering, _, _, _), - rewritePlan: ReplaceIcebergData) if conf.dynamicPartitionPruningEnabled && isCandidate(command) => + command: RowLevelCommand, + DataSourceV2ScanRelation(_, scan: SupportsRuntimeFiltering, _, _, _), + rewritePlan: ReplaceIcebergData) + if conf.dynamicPartitionPruningEnabled && isCandidate(command) => // use reference equality to find exactly the required scan relations val newRewritePlan = rewritePlan transformUp { @@ -154,15 +156,15 @@ case class RowLevelCommandDynamicPruning(spark: SparkSession) extends Rule[Logic } // borrowed from OptimizeSubqueries in Spark - private def optimizeSubquery(plan: LogicalPlan): LogicalPlan = plan.transformAllExpressionsWithPruning( - _.containsPattern(PLAN_EXPRESSION)) { - case s: SubqueryExpression => - val Subquery(newPlan, _) = spark.sessionState.optimizer.execute(Subquery.fromExpression(s)) - // At this point we have an optimized subquery plan that we are going to attach - // to this subquery expression. Here we can safely remove any top level sort - // in the plan as tuples produced by a subquery are un-ordered. - s.withNewPlan(removeTopLevelSort(newPlan)) - } + private def optimizeSubquery(plan: LogicalPlan): LogicalPlan = + plan.transformAllExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { + case s: SubqueryExpression => + val Subquery(newPlan, _) = spark.sessionState.optimizer.execute(Subquery.fromExpression(s)) + // At this point we have an optimized subquery plan that we are going to attach + // to this subquery expression. Here we can safely remove any top level sort + // in the plan as tuples produced by a subquery are un-ordered. + s.withNewPlan(removeTopLevelSort(newPlan)) + } // borrowed from OptimizeSubqueries in Spark private def removeTopLevelSort(plan: LogicalPlan): LogicalPlan = { diff --git a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala index 5acaa6800e68..5d5990c23b57 100644 --- a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala +++ b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.rdd.PartitionCoalescer @@ -26,10 +25,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute // this node doesn't extend RepartitionOperation on purpose to keep this logic isolated // and ignore it in optimizer rules such as CollapseRepartition -case class OrderAwareCoalesce( - numPartitions: Int, - coalescer: PartitionCoalescer, - child: LogicalPlan) extends OrderPreservingUnaryNode { +case class OrderAwareCoalesce(numPartitions: Int, coalescer: PartitionCoalescer, child: LogicalPlan) + extends OrderPreservingUnaryNode { override def output: Seq[Attribute] = child.output diff --git a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala index 7b599eb3da1d..3ce3d3e82ad8 100644 --- a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala +++ b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.iceberg.DistributionMode @@ -29,16 +28,19 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits case class SetWriteDistributionAndOrdering( table: Seq[String], distributionMode: Option[DistributionMode], - sortOrder: Seq[(Term, SortDirection, NullOrder)]) extends LeafCommand { + sortOrder: Seq[(Term, SortDirection, NullOrder)]) + extends LeafCommand { import CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil override def simpleString(maxFields: Int): String = { - val order = sortOrder.map { - case (term, direction, nullOrder) => s"$term $direction $nullOrder" - }.mkString(", ") + val order = sortOrder + .map { case (term, direction, nullOrder) => + s"$term $direction $nullOrder" + } + .mkString(", ") s"SetWriteDistributionAndOrdering ${table.quoted} $distributionMode $order" } } diff --git a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala index bf19ef8a2167..71b6107d048e 100644 --- a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala +++ b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.iceberg.NullOrder @@ -27,7 +26,9 @@ import org.apache.iceberg.expressions.Term class SortOrderParserUtil { - def collectSortOrder(tableSchema:Schema, sortOrder: Seq[(Term, SortDirection, NullOrder)]): SortOrder = { + def collectSortOrder( + tableSchema: Schema, + sortOrder: Seq[(Term, SortDirection, NullOrder)]): SortOrder = { val orderBuilder = SortOrder.builderFor(tableSchema) sortOrder.foreach { case (term, SortDirection.ASC, nullOrder) => diff --git a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala index aa9e9c553346..f46127f818e5 100644 --- a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala +++ b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.utils import org.apache.iceberg.spark.source.SparkTable diff --git a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala index 2ef99550524a..83dec98c0d9a 100644 --- a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala +++ b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution import org.apache.spark.rdd.PartitionCoalescer @@ -31,7 +30,8 @@ import org.apache.spark.sql.catalyst.plans.physical.UnknownPartitioning case class OrderAwareCoalesceExec( numPartitions: Int, coalescer: PartitionCoalescer, - child: SparkPlan) extends UnaryExecNode { + child: SparkPlan) + extends UnaryExecNode { override def output: Seq[Attribute] = child.output diff --git a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala index 95a0315e7189..c50584be8278 100644 --- a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala +++ b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources import org.apache.iceberg.spark.SparkV2Filters @@ -32,7 +31,8 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy object SparkExpressionConverter { - def convertToIcebergExpression(sparkExpression: Expression): org.apache.iceberg.expressions.Expression = { + def convertToIcebergExpression( + sparkExpression: Expression): org.apache.iceberg.expressions.Expression = { // Currently, it is a double conversion as we are converting Spark expression to Spark predicate // and then converting Spark predicate to Iceberg expression. // But these two conversions already exist and well tested. So, we are going with this approach. @@ -40,26 +40,33 @@ object SparkExpressionConverter { case Some(filter) => val converted = SparkV2Filters.convert(filter) if (converted == null) { - throw new IllegalArgumentException(s"Cannot convert Spark filter: $filter to Iceberg expression") + throw new IllegalArgumentException( + s"Cannot convert Spark filter: $filter to Iceberg expression") } converted case _ => - throw new IllegalArgumentException(s"Cannot translate Spark expression: $sparkExpression to data source filter") + throw new IllegalArgumentException( + s"Cannot translate Spark expression: $sparkExpression to data source filter") } } @throws[AnalysisException] - def collectResolvedSparkExpression(session: SparkSession, tableName: String, where: String): Expression = { + def collectResolvedSparkExpression( + session: SparkSession, + tableName: String, + where: String): Expression = { val tableAttrs = session.table(tableName).queryExecution.analyzed.output val unresolvedExpression = session.sessionState.sqlParser.parseExpression(where) val filter = Filter(unresolvedExpression, DummyRelation(tableAttrs)) val optimizedLogicalPlan = session.sessionState.executePlan(filter).optimizedPlan - optimizedLogicalPlan.collectFirst { - case filter: Filter => filter.condition - case _: DummyRelation => Literal.TrueLiteral - case _: LocalRelation => Literal.FalseLiteral - }.getOrElse(throw new AnalysisException("Failed to find filter expression")) + optimizedLogicalPlan + .collectFirst { + case filter: Filter => filter.condition + case _: DummyRelation => Literal.TrueLiteral + case _: LocalRelation => Literal.FalseLiteral + } + .getOrElse(throw new AnalysisException("Failed to find filter expression")) } case class DummyRelation(output: Seq[Attribute]) extends LeafNode diff --git a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala index cca16960f434..10e25311dfb5 100644 --- a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala +++ b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.stats import java.nio.ByteBuffer @@ -46,12 +45,13 @@ import org.apache.spark.unsafe.types.UTF8String * which should be deserialized to Compact sketch before using. * * See [[https://iceberg.apache.org/puffin-spec/]] for more information. - * */ case class ThetaSketchAgg( - child: Expression, - mutableAggBufferOffset: Int = 0, - inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[Sketch] with UnaryLike[Expression] { + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) + extends TypedImperativeAggregate[Sketch] + with UnaryLike[Expression] { private lazy val icebergType = SparkSchemaUtil.convert(child.dataType) @@ -102,7 +102,8 @@ case class ThetaSketchAgg( CompactSketch.wrap(Memory.wrap(storageFormat)) } - override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = { + override def withNewMutableAggBufferOffset( + newMutableAggBufferOffset: Int): ImperativeAggregate = { copy(mutableAggBufferOffset = newMutableAggBufferOffset) } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala index aa0d7430a83a..54d2f8002e86 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.iceberg.spark.extensions import org.apache.spark.sql.SparkSessionExtensions @@ -42,12 +41,12 @@ class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) { extensions.injectResolutionRule { spark => ResolveViews(spark) } extensions.injectResolutionRule { _ => ProcedureArgumentCoercion } extensions.injectCheckRule(_ => CheckViews) - extensions.injectResolutionRule { _ => RewriteUpdateTableForRowLineage} - extensions.injectResolutionRule { _ => RewriteMergeIntoTableForRowLineage} + extensions.injectResolutionRule { _ => RewriteUpdateTableForRowLineage } + extensions.injectResolutionRule { _ => RewriteMergeIntoTableForRowLineage } // optimizer extensions extensions.injectOptimizerRule { _ => ReplaceStaticInvoke } - extensions.injectOptimizerRule { _ => RemoveRowLineageOutputFromOriginalTable} + extensions.injectOptimizerRule { _ => RemoveRowLineageOutputFromOriginalTable } // planner extensions extensions.injectPlannerStrategy { spark => ExtendedDataSourceV2Strategy(spark) } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala index 2a0e573b18e3..aefc672dc878 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException @@ -38,23 +37,40 @@ object CheckViews extends (LogicalPlan => Unit) { override def apply(plan: LogicalPlan): Unit = { plan foreach { - case CreateIcebergView(resolvedIdent@ResolvedIdentifier(_: ViewCatalog, _), _, query, columnAliases, _, - _, _, _, _, replace, _) => + case CreateIcebergView( + resolvedIdent @ ResolvedIdentifier(_: ViewCatalog, _), + _, + query, + columnAliases, + _, + _, + _, + _, + _, + replace, + _) => verifyColumnCount(resolvedIdent, columnAliases, query) - SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames.toIndexedSeq, SQLConf.get.resolver) + SchemaUtils.checkColumnNameDuplication( + query.schema.fieldNames.toIndexedSeq, + SQLConf.get.resolver) if (replace) { - val viewIdent: Seq[String] = resolvedIdent.catalog.name() +: resolvedIdent.identifier.asMultipartIdentifier + val viewIdent: Seq[String] = + resolvedIdent.catalog.name() +: resolvedIdent.identifier.asMultipartIdentifier checkCyclicViewReference(viewIdent, query, Seq(viewIdent)) } case AlterViewAs(ResolvedV2View(_, _), _, _) => - throw new AnalysisException("ALTER VIEW AS is not supported. Use CREATE OR REPLACE VIEW instead") + throw new AnalysisException( + "ALTER VIEW AS is not supported. Use CREATE OR REPLACE VIEW instead") case _ => // OK } } - private def verifyColumnCount(ident: ResolvedIdentifier, columns: Seq[String], query: LogicalPlan): Unit = { + private def verifyColumnCount( + ident: ResolvedIdentifier, + columns: Seq[String], + query: LogicalPlan): Unit = { if (columns.nonEmpty) { if (columns.length > query.output.length) { throw new AnalysisException( @@ -75,11 +91,11 @@ object CheckViews extends (LogicalPlan => Unit) { } private def checkCyclicViewReference( - viewIdent: Seq[String], - plan: LogicalPlan, - cyclePath: Seq[Seq[String]]): Unit = { + viewIdent: Seq[String], + plan: LogicalPlan, + cyclePath: Seq[Seq[String]]): Unit = { plan match { - case sub@SubqueryAlias(_, Project(_, _)) => + case sub @ SubqueryAlias(_, Project(_, _)) => val currentViewIdent: Seq[String] = sub.identifier.qualifier :+ sub.identifier.name checkIfRecursiveView(viewIdent, currentViewIdent, cyclePath, sub.children) case v1View: View => @@ -98,15 +114,17 @@ object CheckViews extends (LogicalPlan => Unit) { } private def checkIfRecursiveView( - viewIdent: Seq[String], - currentViewIdent: Seq[String], - cyclePath: Seq[Seq[String]], - children: Seq[LogicalPlan] - ): Unit = { + viewIdent: Seq[String], + currentViewIdent: Seq[String], + cyclePath: Seq[Seq[String]], + children: Seq[LogicalPlan]): Unit = { val newCyclePath = cyclePath :+ currentViewIdent if (currentViewIdent == viewIdent) { - throw new AnalysisException(String.format("Recursive cycle in view detected: %s (cycle: %s)", - viewIdent.asIdentifier, newCyclePath.map(p => p.mkString(".")).mkString(" -> "))) + throw new AnalysisException( + String.format( + "Recursive cycle in view detected: %s (cycle: %s)", + viewIdent.asIdentifier, + newCyclePath.map(p => p.mkString(".")).mkString(" -> "))) } else { children.foreach { c => checkCyclicViewReference(viewIdent, c, newCyclePath) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala index 7f0ca8fadded..aadc332181e2 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ProcedureArgumentCoercion.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala index ee69b5e344f0..c2d13bfa021e 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveProcedures.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import java.util.Locale @@ -61,7 +60,8 @@ case class ResolveProcedures(spark: SparkSession) extends Rule[LogicalPlan] with } if (duplicateParamNames.nonEmpty) { - throw new AnalysisException(s"Duplicate parameter names: ${duplicateParamNames.mkString("[", ",", "]")}") + throw new AnalysisException( + s"Duplicate parameter names: ${duplicateParamNames.mkString("[", ",", "]")}") } // optional params should be at the end @@ -89,7 +89,8 @@ case class ResolveProcedures(spark: SparkSession) extends Rule[LogicalPlan] with } if (missingParamNames.nonEmpty) { - throw new AnalysisException(s"Missing required parameters: ${missingParamNames.mkString("[", ",", "]")}") + throw new AnalysisException( + s"Missing required parameters: ${missingParamNames.mkString("[", ",", "]")}") } val argExprs = new Array[Expression](params.size) @@ -141,7 +142,8 @@ case class ResolveProcedures(spark: SparkSession) extends Rule[LogicalPlan] with } if (validationErrors.nonEmpty) { - throw new AnalysisException(s"Could not build name to arg map: ${validationErrors.mkString(", ")}") + throw new AnalysisException( + s"Could not build name to arg map: ${validationErrors.mkString(", ")}") } namedArgs.map(arg => arg.name -> arg).toMap diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala index 0fb5d30a91d5..1e0e5e8951ba 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.SparkSession @@ -47,36 +46,52 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case u@UnresolvedRelation(nameParts, _, _) - if catalogManager.v1SessionCatalog.isTempView(nameParts) => + case u @ UnresolvedRelation(nameParts, _, _) + if catalogManager.v1SessionCatalog.isTempView(nameParts) => u - case u@UnresolvedRelation(parts@CatalogAndIdentifier(catalog, ident), _, _) => - ViewUtil.loadView(catalog, ident) + case u @ UnresolvedRelation(parts @ CatalogAndIdentifier(catalog, ident), _, _) => + ViewUtil + .loadView(catalog, ident) .map(createViewRelation(parts, _)) .getOrElse(u) - case u@UnresolvedTableOrView(CatalogAndIdentifier(catalog, ident), _, _) => - ViewUtil.loadView(catalog, ident) + case u @ UnresolvedTableOrView(CatalogAndIdentifier(catalog, ident), _, _) => + ViewUtil + .loadView(catalog, ident) .map(_ => ResolvedV2View(catalog.asViewCatalog, ident)) .getOrElse(u) - case c@CreateIcebergView(ResolvedIdentifier(_, _), _, query, columnAliases, columnComments, _, _, _, _, _, _) - if query.resolved && !c.rewritten => + case c @ CreateIcebergView( + ResolvedIdentifier(_, _), + _, + query, + columnAliases, + columnComments, + _, + _, + _, + _, + _, + _) if query.resolved && !c.rewritten => val aliased = aliasColumns(query, columnAliases, columnComments) - c.copy(query = aliased, queryColumnNames = query.schema.fieldNames.toIndexedSeq, rewritten = true) + c.copy( + query = aliased, + queryColumnNames = query.schema.fieldNames.toIndexedSeq, + rewritten = true) } private def aliasColumns( - plan: LogicalPlan, - columnAliases: Seq[String], - columnComments: Seq[Option[String]]): LogicalPlan = { + plan: LogicalPlan, + columnAliases: Seq[String], + columnComments: Seq[Option[String]]): LogicalPlan = { if (columnAliases.isEmpty || columnAliases.length != plan.output.length) { plan } else { val projectList = plan.output.zipWithIndex.map { case (attr, pos) => if (columnComments.apply(pos).isDefined) { - val meta = new MetadataBuilder().putString("comment", columnComments.apply(pos).get).build() + val meta = + new MetadataBuilder().putString("comment", columnComments.apply(pos).get).build() Alias(attr, columnAliases.apply(pos))(explicitMetadata = Some(meta)) } else { Alias(attr, columnAliases.apply(pos))() @@ -86,7 +101,6 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look } } - private def createViewRelation(nameParts: Seq[String], view: View): LogicalPlan = { val parsed = parseViewText(nameParts.quoted, view.query) @@ -99,17 +113,15 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look // This is more strict because it doesn't allow resolution by field name. val aliases = view.schema.fields.zipWithIndex.map { case (expected, pos) => val attr = GetColumnByOrdinal(pos, expected.dataType) - Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = Some(expected.metadata)) + Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = + Some(expected.metadata)) }.toIndexedSeq SubqueryAlias(nameParts, Project(aliases, rewritten)) } private def parseViewText(name: String, viewText: String): LogicalPlan = { - val origin = Origin( - objectType = Some("VIEW"), - objectName = Some(name) - ) + val origin = Origin(objectType = Some("VIEW"), objectName = Some(name)) try { CurrentOrigin.withOrigin(origin) { @@ -122,8 +134,8 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look } private def rewriteIdentifiers( - plan: LogicalPlan, - catalogAndNamespace: Seq[String]): LogicalPlan = { + plan: LogicalPlan, + catalogAndNamespace: Seq[String]): LogicalPlan = { // Substitute CTEs and Unresolved Ordinals within the view, then rewrite unresolved functions and relations qualifyTableIdentifiers( qualifyFunctionIdentifiers( @@ -133,15 +145,15 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look } private def qualifyFunctionIdentifiers( - plan: LogicalPlan, - catalogAndNamespace: Seq[String]): LogicalPlan = plan transformExpressions { - case u@UnresolvedFunction(Seq(name), _, _, _, _) => + plan: LogicalPlan, + catalogAndNamespace: Seq[String]): LogicalPlan = plan transformExpressions { + case u @ UnresolvedFunction(Seq(name), _, _, _, _) => if (!isBuiltinFunction(name)) { u.copy(nameParts = catalogAndNamespace :+ name) } else { u } - case u@UnresolvedFunction(parts, _, _, _, _) if !isCatalog(parts.head) => + case u @ UnresolvedFunction(parts, _, _, _, _) if !isCatalog(parts.head) => u.copy(nameParts = catalogAndNamespace.head +: parts) } @@ -149,17 +161,16 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look * Qualify table identifiers with default catalog and namespace if necessary. */ private def qualifyTableIdentifiers( - child: LogicalPlan, - catalogAndNamespace: Seq[String]): LogicalPlan = + child: LogicalPlan, + catalogAndNamespace: Seq[String]): LogicalPlan = child transform { - case u@UnresolvedRelation(Seq(table), _, _) => + case u @ UnresolvedRelation(Seq(table), _, _) => u.copy(multipartIdentifier = catalogAndNamespace :+ table) - case u@UnresolvedRelation(parts, _, _) if !isCatalog(parts.head) => + case u @ UnresolvedRelation(parts, _, _) if !isCatalog(parts.head) => u.copy(multipartIdentifier = catalogAndNamespace.head +: parts) case other => - other.transformExpressions { - case subquery: SubqueryExpression => - subquery.withNewPlan(qualifyTableIdentifiers(subquery.plan, catalogAndNamespace)) + other.transformExpressions { case subquery: SubqueryExpression => + subquery.withNewPlan(qualifyTableIdentifiers(subquery.plan, catalogAndNamespace)) } } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTableForRowLineage.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTableForRowLineage.scala index 069caf1a71ca..39d07c40eae4 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTableForRowLineage.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTableForRowLineage.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Literal @@ -31,9 +30,9 @@ object RewriteMergeIntoTableForRowLineage extends RewriteOperationForRowLineage override def apply(plan: LogicalPlan): LogicalPlan = { plan.resolveOperators { case m @ MergeIntoTable(_, _, _, matchedActions, _, notMatchedBySourceActions) - if m.resolved && m.rewritable && m.aligned && - (matchedActions.nonEmpty || notMatchedBySourceActions.nonEmpty) && - shouldUpdatePlan(m.targetTable) => + if m.resolved && m.rewritable && m.aligned && + (matchedActions.nonEmpty || notMatchedBySourceActions.nonEmpty) && + shouldUpdatePlan(m.targetTable) => updateMergeIntoForRowLineage(m) } } @@ -47,8 +46,11 @@ object RewriteMergeIntoTableForRowLineage extends RewriteOperationForRowLineage val matchedAssignmentsForLineage = matchedActions.map { case UpdateAction(cond, assignments) => - UpdateAction(cond, assignments ++ Seq(Assignment(rowId, rowId), - Assignment(lastUpdatedSequenceNumber, Literal(null)))) + UpdateAction( + cond, + assignments ++ Seq( + Assignment(rowId, rowId), + Assignment(lastUpdatedSequenceNumber, Literal(null)))) case deleteAction => deleteAction } @@ -58,8 +60,11 @@ object RewriteMergeIntoTableForRowLineage extends RewriteOperationForRowLineage // during alignment in ResolveRowLevelCommandAssignments. val notMatchedBySourceActionsForLineage = notMatchedBySourceActions.map { case UpdateAction(cond, assignments) => - UpdateAction(cond, assignments ++ Seq(Assignment(rowId, rowId), - Assignment(lastUpdatedSequenceNumber, Literal(null)))) + UpdateAction( + cond, + assignments ++ Seq( + Assignment(rowId, rowId), + Assignment(lastUpdatedSequenceNumber, Literal(null)))) case deleteAction => deleteAction } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteOperationForRowLineage.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteOperationForRowLineage.scala index 55c248178628..5be30ac60633 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteOperationForRowLineage.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteOperationForRowLineage.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.iceberg.MetadataColumns @@ -32,7 +31,8 @@ import org.apache.spark.sql.types.MetadataBuilder trait RewriteOperationForRowLineage extends RewriteRowLevelCommand { protected val ROW_ID_ATTRIBUTE_NAME = MetadataColumns.ROW_ID.name() - protected val LAST_UPDATED_SEQUENCE_NUMBER_ATTRIBUTE_NAME = MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name() + protected val LAST_UPDATED_SEQUENCE_NUMBER_ATTRIBUTE_NAME = + MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name() // The plan should only be updated if the underlying Iceberg table supports row lineage AND // lineage attributes are not already on the output of operation which indicates the rule already ran @@ -52,16 +52,17 @@ trait RewriteOperationForRowLineage extends RewriteRowLevelCommand { } protected def findRowLineageAttributes( - expressions: Seq[Expression] - ): Option[(AttributeReference, AttributeReference)] = { + expressions: Seq[Expression]): Option[(AttributeReference, AttributeReference)] = { val rowIdAttr = expressions.collectFirst { case attr: AttributeReference - if isMetadataColumn(attr) && attr.name == ROW_ID_ATTRIBUTE_NAME => attr + if isMetadataColumn(attr) && attr.name == ROW_ID_ATTRIBUTE_NAME => + attr } val lastUpdatedAttr = expressions.collectFirst { case attr: AttributeReference - if isMetadataColumn(attr) && attr.name == LAST_UPDATED_SEQUENCE_NUMBER_ATTRIBUTE_NAME => attr + if isMetadataColumn(attr) && attr.name == LAST_UPDATED_SEQUENCE_NUMBER_ATTRIBUTE_NAME => + attr } // Treat row lineage columns as data columns by removing the metadata attribute @@ -78,7 +79,8 @@ trait RewriteOperationForRowLineage extends RewriteRowLevelCommand { attr.withMetadata( new MetadataBuilder() .withMetadata(attr.metadata) - .remove(METADATA_COL_ATTR_KEY).build()) + .remove(METADATA_COL_ATTR_KEY) + .build()) } private def isMetadataColumn(attributeReference: AttributeReference): Boolean = { diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteUpdateTableForRowLineage.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteUpdateTableForRowLineage.scala index ea8fd536bed2..8a8542e10056 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteUpdateTableForRowLineage.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteUpdateTableForRowLineage.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Literal @@ -30,17 +29,17 @@ object RewriteUpdateTableForRowLineage extends RewriteOperationForRowLineage { override def apply(plan: LogicalPlan): LogicalPlan = { plan resolveOperators { - case updateTable@UpdateTable(_, _, _) if shouldUpdatePlan(updateTable.table) => + case updateTable @ UpdateTable(_, _, _) if shouldUpdatePlan(updateTable.table) => updatePlanWithRowLineage(updateTable) } } - private def updatePlanWithRowLineage(updateTable: UpdateTable): LogicalPlan = { EliminateSubqueryAliases(updateTable.table) match { case r @ DataSourceV2Relation(_: SupportsRowLevelOperations, _, _, _, _) => val lineageAttributes = findRowLineageAttributes(r.metadataOutput).get - val (rowId, lastUpdatedSequence) = (removeMetadataColumnAttribute(lineageAttributes._1), + val (rowId, lastUpdatedSequence) = ( + removeMetadataColumnAttribute(lineageAttributes._1), removeMetadataColumnAttribute(lineageAttributes._2)) val lineageAssignments = updateTable.assignments ++ diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index 800ad1f66abc..c47b7d6ef6ac 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException @@ -52,11 +51,19 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case DropView(ResolvedIdent(resolved), ifExists) => DropIcebergView(resolved, ifExists) - case CreateView(ResolvedIdent(resolved), userSpecifiedColumns, comment, properties, - Some(queryText), query, allowExisting, replace) => + case CreateView( + ResolvedIdent(resolved), + userSpecifiedColumns, + comment, + properties, + Some(queryText), + query, + allowExisting, + replace) => val q = CTESubstitution.apply(query) verifyTemporaryObjectsDontExist(resolved, q) - CreateIcebergView(child = resolved, + CreateIcebergView( + child = resolved, queryText = queryText, query = q, columnAliases = userSpecifiedColumns.map(_._1), @@ -68,20 +75,25 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case view @ ShowViews(UnresolvedNamespace(Seq()), pattern, output) => if (ViewUtil.isViewCatalog(catalogManager.currentCatalog)) { - ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace.toIndexedSeq), - pattern, output) + ShowIcebergViews( + ResolvedNamespace( + catalogManager.currentCatalog, + catalogManager.currentNamespace.toIndexedSeq), + pattern, + output) } else { view } case ShowViews(UnresolvedNamespace(CatalogAndNamespace(catalog, ns)), pattern, output) - if ViewUtil.isViewCatalog(catalog) => + if ViewUtil.isViewCatalog(catalog) => ShowIcebergViews(ResolvedNamespace(catalog, ns), pattern, output) // needs to be done here instead of in ResolveViews, so that a V2 view can be resolved before the Analyzer // tries to resolve it, which would result in an error, saying that V2 views aren't supported - case u@UnresolvedView(ResolvedView(resolved), _, _, _) => - ViewUtil.loadView(resolved.catalog, resolved.identifier) + case u @ UnresolvedView(ResolvedView(resolved), _, _, _) => + ViewUtil + .loadView(resolved.catalog, resolved.identifier) .map(_ => ResolvedV2View(resolved.catalog.asViewCatalog, resolved.identifier)) .getOrElse(u) } @@ -102,7 +114,8 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case UnresolvedIdentifier(nameParts, true) if isTempView(nameParts) => None - case UnresolvedIdentifier(CatalogAndIdentifier(catalog, ident), _) if ViewUtil.isViewCatalog(catalog) => + case UnresolvedIdentifier(CatalogAndIdentifier(catalog, ident), _) + if ViewUtil.isViewCatalog(catalog) => Some(ResolvedIdentifier(catalog, ident)) case _ => @@ -114,11 +127,14 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi * Permanent views are not allowed to reference temp objects */ private def verifyTemporaryObjectsDontExist( - identifier: ResolvedIdentifier, - child: LogicalPlan): Unit = { + identifier: ResolvedIdentifier, + child: LogicalPlan): Unit = { val tempViews = collectTemporaryViews(child) if (tempViews.nonEmpty) { - throw invalidRefToTempObject(identifier, tempViews.map(v => v.quoted).mkString("[", ", ", "]"), "view") + throw invalidRefToTempObject( + identifier, + tempViews.map(v => v.quoted).mkString("[", ", ", "]"), + "view") } val tempFunctions = collectTemporaryFunctions(child) @@ -127,9 +143,17 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi } } - private def invalidRefToTempObject(ident: ResolvedIdentifier, tempObjectNames: String, tempObjectType: String) = { - new AnalysisException(String.format("Cannot create view %s.%s that references temporary %s: %s", - ident.catalog.name(), ident.identifier, tempObjectType, tempObjectNames)) + private def invalidRefToTempObject( + ident: ResolvedIdentifier, + tempObjectNames: String, + tempObjectType: String) = { + new AnalysisException( + String.format( + "Cannot create view %s.%s that references temporary %s: %s", + ident.catalog.name(), + ident.identifier, + tempObjectType, + tempObjectNames)) } /** @@ -141,10 +165,11 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case unresolved: UnresolvedRelation if isTempView(unresolved.multipartIdentifier) => Seq(unresolved.multipartIdentifier) case view: View if view.isTempView => Seq(view.desc.identifier.nameParts) - case plan => plan.expressions.flatMap(_.flatMap { - case e: SubqueryExpression => collectTempViews(e.plan) - case _ => Seq.empty - }) + case plan => + plan.expressions.flatMap(_.flatMap { + case e: SubqueryExpression => collectTempViews(e.plan) + case _ => Seq.empty + }) }.distinct } @@ -157,7 +182,9 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi None case CatalogAndIdentifier(catalog, ident) if ViewUtil.isViewCatalog(catalog) => - ViewUtil.loadView(catalog, ident).flatMap(_ => Some(ResolvedV2View(catalog.asViewCatalog, ident))) + ViewUtil + .loadView(catalog, ident) + .flatMap(_ => Some(ResolvedV2View(catalog.asViewCatalog, ident))) case _ => None diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala index d46f10b7f5a2..a7188837c51e 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.connector.catalog.CatalogPlugin diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRowLineageOutputFromOriginalTable.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRowLineageOutputFromOriginalTable.scala index 70095f95ab8d..5fd98d03fa25 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRowLineageOutputFromOriginalTable.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRowLineageOutputFromOriginalTable.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.optimizer import org.apache.iceberg.MetadataColumns @@ -35,20 +34,19 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation object RemoveRowLineageOutputFromOriginalTable extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = { plan.transform { - case writeDelta@WriteDelta(_, _, _, originalTable, _, _) => + case writeDelta @ WriteDelta(_, _, _, originalTable, _, _) => writeDelta.copy(originalTable = removeRowLineageOutput(originalTable)) - case replaceData@ReplaceData(_, _, _, originalTable, _, _) => + case replaceData @ ReplaceData(_, _, _, originalTable, _, _) => replaceData.copy(originalTable = removeRowLineageOutput(originalTable)) } } private def removeRowLineageOutput(table: NamedRelation): DataSourceV2Relation = { table match { - case dsv2Relation@DataSourceV2Relation(_, _, _, _, _) => - dsv2Relation.copy(output = dsv2Relation.output.filterNot( - attr => attr.name == MetadataColumns.ROW_ID.name() || + case dsv2Relation @ DataSourceV2Relation(_, _, _, _, _) => + dsv2Relation.copy(output = dsv2Relation.output.filterNot(attr => + attr.name == MetadataColumns.ROW_ID.name() || attr.name == MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name())) } } } - diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala index d5c4cb84a02a..bd22d2ca1536 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala @@ -48,7 +48,7 @@ import org.apache.spark.sql.types.StructType object ReplaceStaticInvoke extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = - plan.transformWithPruning (_.containsAnyPattern(COMMAND, FILTER, JOIN)) { + plan.transformWithPruning(_.containsAnyPattern(COMMAND, FILTER, JOIN)) { case replace @ ReplaceData(_, cond, _, _, _, _) => replaceStaticInvoke(replace, cond, newCond => replace.copy(condition = newCond)) @@ -57,7 +57,7 @@ object ReplaceStaticInvoke extends Rule[LogicalPlan] { case filter @ Filter(cond, _) => replaceStaticInvoke(filter, cond, newCond => filter.copy(condition = newCond)) - } + } private def replaceStaticInvoke[T <: LogicalPlan]( node: T, @@ -90,16 +90,17 @@ object ReplaceStaticInvoke extends Rule[LogicalPlan] { return invoke } - val inputType = StructType(invoke.arguments.zipWithIndex.map { - case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable) + val inputType = StructType(invoke.arguments.zipWithIndex.map { case (exp, pos) => + StructField(s"_$pos", exp.dataType, exp.nullable) }) - val bound = try { - unbound.bind(inputType) - } catch { - case _: Exception => - return invoke - } + val bound = + try { + unbound.bind(inputType) + } catch { + case _: Exception => + return invoke + } if (bound.inputTypes().length != invoke.arguments.length) { return invoke diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala index 365a1428f5bf..b25333d56787 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.parser.extensions import java.util.Locale @@ -46,7 +45,9 @@ import org.apache.spark.sql.types.DataType import org.apache.spark.sql.types.StructType import scala.jdk.CollectionConverters._ -class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserInterface with ExtendedParser { +class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) + extends ParserInterface + with ExtendedParser { import IcebergSparkSqlExtensionsParser._ @@ -115,14 +116,17 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI override def parsePlan(sqlText: String): LogicalPlan = { val sqlTextAfterSubstitution = substitutor.substitute(sqlText) if (isIcebergCommand(sqlTextAfterSubstitution)) { - parse(sqlTextAfterSubstitution) { parser => astBuilder.visit(parser.singleStatement()) }.asInstanceOf[LogicalPlan] + parse(sqlTextAfterSubstitution) { parser => astBuilder.visit(parser.singleStatement()) } + .asInstanceOf[LogicalPlan] } else { RewriteViewCommands(SparkSession.active).apply(delegate.parsePlan(sqlText)) } } private def isIcebergCommand(sqlText: String): Boolean = { - val normalized = sqlText.toLowerCase(Locale.ROOT).trim() + val normalized = sqlText + .toLowerCase(Locale.ROOT) + .trim() // Strip simple SQL comments that terminate a line, e.g. comments starting with `--` . .replaceAll("--.*?\\n", " ") // Strip newlines. @@ -134,18 +138,17 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI .replaceAll("`", "") .trim() - isIcebergProcedure(normalized) || ( - normalized.startsWith("alter table") && ( - normalized.contains("add partition field") || - normalized.contains("drop partition field") || - normalized.contains("replace partition field") || - normalized.contains("write ordered by") || - normalized.contains("write locally ordered by") || - normalized.contains("write distributed by") || - normalized.contains("write unordered") || - normalized.contains("set identifier fields") || - normalized.contains("drop identifier fields") || - isSnapshotRefDdl(normalized))) + isIcebergProcedure(normalized) || (normalized + .startsWith("alter table") && (normalized.contains("add partition field") || + normalized.contains("drop partition field") || + normalized.contains("replace partition field") || + normalized.contains("write ordered by") || + normalized.contains("write locally ordered by") || + normalized.contains("write distributed by") || + normalized.contains("write unordered") || + normalized.contains("set identifier fields") || + normalized.contains("drop identifier fields") || + isSnapshotRefDdl(normalized))) } // All builtin Iceberg procedures are under the 'system' namespace @@ -156,15 +159,16 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI private def isSnapshotRefDdl(normalized: String): Boolean = { normalized.contains("create branch") || - normalized.contains("replace branch") || - normalized.contains("create tag") || - normalized.contains("replace tag") || - normalized.contains("drop branch") || - normalized.contains("drop tag") + normalized.contains("replace branch") || + normalized.contains("create tag") || + normalized.contains("replace tag") || + normalized.contains("drop branch") || + normalized.contains("drop tag") } protected def parse[T](command: String)(toResult: IcebergSqlExtensionsParser => T): T = { - val lexer = new IcebergSqlExtensionsLexer(new UpperCaseCharStream(CharStreams.fromString(command))) + val lexer = new IcebergSqlExtensionsLexer( + new UpperCaseCharStream(CharStreams.fromString(command))) lexer.removeErrorListeners() lexer.addErrorListener(IcebergParseErrorListener) @@ -182,8 +186,7 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI parser.setErrorHandler(new BailErrorStrategy) parser.getInterpreter.setPredictionMode(PredictionMode.SLL) toResult(parser) - } - catch { + } catch { case _: ParseCancellationException => // if we fail, parse with LL mode with DefaultErrorStrategy tokenStream.seek(0) // rewind input stream @@ -194,8 +197,7 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI parser.getInterpreter.setPredictionMode(PredictionMode.LL) toResult(parser) } - } - catch { + } catch { case e: IcebergParseException if e.command.isDefined => throw e case e: IcebergParseException => @@ -213,7 +215,8 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI object IcebergSparkSqlExtensionsParser { private val substitutorCtor: DynConstructors.Ctor[VariableSubstitution] = - DynConstructors.builder() + DynConstructors + .builder() .impl(classOf[VariableSubstitution]) .impl(classOf[VariableSubstitution], classOf[SQLConf]) .build() @@ -259,9 +262,7 @@ case object IcebergSqlExtensionsPostProcessor extends IcebergSqlExtensionsBaseLi replaceTokenByIdentifier(ctx, 0)(identity) } - private def replaceTokenByIdentifier( - ctx: ParserRuleContext, - stripMargins: Int)( + private def replaceTokenByIdentifier(ctx: ParserRuleContext, stripMargins: Int)( f: CommonToken => CommonToken = identity): Unit = { val parent = ctx.getParent parent.removeLastChild() @@ -308,10 +309,12 @@ class IcebergParseException( val command: Option[String], message: String, val start: Origin, - val stop: Origin) extends AnalysisException(message, start.line, start.startPosition) { + val stop: Origin) + extends AnalysisException(message, start.line, start.startPosition) { def this(message: String, ctx: ParserRuleContext) = { - this(Option(IcebergParserUtils.command(ctx)), + this( + Option(IcebergParserUtils.command(ctx)), message, IcebergParserUtils.position(ctx.getStart), IcebergParserUtils.position(ctx.getStop)) @@ -321,8 +324,7 @@ class IcebergParseException( val builder = new StringBuilder builder ++= "\n" ++= message start match { - case Origin( - Some(l), Some(p), Some(_), Some(_), Some(_), Some(_), Some(_)) => + case Origin(Some(l), Some(p), Some(_), Some(_), Some(_), Some(_), Some(_)) => builder ++= s"(line $l, pos $p)\n" command.foreach { cmd => val (above, below) = cmd.split("\n").splitAt(l) @@ -342,4 +344,4 @@ class IcebergParseException( def withCommand(cmd: String): IcebergParseException = { new IcebergParseException(Option(cmd), message, start, stop) } -} \ No newline at end of file +} diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala index 6b1cc41da04c..f87733e6b79b 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.parser.extensions import java.util.Locale @@ -63,9 +62,11 @@ import org.apache.spark.sql.connector.expressions.LiteralValue import org.apache.spark.sql.connector.expressions.Transform import scala.jdk.CollectionConverters._ -class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergSqlExtensionsBaseVisitor[AnyRef] { +class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) + extends IcebergSqlExtensionsBaseVisitor[AnyRef] { - private def toBuffer[T](list: java.util.List[T]): scala.collection.mutable.Buffer[T] = list.asScala + private def toBuffer[T](list: java.util.List[T]): scala.collection.mutable.Buffer[T] = + list.asScala private def toSeq[T](list: java.util.List[T]): Seq[T] = toBuffer(list).toSeq /** @@ -80,52 +81,59 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS /** * Create an ADD PARTITION FIELD logical command. */ - override def visitAddPartitionField(ctx: AddPartitionFieldContext): AddPartitionField = withOrigin(ctx) { - AddPartitionField( - typedVisit[Seq[String]](ctx.multipartIdentifier), - typedVisit[Transform](ctx.transform), - Option(ctx.name).map(_.getText)) - } + override def visitAddPartitionField(ctx: AddPartitionFieldContext): AddPartitionField = + withOrigin(ctx) { + AddPartitionField( + typedVisit[Seq[String]](ctx.multipartIdentifier), + typedVisit[Transform](ctx.transform), + Option(ctx.name).map(_.getText)) + } /** * Create a DROP PARTITION FIELD logical command. */ - override def visitDropPartitionField(ctx: DropPartitionFieldContext): DropPartitionField = withOrigin(ctx) { - DropPartitionField( - typedVisit[Seq[String]](ctx.multipartIdentifier), - typedVisit[Transform](ctx.transform)) - } + override def visitDropPartitionField(ctx: DropPartitionFieldContext): DropPartitionField = + withOrigin(ctx) { + DropPartitionField( + typedVisit[Seq[String]](ctx.multipartIdentifier), + typedVisit[Transform](ctx.transform)) + } /** * Create a CREATE OR REPLACE BRANCH logical command. */ - override def visitCreateOrReplaceBranch(ctx: CreateOrReplaceBranchContext): CreateOrReplaceBranch = withOrigin(ctx) { + override def visitCreateOrReplaceBranch( + ctx: CreateOrReplaceBranchContext): CreateOrReplaceBranch = withOrigin(ctx) { val createOrReplaceBranchClause = ctx.createReplaceBranchClause() val branchName = createOrReplaceBranchClause.identifier() val branchOptionsContext = Option(createOrReplaceBranchClause.branchOptions()) - val snapshotId = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.snapshotId())) + val snapshotId = branchOptionsContext + .flatMap(branchOptions => Option(branchOptions.snapshotId())) .map(_.getText.toLong) - val snapshotRetention = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.snapshotRetention())) - val minSnapshotsToKeep = snapshotRetention.flatMap(retention => Option(retention.minSnapshotsToKeep())) + val snapshotRetention = + branchOptionsContext.flatMap(branchOptions => Option(branchOptions.snapshotRetention())) + val minSnapshotsToKeep = snapshotRetention + .flatMap(retention => Option(retention.minSnapshotsToKeep())) .map(minSnapshots => minSnapshots.number().getText.toLong) val maxSnapshotAgeMs = snapshotRetention .flatMap(retention => Option(retention.maxSnapshotAge())) - .map(retention => TimeUnit.valueOf(retention.timeUnit().getText.toUpperCase(Locale.ENGLISH)) - .toMillis(retention.number().getText.toLong)) - val branchRetention = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.refRetain())) + .map(retention => + TimeUnit + .valueOf(retention.timeUnit().getText.toUpperCase(Locale.ENGLISH)) + .toMillis(retention.number().getText.toLong)) + val branchRetention = + branchOptionsContext.flatMap(branchOptions => Option(branchOptions.refRetain())) val branchRefAgeMs = branchRetention.map(retain => - TimeUnit.valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)).toMillis(retain.number().getText.toLong)) + TimeUnit + .valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)) + .toMillis(retain.number().getText.toLong)) val create = createOrReplaceBranchClause.CREATE() != null val replace = ctx.createReplaceBranchClause().REPLACE() != null val ifNotExists = createOrReplaceBranchClause.EXISTS() != null - val branchOptions = BranchOptions( - snapshotId, - minSnapshotsToKeep, - maxSnapshotAgeMs, - branchRefAgeMs - ) + val branchOptions = + BranchOptions(snapshotId, minSnapshotsToKeep, maxSnapshotAgeMs, branchRefAgeMs) CreateOrReplaceBranch( typedVisit[Seq[String]](ctx.multipartIdentifier), @@ -139,52 +147,61 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS /** * Create an CREATE OR REPLACE TAG logical command. */ - override def visitCreateOrReplaceTag(ctx: CreateOrReplaceTagContext): CreateOrReplaceTag = withOrigin(ctx) { - val createTagClause = ctx.createReplaceTagClause() - - val tagName = createTagClause.identifier().getText - - val tagOptionsContext = Option(createTagClause.tagOptions()) - val snapshotId = tagOptionsContext.flatMap(tagOptions => Option(tagOptions.snapshotId())) - .map(_.getText.toLong) - val tagRetain = tagOptionsContext.flatMap(tagOptions => Option(tagOptions.refRetain())) - val tagRefAgeMs = tagRetain.map(retain => - TimeUnit.valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)).toMillis(retain.number().getText.toLong)) - val tagOptions = TagOptions( - snapshotId, - tagRefAgeMs - ) - - val create = createTagClause.CREATE() != null - val replace = createTagClause.REPLACE() != null - val ifNotExists = createTagClause.EXISTS() != null - - CreateOrReplaceTag(typedVisit[Seq[String]](ctx.multipartIdentifier), - tagName, - tagOptions, - create, - replace, - ifNotExists) - } + override def visitCreateOrReplaceTag(ctx: CreateOrReplaceTagContext): CreateOrReplaceTag = + withOrigin(ctx) { + val createTagClause = ctx.createReplaceTagClause() + + val tagName = createTagClause.identifier().getText + + val tagOptionsContext = Option(createTagClause.tagOptions()) + val snapshotId = tagOptionsContext + .flatMap(tagOptions => Option(tagOptions.snapshotId())) + .map(_.getText.toLong) + val tagRetain = tagOptionsContext.flatMap(tagOptions => Option(tagOptions.refRetain())) + val tagRefAgeMs = tagRetain.map(retain => + TimeUnit + .valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)) + .toMillis(retain.number().getText.toLong)) + val tagOptions = TagOptions(snapshotId, tagRefAgeMs) + + val create = createTagClause.CREATE() != null + val replace = createTagClause.REPLACE() != null + val ifNotExists = createTagClause.EXISTS() != null + + CreateOrReplaceTag( + typedVisit[Seq[String]](ctx.multipartIdentifier), + tagName, + tagOptions, + create, + replace, + ifNotExists) + } /** * Create an DROP BRANCH logical command. */ override def visitDropBranch(ctx: DropBranchContext): DropBranch = withOrigin(ctx) { - DropBranch(typedVisit[Seq[String]](ctx.multipartIdentifier), ctx.identifier().getText, ctx.EXISTS() != null) + DropBranch( + typedVisit[Seq[String]](ctx.multipartIdentifier), + ctx.identifier().getText, + ctx.EXISTS() != null) } /** * Create an DROP TAG logical command. */ override def visitDropTag(ctx: DropTagContext): DropTag = withOrigin(ctx) { - DropTag(typedVisit[Seq[String]](ctx.multipartIdentifier), ctx.identifier().getText, ctx.EXISTS() != null) + DropTag( + typedVisit[Seq[String]](ctx.multipartIdentifier), + ctx.identifier().getText, + ctx.EXISTS() != null) } /** * Create an REPLACE PARTITION FIELD logical command. */ - override def visitReplacePartitionField(ctx: ReplacePartitionFieldContext): ReplacePartitionField = withOrigin(ctx) { + override def visitReplacePartitionField( + ctx: ReplacePartitionFieldContext): ReplacePartitionField = withOrigin(ctx) { ReplacePartitionField( typedVisit[Seq[String]](ctx.multipartIdentifier), typedVisit[Transform](ctx.transform(0)), @@ -195,20 +212,22 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS /** * Create an SET IDENTIFIER FIELDS logical command. */ - override def visitSetIdentifierFields(ctx: SetIdentifierFieldsContext): SetIdentifierFields = withOrigin(ctx) { - SetIdentifierFields( - typedVisit[Seq[String]](ctx.multipartIdentifier), - toSeq(ctx.fieldList.fields).map(_.getText)) - } + override def visitSetIdentifierFields(ctx: SetIdentifierFieldsContext): SetIdentifierFields = + withOrigin(ctx) { + SetIdentifierFields( + typedVisit[Seq[String]](ctx.multipartIdentifier), + toSeq(ctx.fieldList.fields).map(_.getText)) + } /** * Create an DROP IDENTIFIER FIELDS logical command. */ - override def visitDropIdentifierFields(ctx: DropIdentifierFieldsContext): DropIdentifierFields = withOrigin(ctx) { - DropIdentifierFields( - typedVisit[Seq[String]](ctx.multipartIdentifier), - toSeq(ctx.fieldList.fields).map(_.getText)) - } + override def visitDropIdentifierFields(ctx: DropIdentifierFieldsContext): DropIdentifierFields = + withOrigin(ctx) { + DropIdentifierFields( + typedVisit[Seq[String]](ctx.multipartIdentifier), + toSeq(ctx.fieldList.fields).map(_.getText)) + } /** * Create a [[SetWriteDistributionAndOrdering]] for changing the write distribution and ordering. @@ -266,12 +285,15 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS */ override def visitOrderField(ctx: OrderFieldContext): (Term, SortDirection, NullOrder) = { val term = Spark3Util.toIcebergTerm(typedVisit[Transform](ctx.transform)) - val direction = Option(ctx.ASC).map(_ => SortDirection.ASC) - .orElse(Option(ctx.DESC).map(_ => SortDirection.DESC)) - .getOrElse(SortDirection.ASC) - val nullOrder = Option(ctx.FIRST).map(_ => NullOrder.NULLS_FIRST) - .orElse(Option(ctx.LAST).map(_ => NullOrder.NULLS_LAST)) - .getOrElse(if (direction == SortDirection.ASC) NullOrder.NULLS_FIRST else NullOrder.NULLS_LAST) + val direction = Option(ctx.ASC) + .map(_ => SortDirection.ASC) + .orElse(Option(ctx.DESC).map(_ => SortDirection.DESC)) + .getOrElse(SortDirection.ASC) + val nullOrder = Option(ctx.FIRST) + .map(_ => NullOrder.NULLS_FIRST) + .orElse(Option(ctx.LAST).map(_ => NullOrder.NULLS_LAST)) + .getOrElse( + if (direction == SortDirection.ASC) NullOrder.NULLS_FIRST else NullOrder.NULLS_LAST) (term, direction, nullOrder) } @@ -293,35 +315,40 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS /** * Create a transform argument from a column reference or a constant. */ - override def visitTransformArgument(ctx: TransformArgumentContext): expressions.Expression = withOrigin(ctx) { - val reference = Option(ctx.multipartIdentifier()) + override def visitTransformArgument(ctx: TransformArgumentContext): expressions.Expression = + withOrigin(ctx) { + val reference = Option(ctx.multipartIdentifier()) .map(typedVisit[Seq[String]]) .map(FieldReference(_)) - val literal = Option(ctx.constant) + val literal = Option(ctx.constant) .map(visitConstant) .map(lit => LiteralValue(lit.value, lit.dataType)) - reference.orElse(literal) + reference + .orElse(literal) .getOrElse(throw new IcebergParseException(s"Invalid transform argument", ctx)) - } + } /** * Return a multi-part identifier as Seq[String]. */ - override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = withOrigin(ctx) { - toSeq(ctx.parts).map(_.getText) - } + override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = + withOrigin(ctx) { + toSeq(ctx.parts).map(_.getText) + } - override def visitSingleOrder(ctx: SingleOrderContext): Seq[(Term, SortDirection, NullOrder)] = withOrigin(ctx) { - toSeq(ctx.order.fields).map(typedVisit[(Term, SortDirection, NullOrder)]) - } + override def visitSingleOrder(ctx: SingleOrderContext): Seq[(Term, SortDirection, NullOrder)] = + withOrigin(ctx) { + toSeq(ctx.order.fields).map(typedVisit[(Term, SortDirection, NullOrder)]) + } /** * Create a positional argument in a stored procedure call. */ - override def visitPositionalArgument(ctx: PositionalArgumentContext): CallArgument = withOrigin(ctx) { - val expr = typedVisit[Expression](ctx.expression) - PositionalArgument(expr) - } + override def visitPositionalArgument(ctx: PositionalArgumentContext): CallArgument = + withOrigin(ctx) { + val expr = typedVisit[Expression](ctx.expression) + PositionalArgument(expr) + } /** * Create a named argument in a stored procedure call. @@ -350,10 +377,12 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS } private def reconstructSqlString(ctx: ParserRuleContext): String = { - toBuffer(ctx.children).map { - case c: ParserRuleContext => reconstructSqlString(c) - case t: TerminalNode => t.getText - }.mkString(" ") + toBuffer(ctx.children) + .map { + case c: ParserRuleContext => reconstructSqlString(c) + case t: TerminalNode => t.getText + } + .mkString(" ") } private def typedVisit[T](ctx: ParseTree): T = { diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala index e8b1b2941161..0a830dbd4f6a 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala @@ -16,13 +16,13 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.expressions.Transform -case class AddPartitionField(table: Seq[String], transform: Transform, name: Option[String]) extends LeafCommand { +case class AddPartitionField(table: Seq[String], transform: Transform, name: Option[String]) + extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala index 4d7e0a086bda..15b908300213 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala @@ -16,8 +16,10 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical -case class BranchOptions (snapshotId: Option[Long], numSnapshots: Option[Long], - snapshotRetain: Option[Long], snapshotRefRetain: Option[Long]) +case class BranchOptions( + snapshotId: Option[Long], + numSnapshots: Option[Long], + snapshotRetain: Option[Long], + snapshotRefRetain: Option[Long]) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala index 9e3fdb0e9e0e..7fb871bafddb 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Call.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala index b7981a3c7a0d..6900f6e8cc50 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute @@ -27,7 +26,8 @@ case class CreateOrReplaceBranch( branchOptions: BranchOptions, create: Boolean, replace: Boolean, - ifNotExists: Boolean) extends LeafCommand { + ifNotExists: Boolean) + extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala index 6e7db84a90fb..957c68e7a540 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute @@ -27,7 +26,8 @@ case class CreateOrReplaceTag( tagOptions: TagOptions, create: Boolean, replace: Boolean, - ifNotExists: Boolean) extends LeafCommand { + ifNotExists: Boolean) + extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala index bee0b0fae688..ed4f1f512b85 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala index 29dd686a0fba..1a91806280b3 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala @@ -16,14 +16,11 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute -case class DropIdentifierFields( - table: Seq[String], - fields: Seq[String]) extends LeafCommand { +case class DropIdentifierFields(table: Seq[String], fields: Seq[String]) extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala index fb1451324182..ec952c8c7118 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala index 7e4b38e74d2f..da69ca0383a1 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala index 8c660c6f37b1..c2525369e7c7 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute @@ -26,13 +25,14 @@ case class ReplacePartitionField( table: Seq[String], transformFrom: Transform, transformTo: Transform, - name: Option[String]) extends LeafCommand { + name: Option[String]) + extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil override def simpleString(maxFields: Int): String = { s"ReplacePartitionField ${table.quoted} ${transformFrom.describe} " + - s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" + s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" } } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala index 1e5622e9c374..8cd2c0ddad05 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala @@ -16,14 +16,11 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute -case class SetIdentifierFields( - table: Seq[String], - fields: Seq[String]) extends LeafCommand { +case class SetIdentifierFields(table: Seq[String], fields: Seq[String]) extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala index 85e3b95f4aba..6afe1478d747 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical case class TagOptions(snapshotId: Option[Long], snapshotRefRetain: Option[Long]) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala index be15f32bc1b8..9986b30c45ed 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statements.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Expression diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala index 9366d5efe163..c5dadae70110 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala @@ -16,29 +16,30 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical.views import org.apache.spark.sql.catalyst.plans.logical.BinaryCommand import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan case class CreateIcebergView( - child: LogicalPlan, - queryText: String, - query: LogicalPlan, - columnAliases: Seq[String], - columnComments: Seq[Option[String]], - queryColumnNames: Seq[String] = Seq.empty, - comment: Option[String], - properties: Map[String, String], - allowExisting: Boolean, - replace: Boolean, - rewritten: Boolean = false) extends BinaryCommand { + child: LogicalPlan, + queryText: String, + query: LogicalPlan, + columnAliases: Seq[String], + columnComments: Seq[Option[String]], + queryColumnNames: Seq[String] = Seq.empty, + comment: Option[String], + properties: Map[String, String], + allowExisting: Boolean, + replace: Boolean, + rewritten: Boolean = false) + extends BinaryCommand { override def left: LogicalPlan = child override def right: LogicalPlan = query override protected def withNewChildrenInternal( - newLeft: LogicalPlan, newRight: LogicalPlan): LogicalPlan = + newLeft: LogicalPlan, + newRight: LogicalPlan): LogicalPlan = copy(child = newLeft, query = newRight) } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala index 275dba6fbf5e..092b6b33fb0c 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala @@ -16,15 +16,12 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical.views import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.UnaryCommand -case class DropIcebergView( - child: LogicalPlan, - ifExists: Boolean) extends UnaryCommand { +case class DropIcebergView(child: LogicalPlan, ifExists: Boolean) extends UnaryCommand { override protected def withNewChildInternal(newChild: LogicalPlan): DropIcebergView = copy(child = newChild) } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala index b9c05ff0061d..4d384e857703 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical.views import org.apache.spark.sql.catalyst.analysis.LeafNodeWithoutStats @@ -24,8 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog -case class ResolvedV2View( - catalog: ViewCatalog, - identifier: Identifier) extends LeafNodeWithoutStats { +case class ResolvedV2View(catalog: ViewCatalog, identifier: Identifier) + extends LeafNodeWithoutStats { override def output: Seq[Attribute] = Nil } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala index b09c27acdc16..cbfe23d94cbe 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala @@ -16,8 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - - package org.apache.spark.sql.catalyst.plans.logical.views import org.apache.spark.sql.catalyst.expressions.Attribute @@ -26,9 +24,10 @@ import org.apache.spark.sql.catalyst.plans.logical.ShowViews import org.apache.spark.sql.catalyst.plans.logical.UnaryCommand case class ShowIcebergViews( - namespace: LogicalPlan, - pattern: Option[String], - override val output: Seq[Attribute] = ShowViews.getOutputAttrs) extends UnaryCommand { + namespace: LogicalPlan, + pattern: Option[String], + override val output: Seq[Attribute] = ShowViews.getOutputAttrs) + extends UnaryCommand { override def child: LogicalPlan = namespace override protected def withNewChildInternal(newChild: LogicalPlan): ShowIcebergViews = diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala index 55f327f7e45e..e28dcfb194b6 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.Spark3Util @@ -31,7 +30,8 @@ case class AddPartitionFieldExec( catalog: TableCatalog, ident: Identifier, transform: Transform, - name: Option[String]) extends LeafV2CommandExec { + name: Option[String]) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -39,12 +39,14 @@ case class AddPartitionFieldExec( override protected def run(): Seq[InternalRow] = { catalog.loadTable(ident) match { case iceberg: SparkTable => - iceberg.table.updateSpec() - .addField(name.orNull, Spark3Util.toIcebergTerm(transform)) - .commit() + iceberg.table + .updateSpec() + .addField(name.orNull, Spark3Util.toIcebergTerm(transform)) + .commit() case table => - throw new UnsupportedOperationException(s"Cannot add partition field to non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot add partition field to non-Iceberg table: $table") } Nil diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala index b103d1ee2c58..d6630e51ff5a 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -25,17 +24,17 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.connector.catalog.ViewChange - case class AlterV2ViewSetPropertiesExec( - catalog: ViewCatalog, - ident: Identifier, - properties: Map[String, String]) extends LeafV2CommandExec { + catalog: ViewCatalog, + ident: Identifier, + properties: Map[String, String]) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil override protected def run(): Seq[InternalRow] = { - val changes = properties.map { - case (property, value) => ViewChange.setProperty(property, value) + val changes = properties.map { case (property, value) => + ViewChange.setProperty(property, value) }.toSeq catalog.alterView(ident, changes: _*) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala index 91abd95f574c..7ebf0b9f7085 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.AnalysisException @@ -26,12 +25,12 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.connector.catalog.ViewChange - case class AlterV2ViewUnsetPropertiesExec( - catalog: ViewCatalog, - ident: Identifier, - propertyKeys: Seq[String], - ifExists: Boolean) extends LeafV2CommandExec { + catalog: ViewCatalog, + ident: Identifier, + propertyKeys: Seq[String], + ifExists: Boolean) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala index f66962a8c453..a12277502521 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CallExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -25,10 +24,8 @@ import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.connector.iceberg.catalog.Procedure import scala.collection.compat.immutable.ArraySeq -case class CallExec( - output: Seq[Attribute], - procedure: Procedure, - input: InternalRow) extends LeafV2CommandExec { +case class CallExec(output: Seq[Attribute], procedure: Procedure, input: InternalRow) + extends LeafV2CommandExec { override protected def run(): Seq[InternalRow] = { ArraySeq.unsafeWrapArray(procedure.call(input)) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala index 2be406e7f344..baf985f53a22 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.relocated.com.google.common.base.Preconditions @@ -28,13 +27,14 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog case class CreateOrReplaceBranchExec( - catalog: TableCatalog, - ident: Identifier, - branch: String, - branchOptions: BranchOptions, - create: Boolean, - replace: Boolean, - ifNotExists: Boolean) extends LeafV2CommandExec { + catalog: TableCatalog, + ident: Identifier, + branch: String, + branchOptions: BranchOptions, + create: Boolean, + replace: Boolean, + ifNotExists: Boolean) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -62,8 +62,10 @@ case class CreateOrReplaceBranchExec( if (create && replace && !refExists) { safeCreateBranch() } else if (replace) { - Preconditions.checkArgument(snapshotId != null, - "Cannot complete replace branch operation on %s, main has no snapshot", ident) + Preconditions.checkArgument( + snapshotId != null, + "Cannot complete replace branch operation on %s, main has no snapshot", + ident) manageSnapshots.replaceBranch(branch, snapshotId) } else { if (refExists && ifNotExists) { @@ -88,7 +90,8 @@ case class CreateOrReplaceBranchExec( manageSnapshots.commit() case table => - throw new UnsupportedOperationException(s"Cannot create or replace branch on non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot create or replace branch on non-Iceberg table: $table") } Nil diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala index 372cd7548632..e486892614cb 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.relocated.com.google.common.base.Preconditions @@ -33,7 +32,8 @@ case class CreateOrReplaceTagExec( tagOptions: TagOptions, create: Boolean, replace: Boolean, - ifNotExists: Boolean) extends LeafV2CommandExec { + ifNotExists: Boolean) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -47,8 +47,10 @@ case class CreateOrReplaceTagExec( .map(java.lang.Long.valueOf) .orNull - Preconditions.checkArgument(snapshotId != null, - "Cannot complete create or replace tag operation on %s, main has no snapshot", ident) + Preconditions.checkArgument( + snapshotId != null, + "Cannot complete create or replace tag operation on %s, main has no snapshot", + ident) val manageSnapshot = iceberg.table.manageSnapshots() val refExists = null != iceberg.table().refs().get(tag) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala index eb4cc7de48c4..c9d404765845 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.SupportsReplaceView @@ -29,31 +28,33 @@ import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.types.StructType import scala.jdk.CollectionConverters._ - case class CreateV2ViewExec( - catalog: ViewCatalog, - ident: Identifier, - queryText: String, - viewSchema: StructType, - columnAliases: Seq[String], - columnComments: Seq[Option[String]], - queryColumnNames: Seq[String], - comment: Option[String], - properties: Map[String, String], - allowExisting: Boolean, - replace: Boolean) extends LeafV2CommandExec { + catalog: ViewCatalog, + ident: Identifier, + queryText: String, + viewSchema: StructType, + columnAliases: Seq[String], + columnComments: Seq[Option[String]], + queryColumnNames: Seq[String], + comment: Option[String], + properties: Map[String, String], + allowExisting: Boolean, + replace: Boolean) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil override protected def run(): Seq[InternalRow] = { val currentCatalogName = session.sessionState.catalogManager.currentCatalog.name - val currentCatalog = if (!catalog.name().equals(currentCatalogName)) currentCatalogName else null + val currentCatalog = + if (!catalog.name().equals(currentCatalogName)) currentCatalogName else null val currentNamespace = session.sessionState.catalogManager.currentNamespace val engineVersion = "Spark " + org.apache.spark.SPARK_VERSION val newProperties = properties ++ comment.map(ViewCatalog.PROP_COMMENT -> _) ++ - Map(ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion, + Map( + ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion, ViewCatalog.PROP_ENGINE_VERSION -> engineVersion) if (replace) { @@ -87,10 +88,10 @@ case class CreateV2ViewExec( } private def replaceView( - supportsReplaceView: SupportsReplaceView, - currentCatalog: String, - currentNamespace: Array[String], - newProperties: Map[String, String]) = { + supportsReplaceView: SupportsReplaceView, + currentCatalog: String, + currentNamespace: Array[String], + newProperties: Map[String, String]) = { supportsReplaceView.replaceView( ident, queryText, @@ -104,9 +105,9 @@ case class CreateV2ViewExec( } private def createView( - currentCatalog: String, - currentNamespace: Array[String], - newProperties: Map[String, String]) = { + currentCatalog: String, + currentNamespace: Array[String], + newProperties: Map[String, String]) = { catalog.createView( ident, queryText, diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala index a9c7312d4a90..f25ac8e3c3dc 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -27,10 +26,9 @@ import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode import scala.jdk.CollectionConverters._ -case class DescribeV2ViewExec( - output: Seq[Attribute], - view: View, - isExtended: Boolean) extends V2CommandExec with LeafExecNode { +case class DescribeV2ViewExec(output: Seq[Attribute], view: View, isExtended: Boolean) + extends V2CommandExec + with LeafExecNode { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -44,23 +42,22 @@ case class DescribeV2ViewExec( private def describeSchema: Seq[InternalRow] = view.schema().map { column => - toCatalystRow( - column.name, - column.dataType.simpleString, - column.getComment().getOrElse("")) + toCatalystRow(column.name, column.dataType.simpleString, column.getComment().getOrElse("")) } private def emptyRow: InternalRow = toCatalystRow("", "", "") private def describeExtended: Seq[InternalRow] = { val outputColumns = view.queryColumnNames.mkString("[", ", ", "]") - val properties: Map[String, String] = view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala + val properties: Map[String, String] = + view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala val viewCatalogAndNamespace: Seq[String] = view.name.split("\\.").take(2) - val viewProperties = properties.toSeq.sortBy(_._1).map { - case (key, value) => + val viewProperties = properties.toSeq + .sortBy(_._1) + .map { case (key, value) => s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" - }.mkString("[", ", ", "]") - + } + .mkString("[", ", ", "]") // omitting view text here because it is shown as // part of SHOW CREATE TABLE and can result in weird formatting in the DESCRIBE output @@ -69,7 +66,10 @@ case class DescribeV2ViewExec( toCatalystRow("View Catalog and Namespace", viewCatalogAndNamespace.quoted, "") :: toCatalystRow("View Query Output Columns", outputColumns, "") :: toCatalystRow("View Properties", viewProperties, "") :: - toCatalystRow("Created By", view.properties.getOrDefault(ViewCatalog.PROP_CREATE_ENGINE_VERSION, ""), "") :: + toCatalystRow( + "Created By", + view.properties.getOrDefault(ViewCatalog.PROP_CREATE_ENGINE_VERSION, ""), + "") :: Nil } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala index ff8f1820099a..e7d9c7b70d82 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.source.SparkTable @@ -29,7 +28,8 @@ case class DropBranchExec( catalog: TableCatalog, ident: Identifier, branch: String, - ifExists: Boolean) extends LeafV2CommandExec { + ifExists: Boolean) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala index dee778b474f9..87b18594d573 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.relocated.com.google.common.base.Preconditions @@ -27,10 +26,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog -case class DropIdentifierFieldsExec( - catalog: TableCatalog, - ident: Identifier, - fields: Seq[String]) extends LeafV2CommandExec { +case class DropIdentifierFieldsExec(catalog: TableCatalog, ident: Identifier, fields: Seq[String]) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -42,18 +39,24 @@ case class DropIdentifierFieldsExec( val identifierFieldNames = Sets.newHashSet(schema.identifierFieldNames) for (name <- fields) { - Preconditions.checkArgument(schema.findField(name) != null, - "Cannot complete drop identifier fields operation: field %s not found", name) - Preconditions.checkArgument(identifierFieldNames.contains(name), - "Cannot complete drop identifier fields operation: %s is not an identifier field", name) + Preconditions.checkArgument( + schema.findField(name) != null, + "Cannot complete drop identifier fields operation: field %s not found", + name) + Preconditions.checkArgument( + identifierFieldNames.contains(name), + "Cannot complete drop identifier fields operation: %s is not an identifier field", + name) identifierFieldNames.remove(name) } - iceberg.table.updateSchema() + iceberg.table + .updateSchema() .setIdentifierFields(identifierFieldNames) .commit(); case table => - throw new UnsupportedOperationException(s"Cannot drop identifier fields in non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot drop identifier fields in non-Iceberg table: $table") } Nil diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala index 9a153f0c004e..db43263e0e66 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.Spark3Util @@ -29,10 +28,8 @@ import org.apache.spark.sql.connector.expressions.FieldReference import org.apache.spark.sql.connector.expressions.IdentityTransform import org.apache.spark.sql.connector.expressions.Transform -case class DropPartitionFieldExec( - catalog: TableCatalog, - ident: Identifier, - transform: Transform) extends LeafV2CommandExec { +case class DropPartitionFieldExec(catalog: TableCatalog, ident: Identifier, transform: Transform) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -42,20 +39,24 @@ case class DropPartitionFieldExec( case iceberg: SparkTable => val schema = iceberg.table.schema transform match { - case IdentityTransform(FieldReference(parts)) if parts.size == 1 && schema.findField(parts.head) == null => + case IdentityTransform(FieldReference(parts)) + if parts.size == 1 && schema.findField(parts.head) == null => // the name is not present in the Iceberg schema, so it must be a partition field name, not a column name - iceberg.table.updateSpec() - .removeField(parts.head) - .commit() + iceberg.table + .updateSpec() + .removeField(parts.head) + .commit() case _ => - iceberg.table.updateSpec() - .removeField(Spark3Util.toIcebergTerm(transform)) - .commit() + iceberg.table + .updateSpec() + .removeField(Spark3Util.toIcebergTerm(transform)) + .commit() } case table => - throw new UnsupportedOperationException(s"Cannot drop partition field in non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot drop partition field in non-Iceberg table: $table") } Nil diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala index 8df88765a986..79b4a1525591 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.source.SparkTable @@ -25,11 +24,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog -case class DropTagExec( - catalog: TableCatalog, - ident: Identifier, - tag: String, - ifExists: Boolean) extends LeafV2CommandExec { +case class DropTagExec(catalog: TableCatalog, ident: Identifier, tag: String, ifExists: Boolean) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala index c35af1486fc7..6dd1188b78e8 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -25,11 +24,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog - -case class DropV2ViewExec( - catalog: ViewCatalog, - ident: Identifier, - ifExists: Boolean) extends LeafV2CommandExec { +case class DropV2ViewExec(catalog: ViewCatalog, ident: Identifier, ifExists: Boolean) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala index 6a4dc4460839..64b228a4247b 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.Spark3Util @@ -72,11 +71,28 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi AddPartitionFieldExec(catalog, ident, transform, name) :: Nil case CreateOrReplaceBranch( - IcebergCatalogAndIdentifier(catalog, ident), branch, branchOptions, create, replace, ifNotExists) => - CreateOrReplaceBranchExec(catalog, ident, branch, branchOptions, create, replace, ifNotExists) :: Nil + IcebergCatalogAndIdentifier(catalog, ident), + branch, + branchOptions, + create, + replace, + ifNotExists) => + CreateOrReplaceBranchExec( + catalog, + ident, + branch, + branchOptions, + create, + replace, + ifNotExists) :: Nil case CreateOrReplaceTag( - IcebergCatalogAndIdentifier(catalog, ident), tag, tagOptions, create, replace, ifNotExists) => + IcebergCatalogAndIdentifier(catalog, ident), + tag, + tagOptions, + create, + replace, + ifNotExists) => CreateOrReplaceTagExec(catalog, ident, tag, tagOptions, create, replace, ifNotExists) :: Nil case DropBranch(IcebergCatalogAndIdentifier(catalog, ident), branch, ifExists) => @@ -88,7 +104,11 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi case DropPartitionField(IcebergCatalogAndIdentifier(catalog, ident), transform) => DropPartitionFieldExec(catalog, ident, transform) :: Nil - case ReplacePartitionField(IcebergCatalogAndIdentifier(catalog, ident), transformFrom, transformTo, name) => + case ReplacePartitionField( + IcebergCatalogAndIdentifier(catalog, ident), + transformFrom, + transformTo, + name) => ReplacePartitionFieldExec(catalog, ident, transformFrom, transformTo, name) :: Nil case SetIdentifierFields(IcebergCatalogAndIdentifier(catalog, ident), fields) => @@ -98,13 +118,15 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi DropIdentifierFieldsExec(catalog, ident, fields) :: Nil case SetWriteDistributionAndOrdering( - IcebergCatalogAndIdentifier(catalog, ident), distributionMode, ordering) => + IcebergCatalogAndIdentifier(catalog, ident), + distributionMode, + ordering) => SetWriteDistributionAndOrderingExec(catalog, ident, distributionMode, ordering) :: Nil case OrderAwareCoalesce(numPartitions, coalescer, child) => OrderAwareCoalesceExec(numPartitions, coalescer, planLater(child)) :: Nil - case RenameTable(ResolvedV2View(oldCatalog: ViewCatalog, oldIdent), newName, isView@true) => + case RenameTable(ResolvedV2View(oldCatalog: ViewCatalog, oldIdent), newName, isView @ true) => val newIdent = Spark3Util.catalogAndIdentifier(spark, newName.toList.asJava) if (oldCatalog.name != newIdent.catalog().name()) { throw new AnalysisException( @@ -115,8 +137,18 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi case DropIcebergView(ResolvedIdentifier(viewCatalog: ViewCatalog, ident), ifExists) => DropV2ViewExec(viewCatalog, ident, ifExists) :: Nil - case CreateIcebergView(ResolvedIdentifier(viewCatalog: ViewCatalog, ident), queryText, query, - columnAliases, columnComments, queryColumnNames, comment, properties, allowExisting, replace, _) => + case CreateIcebergView( + ResolvedIdentifier(viewCatalog: ViewCatalog, ident), + queryText, + query, + columnAliases, + columnComments, + queryColumnNames, + comment, + properties, + allowExisting, + replace, + _) => CreateV2ViewExec( catalog = viewCatalog, ident = ident, diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala index 61d362044c3c..5dada1cab0bb 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -24,11 +23,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog - -case class RenameV2ViewExec( - catalog: ViewCatalog, - oldIdent: Identifier, - newIdent: Identifier) extends LeafV2CommandExec { +case class RenameV2ViewExec(catalog: ViewCatalog, oldIdent: Identifier, newIdent: Identifier) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil @@ -38,7 +34,6 @@ case class RenameV2ViewExec( Seq.empty } - override def simpleString(maxFields: Int): String = { s"RenameV2View ${oldIdent} to {newIdent}" } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala index fcae0a5defc4..00b998c49e83 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.Spark3Util @@ -34,7 +33,8 @@ case class ReplacePartitionFieldExec( ident: Identifier, transformFrom: Transform, transformTo: Transform, - name: Option[String]) extends LeafV2CommandExec { + name: Option[String]) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -44,22 +44,26 @@ case class ReplacePartitionFieldExec( case iceberg: SparkTable => val schema = iceberg.table.schema transformFrom match { - case IdentityTransform(FieldReference(parts)) if parts.size == 1 && schema.findField(parts.head) == null => + case IdentityTransform(FieldReference(parts)) + if parts.size == 1 && schema.findField(parts.head) == null => // the name is not present in the Iceberg schema, so it must be a partition field name, not a column name - iceberg.table.updateSpec() - .removeField(parts.head) - .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) - .commit() + iceberg.table + .updateSpec() + .removeField(parts.head) + .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) + .commit() case _ => - iceberg.table.updateSpec() - .removeField(Spark3Util.toIcebergTerm(transformFrom)) - .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) - .commit() + iceberg.table + .updateSpec() + .removeField(Spark3Util.toIcebergTerm(transformFrom)) + .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) + .commit() } case table => - throw new UnsupportedOperationException(s"Cannot replace partition field in non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot replace partition field in non-Iceberg table: $table") } Nil @@ -67,6 +71,6 @@ case class ReplacePartitionFieldExec( override def simpleString(maxFields: Int): String = { s"ReplacePartitionField ${catalog.name}.${ident.quoted} ${transformFrom.describe} " + - s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" + s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" } } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala index b50550ad38ef..50c53473ab60 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.source.SparkTable @@ -26,10 +25,8 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog import scala.jdk.CollectionConverters._ -case class SetIdentifierFieldsExec( - catalog: TableCatalog, - ident: Identifier, - fields: Seq[String]) extends LeafV2CommandExec { +case class SetIdentifierFieldsExec(catalog: TableCatalog, ident: Identifier, fields: Seq[String]) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -37,11 +34,13 @@ case class SetIdentifierFieldsExec( override protected def run(): Seq[InternalRow] = { catalog.loadTable(ident) match { case iceberg: SparkTable => - iceberg.table.updateSchema() + iceberg.table + .updateSchema() .setIdentifierFields(fields.asJava) .commit(); case table => - throw new UnsupportedOperationException(s"Cannot set identifier fields in non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot set identifier fields in non-Iceberg table: $table") } Nil diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala index c9004ddc5bda..9a10949d5e9e 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.DistributionMode @@ -36,7 +35,8 @@ case class SetWriteDistributionAndOrderingExec( catalog: TableCatalog, ident: Identifier, distributionMode: Option[DistributionMode], - sortOrder: Seq[(Term, SortDirection, NullOrder)]) extends LeafV2CommandExec { + sortOrder: Seq[(Term, SortDirection, NullOrder)]) + extends LeafV2CommandExec { import CatalogV2Implicits._ @@ -57,7 +57,8 @@ case class SetWriteDistributionAndOrderingExec( orderBuilder.commit() distributionMode.foreach { mode => - txn.updateProperties() + txn + .updateProperties() .set(WRITE_DISTRIBUTION_MODE, mode.modeName()) .commit() } @@ -65,7 +66,8 @@ case class SetWriteDistributionAndOrderingExec( txn.commitTransaction() case table => - throw new UnsupportedOperationException(s"Cannot set write order of non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot set write order of non-Iceberg table: $table") } Nil @@ -73,9 +75,11 @@ case class SetWriteDistributionAndOrderingExec( override def simpleString(maxFields: Int): String = { val tableIdent = s"${catalog.name}.${ident.quoted}" - val order = sortOrder.map { - case (term, direction, nullOrder) => s"$term $direction $nullOrder" - }.mkString(", ") + val order = sortOrder + .map { case (term, direction, nullOrder) => + s"$term $direction $nullOrder" + } + .mkString(", ") s"SetWriteDistributionAndOrdering $tableIdent $distributionMode $order" } } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala index ecd3e0d3c9ba..07ac4aeda8fb 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -28,7 +27,8 @@ import org.apache.spark.sql.execution.LeafExecNode import scala.jdk.CollectionConverters._ case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View) - extends V2CommandExec with LeafExecNode { + extends V2CommandExec + with LeafExecNode { override protected def run(): Seq[InternalRow] = { val builder = new StringBuilder @@ -43,7 +43,9 @@ case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View) private def showColumns(view: View, builder: StringBuilder): Unit = { val columns = concatByMultiLines( - view.schema().fields + view + .schema() + .fields .map(x => s"${x.name}${x.getComment().map(c => s" COMMENT '$c'").getOrElse("")}")) builder ++= columns } @@ -54,14 +56,11 @@ case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View) .foreach(builder.append) } - private def showProperties( - view: View, - builder: StringBuilder): Unit = { + private def showProperties(view: View, builder: StringBuilder): Unit = { val showProps = view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala if (showProps.nonEmpty) { - val props = conf.redactOptions(showProps).toSeq.sortBy(_._1).map { - case (key, value) => - s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" + val props = conf.redactOptions(showProps).toSeq.sortBy(_._1).map { case (key, value) => + s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" } builder ++= "TBLPROPERTIES " diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala index 553b80833087..ace43eb6c07b 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -26,25 +25,22 @@ import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode import scala.jdk.CollectionConverters._ -case class ShowV2ViewPropertiesExec( - output: Seq[Attribute], - view: View, - propertyKey: Option[String]) extends V2CommandExec with LeafExecNode { +case class ShowV2ViewPropertiesExec(output: Seq[Attribute], view: View, propertyKey: Option[String]) + extends V2CommandExec + with LeafExecNode { override protected def run(): Seq[InternalRow] = { propertyKey match { case Some(p) => - val propValue = properties.getOrElse(p, - s"View ${view.name()} does not have property: $p") + val propValue = properties.getOrElse(p, s"View ${view.name()} does not have property: $p") Seq(toCatalystRow(p, propValue)) case None => - properties.map { - case (k, v) => toCatalystRow(k, v) + properties.map { case (k, v) => + toCatalystRow(k, v) }.toSeq } } - private def properties = { view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala index 3aa85c3db54a..83a1f84aecba 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -27,10 +26,12 @@ import org.apache.spark.sql.execution.LeafExecNode import scala.collection.mutable.ArrayBuffer case class ShowV2ViewsExec( - output: Seq[Attribute], - catalog: ViewCatalog, - namespace: Seq[String], - pattern: Option[String]) extends V2CommandExec with LeafExecNode { + output: Seq[Attribute], + catalog: ViewCatalog, + namespace: Seq[String], + pattern: Option[String]) + extends V2CommandExec + with LeafExecNode { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -40,7 +41,8 @@ case class ShowV2ViewsExec( // handle GLOBAL VIEWS val globalTemp = session.sessionState.catalog.globalTempViewManager.database if (namespace.nonEmpty && globalTemp == namespace.head) { - pattern.map(p => session.sessionState.catalog.globalTempViewManager.listViewNames(p)) + pattern + .map(p => session.sessionState.catalog.globalTempViewManager.listViewNames(p)) .getOrElse(session.sessionState.catalog.globalTempViewManager.listViewNames("*")) .map(name => rows += toCatalystRow(globalTemp, name, true)) } else { @@ -53,7 +55,8 @@ case class ShowV2ViewsExec( } // include TEMP VIEWS - pattern.map(p => session.sessionState.catalog.listLocalTempViews(p)) + pattern + .map(p => session.sessionState.catalog.listLocalTempViews(p)) .getOrElse(session.sessionState.catalog.listLocalTempViews("*")) .map(v => rows += toCatalystRow(v.database.toArray.quoted, v.table, true)) diff --git a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala index 5acaa6800e68..5d5990c23b57 100644 --- a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala +++ b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.rdd.PartitionCoalescer @@ -26,10 +25,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute // this node doesn't extend RepartitionOperation on purpose to keep this logic isolated // and ignore it in optimizer rules such as CollapseRepartition -case class OrderAwareCoalesce( - numPartitions: Int, - coalescer: PartitionCoalescer, - child: LogicalPlan) extends OrderPreservingUnaryNode { +case class OrderAwareCoalesce(numPartitions: Int, coalescer: PartitionCoalescer, child: LogicalPlan) + extends OrderPreservingUnaryNode { override def output: Seq[Attribute] = child.output diff --git a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala index 7b599eb3da1d..3ce3d3e82ad8 100644 --- a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala +++ b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.iceberg.DistributionMode @@ -29,16 +28,19 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits case class SetWriteDistributionAndOrdering( table: Seq[String], distributionMode: Option[DistributionMode], - sortOrder: Seq[(Term, SortDirection, NullOrder)]) extends LeafCommand { + sortOrder: Seq[(Term, SortDirection, NullOrder)]) + extends LeafCommand { import CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil override def simpleString(maxFields: Int): String = { - val order = sortOrder.map { - case (term, direction, nullOrder) => s"$term $direction $nullOrder" - }.mkString(", ") + val order = sortOrder + .map { case (term, direction, nullOrder) => + s"$term $direction $nullOrder" + } + .mkString(", ") s"SetWriteDistributionAndOrdering ${table.quoted} $distributionMode $order" } } diff --git a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala index bf19ef8a2167..71b6107d048e 100644 --- a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala +++ b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.iceberg.NullOrder @@ -27,7 +26,9 @@ import org.apache.iceberg.expressions.Term class SortOrderParserUtil { - def collectSortOrder(tableSchema:Schema, sortOrder: Seq[(Term, SortDirection, NullOrder)]): SortOrder = { + def collectSortOrder( + tableSchema: Schema, + sortOrder: Seq[(Term, SortDirection, NullOrder)]): SortOrder = { val orderBuilder = SortOrder.builderFor(tableSchema) sortOrder.foreach { case (term, SortDirection.ASC, nullOrder) => diff --git a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala index aa9e9c553346..f46127f818e5 100644 --- a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala +++ b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.utils import org.apache.iceberg.spark.source.SparkTable diff --git a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala index 2ef99550524a..83dec98c0d9a 100644 --- a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala +++ b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution import org.apache.spark.rdd.PartitionCoalescer @@ -31,7 +30,8 @@ import org.apache.spark.sql.catalyst.plans.physical.UnknownPartitioning case class OrderAwareCoalesceExec( numPartitions: Int, coalescer: PartitionCoalescer, - child: SparkPlan) extends UnaryExecNode { + child: SparkPlan) + extends UnaryExecNode { override def output: Seq[Attribute] = child.output diff --git a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala index 95a0315e7189..c50584be8278 100644 --- a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala +++ b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources import org.apache.iceberg.spark.SparkV2Filters @@ -32,7 +31,8 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy object SparkExpressionConverter { - def convertToIcebergExpression(sparkExpression: Expression): org.apache.iceberg.expressions.Expression = { + def convertToIcebergExpression( + sparkExpression: Expression): org.apache.iceberg.expressions.Expression = { // Currently, it is a double conversion as we are converting Spark expression to Spark predicate // and then converting Spark predicate to Iceberg expression. // But these two conversions already exist and well tested. So, we are going with this approach. @@ -40,26 +40,33 @@ object SparkExpressionConverter { case Some(filter) => val converted = SparkV2Filters.convert(filter) if (converted == null) { - throw new IllegalArgumentException(s"Cannot convert Spark filter: $filter to Iceberg expression") + throw new IllegalArgumentException( + s"Cannot convert Spark filter: $filter to Iceberg expression") } converted case _ => - throw new IllegalArgumentException(s"Cannot translate Spark expression: $sparkExpression to data source filter") + throw new IllegalArgumentException( + s"Cannot translate Spark expression: $sparkExpression to data source filter") } } @throws[AnalysisException] - def collectResolvedSparkExpression(session: SparkSession, tableName: String, where: String): Expression = { + def collectResolvedSparkExpression( + session: SparkSession, + tableName: String, + where: String): Expression = { val tableAttrs = session.table(tableName).queryExecution.analyzed.output val unresolvedExpression = session.sessionState.sqlParser.parseExpression(where) val filter = Filter(unresolvedExpression, DummyRelation(tableAttrs)) val optimizedLogicalPlan = session.sessionState.executePlan(filter).optimizedPlan - optimizedLogicalPlan.collectFirst { - case filter: Filter => filter.condition - case _: DummyRelation => Literal.TrueLiteral - case _: LocalRelation => Literal.FalseLiteral - }.getOrElse(throw new AnalysisException("Failed to find filter expression")) + optimizedLogicalPlan + .collectFirst { + case filter: Filter => filter.condition + case _: DummyRelation => Literal.TrueLiteral + case _: LocalRelation => Literal.FalseLiteral + } + .getOrElse(throw new AnalysisException("Failed to find filter expression")) } case class DummyRelation(output: Seq[Attribute]) extends LeafNode diff --git a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala index cca16960f434..10e25311dfb5 100644 --- a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala +++ b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.stats import java.nio.ByteBuffer @@ -46,12 +45,13 @@ import org.apache.spark.unsafe.types.UTF8String * which should be deserialized to Compact sketch before using. * * See [[https://iceberg.apache.org/puffin-spec/]] for more information. - * */ case class ThetaSketchAgg( - child: Expression, - mutableAggBufferOffset: Int = 0, - inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[Sketch] with UnaryLike[Expression] { + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) + extends TypedImperativeAggregate[Sketch] + with UnaryLike[Expression] { private lazy val icebergType = SparkSchemaUtil.convert(child.dataType) @@ -102,7 +102,8 @@ case class ThetaSketchAgg( CompactSketch.wrap(Memory.wrap(storageFormat)) } - override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = { + override def withNewMutableAggBufferOffset( + newMutableAggBufferOffset: Int): ImperativeAggregate = { copy(mutableAggBufferOffset = newMutableAggBufferOffset) } diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala index 2dd4b08b3cca..c4de35010c6e 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.iceberg.spark.extensions import org.apache.spark.sql.SparkSessionExtensions diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala index 12c1fab36bd4..343fbe7a91b6 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException @@ -38,12 +37,25 @@ object CheckViews extends (LogicalPlan => Unit) { override def apply(plan: LogicalPlan): Unit = { plan foreach { - case CreateIcebergView(resolvedIdent@ResolvedIdentifier(_: ViewCatalog, _), _, query, columnAliases, _, - _, _, _, _, replace, _) => + case CreateIcebergView( + resolvedIdent @ ResolvedIdentifier(_: ViewCatalog, _), + _, + query, + columnAliases, + _, + _, + _, + _, + _, + replace, + _) => verifyColumnCount(resolvedIdent, columnAliases, query) - SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames.toIndexedSeq, SQLConf.get.resolver) + SchemaUtils.checkColumnNameDuplication( + query.schema.fieldNames.toIndexedSeq, + SQLConf.get.resolver) if (replace) { - val viewIdent: Seq[String] = resolvedIdent.catalog.name() +: resolvedIdent.identifier.asMultipartIdentifier + val viewIdent: Seq[String] = + resolvedIdent.catalog.name() +: resolvedIdent.identifier.asMultipartIdentifier checkCyclicViewReference(viewIdent, query, Seq(viewIdent)) } @@ -55,7 +67,10 @@ object CheckViews extends (LogicalPlan => Unit) { } } - private def verifyColumnCount(ident: ResolvedIdentifier, columns: Seq[String], query: LogicalPlan): Unit = { + private def verifyColumnCount( + ident: ResolvedIdentifier, + columns: Seq[String], + query: LogicalPlan): Unit = { if (columns.nonEmpty) { if (columns.length > query.output.length) { throw new AnalysisException( @@ -76,11 +91,11 @@ object CheckViews extends (LogicalPlan => Unit) { } private def checkCyclicViewReference( - viewIdent: Seq[String], - plan: LogicalPlan, - cyclePath: Seq[Seq[String]]): Unit = { + viewIdent: Seq[String], + plan: LogicalPlan, + cyclePath: Seq[Seq[String]]): Unit = { plan match { - case sub@SubqueryAlias(_, Project(_, _)) => + case sub @ SubqueryAlias(_, Project(_, _)) => val currentViewIdent: Seq[String] = sub.identifier.qualifier :+ sub.identifier.name checkIfRecursiveView(viewIdent, currentViewIdent, cyclePath, sub.children) case v1View: View => @@ -99,15 +114,17 @@ object CheckViews extends (LogicalPlan => Unit) { } private def checkIfRecursiveView( - viewIdent: Seq[String], - currentViewIdent: Seq[String], - cyclePath: Seq[Seq[String]], - children: Seq[LogicalPlan] - ): Unit = { + viewIdent: Seq[String], + currentViewIdent: Seq[String], + cyclePath: Seq[Seq[String]], + children: Seq[LogicalPlan]): Unit = { val newCyclePath = cyclePath :+ currentViewIdent if (currentViewIdent == viewIdent) { - throw new IcebergAnalysisException(String.format("Recursive cycle in view detected: %s (cycle: %s)", - viewIdent.asIdentifier, newCyclePath.map(p => p.mkString(".")).mkString(" -> "))) + throw new IcebergAnalysisException( + String.format( + "Recursive cycle in view detected: %s (cycle: %s)", + viewIdent.asIdentifier, + newCyclePath.map(p => p.mkString(".")).mkString(" -> "))) } else { children.foreach { c => checkCyclicViewReference(viewIdent, c, newCyclePath) diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala index fe9462523405..23d33352f134 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.SparkSession @@ -47,36 +46,52 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { - case u@UnresolvedRelation(nameParts, _, _) - if catalogManager.v1SessionCatalog.isTempView(nameParts) => + case u @ UnresolvedRelation(nameParts, _, _) + if catalogManager.v1SessionCatalog.isTempView(nameParts) => u - case u@UnresolvedRelation(parts@CatalogAndIdentifier(catalog, ident), _, _) => - ViewUtil.loadView(catalog, ident) + case u @ UnresolvedRelation(parts @ CatalogAndIdentifier(catalog, ident), _, _) => + ViewUtil + .loadView(catalog, ident) .map(createViewRelation(parts, _)) .getOrElse(u) - case u@UnresolvedTableOrView(CatalogAndIdentifier(catalog, ident), _, _) => - ViewUtil.loadView(catalog, ident) + case u @ UnresolvedTableOrView(CatalogAndIdentifier(catalog, ident), _, _) => + ViewUtil + .loadView(catalog, ident) .map(_ => ResolvedV2View(catalog.asViewCatalog, ident)) .getOrElse(u) - case c@CreateIcebergView(ResolvedIdentifier(_, _), _, query, columnAliases, columnComments, _, _, _, _, _, _) - if query.resolved && !c.rewritten => + case c @ CreateIcebergView( + ResolvedIdentifier(_, _), + _, + query, + columnAliases, + columnComments, + _, + _, + _, + _, + _, + _) if query.resolved && !c.rewritten => val aliased = aliasColumns(query, columnAliases, columnComments) - c.copy(query = aliased, queryColumnNames = query.schema.fieldNames.toIndexedSeq, rewritten = true) + c.copy( + query = aliased, + queryColumnNames = query.schema.fieldNames.toIndexedSeq, + rewritten = true) } private def aliasColumns( - plan: LogicalPlan, - columnAliases: Seq[String], - columnComments: Seq[Option[String]]): LogicalPlan = { + plan: LogicalPlan, + columnAliases: Seq[String], + columnComments: Seq[Option[String]]): LogicalPlan = { if (columnAliases.isEmpty || columnAliases.length != plan.output.length) { plan } else { val projectList = plan.output.zipWithIndex.map { case (attr, pos) => if (columnComments.apply(pos).isDefined) { - val meta = new MetadataBuilder().putString("comment", columnComments.apply(pos).get).build() + val meta = + new MetadataBuilder().putString("comment", columnComments.apply(pos).get).build() Alias(attr, columnAliases.apply(pos))(explicitMetadata = Some(meta)) } else { Alias(attr, columnAliases.apply(pos))() @@ -86,7 +101,6 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look } } - private def createViewRelation(nameParts: Seq[String], view: View): LogicalPlan = { val parsed = parseViewText(nameParts.quoted, view.query) @@ -99,17 +113,15 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look // This is more strict because it doesn't allow resolution by field name. val aliases = view.schema.fields.zipWithIndex.map { case (expected, pos) => val attr = GetColumnByOrdinal(pos, expected.dataType) - Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = Some(expected.metadata)) + Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = + Some(expected.metadata)) }.toIndexedSeq SubqueryAlias(nameParts, Project(aliases, rewritten)) } private def parseViewText(name: String, viewText: String): LogicalPlan = { - val origin = Origin( - objectType = Some("VIEW"), - objectName = Some(name) - ) + val origin = Origin(objectType = Some("VIEW"), objectName = Some(name)) try { CurrentOrigin.withOrigin(origin) { @@ -122,8 +134,8 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look } private def rewriteIdentifiers( - plan: LogicalPlan, - catalogAndNamespace: Seq[String]): LogicalPlan = { + plan: LogicalPlan, + catalogAndNamespace: Seq[String]): LogicalPlan = { // Substitute CTEs and Unresolved Ordinals within the view, then rewrite unresolved functions and relations qualifyTableIdentifiers( qualifyFunctionIdentifiers( @@ -133,15 +145,15 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look } private def qualifyFunctionIdentifiers( - plan: LogicalPlan, - catalogAndNamespace: Seq[String]): LogicalPlan = plan transformExpressions { - case u@UnresolvedFunction(Seq(name), _, _, _, _, _, _) => + plan: LogicalPlan, + catalogAndNamespace: Seq[String]): LogicalPlan = plan transformExpressions { + case u @ UnresolvedFunction(Seq(name), _, _, _, _, _, _) => if (!isBuiltinFunction(name)) { u.copy(nameParts = catalogAndNamespace :+ name) } else { u } - case u@UnresolvedFunction(parts, _, _, _, _, _, _) if !isCatalog(parts.head) => + case u @ UnresolvedFunction(parts, _, _, _, _, _, _) if !isCatalog(parts.head) => u.copy(nameParts = catalogAndNamespace.head +: parts) } @@ -149,17 +161,16 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look * Qualify table identifiers with default catalog and namespace if necessary. */ private def qualifyTableIdentifiers( - child: LogicalPlan, - catalogAndNamespace: Seq[String]): LogicalPlan = + child: LogicalPlan, + catalogAndNamespace: Seq[String]): LogicalPlan = child transform { - case u@UnresolvedRelation(Seq(table), _, _) => + case u @ UnresolvedRelation(Seq(table), _, _) => u.copy(multipartIdentifier = catalogAndNamespace :+ table) - case u@UnresolvedRelation(parts, _, _) if !isCatalog(parts.head) => + case u @ UnresolvedRelation(parts, _, _) if !isCatalog(parts.head) => u.copy(multipartIdentifier = catalogAndNamespace.head +: parts) case other => - other.transformExpressions { - case subquery: SubqueryExpression => - subquery.withNewPlan(qualifyTableIdentifiers(subquery.plan, catalogAndNamespace)) + other.transformExpressions { case subquery: SubqueryExpression => + subquery.withNewPlan(qualifyTableIdentifiers(subquery.plan, catalogAndNamespace)) } } diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index 1eeb50df8d11..ac0f75c422d1 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.SparkSession @@ -51,11 +50,21 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case DropView(ResolvedIdent(resolved), ifExists) => DropIcebergView(resolved, ifExists) - case CreateView(ResolvedIdent(resolved), userSpecifiedColumns, comment, _, properties, - Some(queryText), query, allowExisting, replace, _) => + case CreateView( + ResolvedIdent(resolved), + userSpecifiedColumns, + comment, + _, + properties, + Some(queryText), + query, + allowExisting, + replace, + _) => val q = CTESubstitution.apply(query) verifyTemporaryObjectsDontExist(resolved, q) - CreateIcebergView(child = resolved, + CreateIcebergView( + child = resolved, queryText = queryText, query = q, columnAliases = userSpecifiedColumns.map(_._1), @@ -67,20 +76,25 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case view @ ShowViews(CurrentNamespace, pattern, output) => if (ViewUtil.isViewCatalog(catalogManager.currentCatalog)) { - ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace.toIndexedSeq), - pattern, output) + ShowIcebergViews( + ResolvedNamespace( + catalogManager.currentCatalog, + catalogManager.currentNamespace.toIndexedSeq), + pattern, + output) } else { view } case ShowViews(UnresolvedNamespace(CatalogAndNamespace(catalog, ns), _), pattern, output) - if ViewUtil.isViewCatalog(catalog) => + if ViewUtil.isViewCatalog(catalog) => ShowIcebergViews(ResolvedNamespace(catalog, ns), pattern, output) // needs to be done here instead of in ResolveViews, so that a V2 view can be resolved before the Analyzer // tries to resolve it, which would result in an error, saying that V2 views aren't supported - case u@UnresolvedView(ResolvedView(resolved), _, _, _) => - ViewUtil.loadView(resolved.catalog, resolved.identifier) + case u @ UnresolvedView(ResolvedView(resolved), _, _, _) => + ViewUtil + .loadView(resolved.catalog, resolved.identifier) .map(_ => ResolvedV2View(resolved.catalog.asViewCatalog, resolved.identifier)) .getOrElse(u) } @@ -101,7 +115,8 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case UnresolvedIdentifier(nameParts, true) if isTempView(nameParts) => None - case UnresolvedIdentifier(CatalogAndIdentifier(catalog, ident), _) if ViewUtil.isViewCatalog(catalog) => + case UnresolvedIdentifier(CatalogAndIdentifier(catalog, ident), _) + if ViewUtil.isViewCatalog(catalog) => Some(ResolvedIdentifier(catalog, ident)) case _ => @@ -113,11 +128,14 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi * Permanent views are not allowed to reference temp objects */ private def verifyTemporaryObjectsDontExist( - identifier: ResolvedIdentifier, - child: LogicalPlan): Unit = { + identifier: ResolvedIdentifier, + child: LogicalPlan): Unit = { val tempViews = collectTemporaryViews(child) if (tempViews.nonEmpty) { - throw invalidRefToTempObject(identifier, tempViews.map(v => v.quoted).mkString("[", ", ", "]"), "view") + throw invalidRefToTempObject( + identifier, + tempViews.map(v => v.quoted).mkString("[", ", ", "]"), + "view") } val tempFunctions = collectTemporaryFunctions(child) @@ -126,9 +144,17 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi } } - private def invalidRefToTempObject(ident: ResolvedIdentifier, tempObjectNames: String, tempObjectType: String) = { - new IcebergAnalysisException(String.format("Cannot create view %s.%s that references temporary %s: %s", - ident.catalog.name(), ident.identifier, tempObjectType, tempObjectNames)) + private def invalidRefToTempObject( + ident: ResolvedIdentifier, + tempObjectNames: String, + tempObjectType: String) = { + new IcebergAnalysisException( + String.format( + "Cannot create view %s.%s that references temporary %s: %s", + ident.catalog.name(), + ident.identifier, + tempObjectType, + tempObjectNames)) } /** @@ -140,10 +166,11 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case unresolved: UnresolvedRelation if isTempView(unresolved.multipartIdentifier) => Seq(unresolved.multipartIdentifier) case view: View if view.isTempView => Seq(view.desc.identifier.nameParts) - case plan => plan.expressions.flatMap(_.flatMap { - case e: SubqueryExpression => collectTempViews(e.plan) - case _ => Seq.empty - }) + case plan => + plan.expressions.flatMap(_.flatMap { + case e: SubqueryExpression => collectTempViews(e.plan) + case _ => Seq.empty + }) }.distinct } @@ -156,7 +183,9 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi None case CatalogAndIdentifier(catalog, ident) if ViewUtil.isViewCatalog(catalog) => - ViewUtil.loadView(catalog, ident).flatMap(_ => Some(ResolvedV2View(catalog.asViewCatalog, ident))) + ViewUtil + .loadView(catalog, ident) + .flatMap(_ => Some(ResolvedV2View(catalog.asViewCatalog, ident))) case _ => None diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala index d46f10b7f5a2..a7188837c51e 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.connector.catalog.CatalogPlugin diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala index 5f78a0b27982..bdec4aae884d 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala @@ -48,7 +48,7 @@ import org.apache.spark.sql.types.StructType object ReplaceStaticInvoke extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = - plan.transformWithPruning (_.containsAnyPattern(COMMAND, FILTER, JOIN)) { + plan.transformWithPruning(_.containsAnyPattern(COMMAND, FILTER, JOIN)) { case replace @ ReplaceData(_, cond, _, _, _, _, _) => replaceStaticInvoke(replace, cond, newCond => replace.copy(condition = newCond)) @@ -57,7 +57,7 @@ object ReplaceStaticInvoke extends Rule[LogicalPlan] { case filter @ Filter(cond, _) => replaceStaticInvoke(filter, cond, newCond => filter.copy(condition = newCond)) - } + } private def replaceStaticInvoke[T <: LogicalPlan]( node: T, @@ -90,16 +90,17 @@ object ReplaceStaticInvoke extends Rule[LogicalPlan] { return invoke } - val inputType = StructType(invoke.arguments.zipWithIndex.map { - case (exp, pos) => StructField(s"_$pos", exp.dataType, exp.nullable) + val inputType = StructType(invoke.arguments.zipWithIndex.map { case (exp, pos) => + StructField(s"_$pos", exp.dataType, exp.nullable) }) - val bound = try { - unbound.bind(inputType) - } catch { - case _: Exception => - return invoke - } + val bound = + try { + unbound.bind(inputType) + } catch { + case _: Exception => + return invoke + } if (bound.inputTypes().length != invoke.arguments.length) { return invoke diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala index b8274983e59b..25e056ee2d96 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.parser.extensions import java.util.Locale @@ -45,7 +44,9 @@ import org.apache.spark.sql.types.DataType import org.apache.spark.sql.types.StructType import scala.jdk.CollectionConverters._ -class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserInterface with ExtendedParser { +class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) + extends ParserInterface + with ExtendedParser { import IcebergSparkSqlExtensionsParser._ @@ -108,7 +109,8 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI }.asJava } - override def parseRoutineParam(sqlText: String): StructType = throw new UnsupportedOperationException() + override def parseRoutineParam(sqlText: String): StructType = + throw new UnsupportedOperationException() /** * Parse a string to a LogicalPlan. @@ -116,14 +118,17 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI override def parsePlan(sqlText: String): LogicalPlan = { val sqlTextAfterSubstitution = substitutor.substitute(sqlText) if (isIcebergCommand(sqlTextAfterSubstitution)) { - parse(sqlTextAfterSubstitution) { parser => astBuilder.visit(parser.singleStatement()) }.asInstanceOf[LogicalPlan] + parse(sqlTextAfterSubstitution) { parser => astBuilder.visit(parser.singleStatement()) } + .asInstanceOf[LogicalPlan] } else { RewriteViewCommands(SparkSession.active).apply(delegate.parsePlan(sqlText)) } } private def isIcebergCommand(sqlText: String): Boolean = { - val normalized = sqlText.toLowerCase(Locale.ROOT).trim() + val normalized = sqlText + .toLowerCase(Locale.ROOT) + .trim() // Strip simple SQL comments that terminate a line, e.g. comments starting with `--` . .replaceAll("--.*?\\n", " ") // Strip newlines. @@ -135,30 +140,30 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI .replaceAll("`", "") .trim() - normalized.startsWith("alter table") && ( - normalized.contains("add partition field") || - normalized.contains("drop partition field") || - normalized.contains("replace partition field") || - normalized.contains("write ordered by") || - normalized.contains("write locally ordered by") || - normalized.contains("write distributed by") || - normalized.contains("write unordered") || - normalized.contains("set identifier fields") || - normalized.contains("drop identifier fields") || - isSnapshotRefDdl(normalized)) + normalized.startsWith("alter table") && (normalized.contains("add partition field") || + normalized.contains("drop partition field") || + normalized.contains("replace partition field") || + normalized.contains("write ordered by") || + normalized.contains("write locally ordered by") || + normalized.contains("write distributed by") || + normalized.contains("write unordered") || + normalized.contains("set identifier fields") || + normalized.contains("drop identifier fields") || + isSnapshotRefDdl(normalized)) } private def isSnapshotRefDdl(normalized: String): Boolean = { normalized.contains("create branch") || - normalized.contains("replace branch") || - normalized.contains("create tag") || - normalized.contains("replace tag") || - normalized.contains("drop branch") || - normalized.contains("drop tag") + normalized.contains("replace branch") || + normalized.contains("create tag") || + normalized.contains("replace tag") || + normalized.contains("drop branch") || + normalized.contains("drop tag") } protected def parse[T](command: String)(toResult: IcebergSqlExtensionsParser => T): T = { - val lexer = new IcebergSqlExtensionsLexer(new UpperCaseCharStream(CharStreams.fromString(command))) + val lexer = new IcebergSqlExtensionsLexer( + new UpperCaseCharStream(CharStreams.fromString(command))) lexer.removeErrorListeners() lexer.addErrorListener(IcebergParseErrorListener) @@ -176,8 +181,7 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI parser.setErrorHandler(new BailErrorStrategy) parser.getInterpreter.setPredictionMode(PredictionMode.SLL) toResult(parser) - } - catch { + } catch { case _: ParseCancellationException => // if we fail, parse with LL mode with DefaultErrorStrategy tokenStream.seek(0) // rewind input stream @@ -188,8 +192,7 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI parser.getInterpreter.setPredictionMode(PredictionMode.LL) toResult(parser) } - } - catch { + } catch { case e: IcebergParseException if e.command.isDefined => throw e case e: IcebergParseException => @@ -207,7 +210,8 @@ class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) extends ParserI object IcebergSparkSqlExtensionsParser { private val substitutorCtor: DynConstructors.Ctor[VariableSubstitution] = - DynConstructors.builder() + DynConstructors + .builder() .impl(classOf[VariableSubstitution]) .impl(classOf[VariableSubstitution], classOf[SQLConf]) .build() @@ -253,9 +257,7 @@ case object IcebergSqlExtensionsPostProcessor extends IcebergSqlExtensionsBaseLi replaceTokenByIdentifier(ctx, 0)(identity) } - private def replaceTokenByIdentifier( - ctx: ParserRuleContext, - stripMargins: Int)( + private def replaceTokenByIdentifier(ctx: ParserRuleContext, stripMargins: Int)( f: CommonToken => CommonToken = identity): Unit = { val parent = ctx.getParent parent.removeLastChild() @@ -302,10 +304,12 @@ class IcebergParseException( val command: Option[String], message: String, val start: Origin, - val stop: Origin) extends AnalysisException(message, start.line, start.startPosition) { + val stop: Origin) + extends AnalysisException(message, start.line, start.startPosition) { def this(message: String, ctx: ParserRuleContext) = { - this(Option(IcebergParserUtils.command(ctx)), + this( + Option(IcebergParserUtils.command(ctx)), message, IcebergParserUtils.position(ctx.getStart), IcebergParserUtils.position(ctx.getStop)) @@ -315,8 +319,7 @@ class IcebergParseException( val builder = new StringBuilder builder ++= "\n" ++= message start match { - case Origin(Some(l), Some(p), Some(_), Some(_), Some(_), Some(_), - Some(_), _, _) => + case Origin(Some(l), Some(p), Some(_), Some(_), Some(_), Some(_), Some(_), _, _) => builder ++= s"(line $l, pos $p)\n" command.foreach { cmd => val (above, below) = cmd.split("\n").splitAt(l) @@ -336,4 +339,4 @@ class IcebergParseException( def withCommand(cmd: String): IcebergParseException = { new IcebergParseException(Option(cmd), message, start, stop) } -} \ No newline at end of file +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala index 326bcf3895e7..724101cfe11d 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.parser.extensions import java.util.Locale @@ -59,60 +58,69 @@ import org.apache.spark.sql.connector.expressions.LiteralValue import org.apache.spark.sql.connector.expressions.Transform import scala.jdk.CollectionConverters._ -class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergSqlExtensionsBaseVisitor[AnyRef] { +class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) + extends IcebergSqlExtensionsBaseVisitor[AnyRef] { - private def toBuffer[T](list: java.util.List[T]): scala.collection.mutable.Buffer[T] = list.asScala + private def toBuffer[T](list: java.util.List[T]): scala.collection.mutable.Buffer[T] = + list.asScala private def toSeq[T](list: java.util.List[T]): Seq[T] = toBuffer(list).toSeq /** * Create an ADD PARTITION FIELD logical command. */ - override def visitAddPartitionField(ctx: AddPartitionFieldContext): AddPartitionField = withOrigin(ctx) { - AddPartitionField( - typedVisit[Seq[String]](ctx.multipartIdentifier), - typedVisit[Transform](ctx.transform), - Option(ctx.name).map(_.getText)) - } + override def visitAddPartitionField(ctx: AddPartitionFieldContext): AddPartitionField = + withOrigin(ctx) { + AddPartitionField( + typedVisit[Seq[String]](ctx.multipartIdentifier), + typedVisit[Transform](ctx.transform), + Option(ctx.name).map(_.getText)) + } /** * Create a DROP PARTITION FIELD logical command. */ - override def visitDropPartitionField(ctx: DropPartitionFieldContext): DropPartitionField = withOrigin(ctx) { - DropPartitionField( - typedVisit[Seq[String]](ctx.multipartIdentifier), - typedVisit[Transform](ctx.transform)) - } + override def visitDropPartitionField(ctx: DropPartitionFieldContext): DropPartitionField = + withOrigin(ctx) { + DropPartitionField( + typedVisit[Seq[String]](ctx.multipartIdentifier), + typedVisit[Transform](ctx.transform)) + } /** * Create a CREATE OR REPLACE BRANCH logical command. */ - override def visitCreateOrReplaceBranch(ctx: CreateOrReplaceBranchContext): CreateOrReplaceBranch = withOrigin(ctx) { + override def visitCreateOrReplaceBranch( + ctx: CreateOrReplaceBranchContext): CreateOrReplaceBranch = withOrigin(ctx) { val createOrReplaceBranchClause = ctx.createReplaceBranchClause() val branchName = createOrReplaceBranchClause.identifier() val branchOptionsContext = Option(createOrReplaceBranchClause.branchOptions()) - val snapshotId = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.snapshotId())) + val snapshotId = branchOptionsContext + .flatMap(branchOptions => Option(branchOptions.snapshotId())) .map(_.getText.toLong) - val snapshotRetention = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.snapshotRetention())) - val minSnapshotsToKeep = snapshotRetention.flatMap(retention => Option(retention.minSnapshotsToKeep())) + val snapshotRetention = + branchOptionsContext.flatMap(branchOptions => Option(branchOptions.snapshotRetention())) + val minSnapshotsToKeep = snapshotRetention + .flatMap(retention => Option(retention.minSnapshotsToKeep())) .map(minSnapshots => minSnapshots.number().getText.toLong) val maxSnapshotAgeMs = snapshotRetention .flatMap(retention => Option(retention.maxSnapshotAge())) - .map(retention => TimeUnit.valueOf(retention.timeUnit().getText.toUpperCase(Locale.ENGLISH)) - .toMillis(retention.number().getText.toLong)) - val branchRetention = branchOptionsContext.flatMap(branchOptions => Option(branchOptions.refRetain())) + .map(retention => + TimeUnit + .valueOf(retention.timeUnit().getText.toUpperCase(Locale.ENGLISH)) + .toMillis(retention.number().getText.toLong)) + val branchRetention = + branchOptionsContext.flatMap(branchOptions => Option(branchOptions.refRetain())) val branchRefAgeMs = branchRetention.map(retain => - TimeUnit.valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)).toMillis(retain.number().getText.toLong)) + TimeUnit + .valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)) + .toMillis(retain.number().getText.toLong)) val create = createOrReplaceBranchClause.CREATE() != null val replace = ctx.createReplaceBranchClause().REPLACE() != null val ifNotExists = createOrReplaceBranchClause.EXISTS() != null - val branchOptions = BranchOptions( - snapshotId, - minSnapshotsToKeep, - maxSnapshotAgeMs, - branchRefAgeMs - ) + val branchOptions = + BranchOptions(snapshotId, minSnapshotsToKeep, maxSnapshotAgeMs, branchRefAgeMs) CreateOrReplaceBranch( typedVisit[Seq[String]](ctx.multipartIdentifier), @@ -126,52 +134,61 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS /** * Create an CREATE OR REPLACE TAG logical command. */ - override def visitCreateOrReplaceTag(ctx: CreateOrReplaceTagContext): CreateOrReplaceTag = withOrigin(ctx) { - val createTagClause = ctx.createReplaceTagClause() - - val tagName = createTagClause.identifier().getText - - val tagOptionsContext = Option(createTagClause.tagOptions()) - val snapshotId = tagOptionsContext.flatMap(tagOptions => Option(tagOptions.snapshotId())) - .map(_.getText.toLong) - val tagRetain = tagOptionsContext.flatMap(tagOptions => Option(tagOptions.refRetain())) - val tagRefAgeMs = tagRetain.map(retain => - TimeUnit.valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)).toMillis(retain.number().getText.toLong)) - val tagOptions = TagOptions( - snapshotId, - tagRefAgeMs - ) - - val create = createTagClause.CREATE() != null - val replace = createTagClause.REPLACE() != null - val ifNotExists = createTagClause.EXISTS() != null - - CreateOrReplaceTag(typedVisit[Seq[String]](ctx.multipartIdentifier), - tagName, - tagOptions, - create, - replace, - ifNotExists) - } + override def visitCreateOrReplaceTag(ctx: CreateOrReplaceTagContext): CreateOrReplaceTag = + withOrigin(ctx) { + val createTagClause = ctx.createReplaceTagClause() + + val tagName = createTagClause.identifier().getText + + val tagOptionsContext = Option(createTagClause.tagOptions()) + val snapshotId = tagOptionsContext + .flatMap(tagOptions => Option(tagOptions.snapshotId())) + .map(_.getText.toLong) + val tagRetain = tagOptionsContext.flatMap(tagOptions => Option(tagOptions.refRetain())) + val tagRefAgeMs = tagRetain.map(retain => + TimeUnit + .valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)) + .toMillis(retain.number().getText.toLong)) + val tagOptions = TagOptions(snapshotId, tagRefAgeMs) + + val create = createTagClause.CREATE() != null + val replace = createTagClause.REPLACE() != null + val ifNotExists = createTagClause.EXISTS() != null + + CreateOrReplaceTag( + typedVisit[Seq[String]](ctx.multipartIdentifier), + tagName, + tagOptions, + create, + replace, + ifNotExists) + } /** * Create an DROP BRANCH logical command. */ override def visitDropBranch(ctx: DropBranchContext): DropBranch = withOrigin(ctx) { - DropBranch(typedVisit[Seq[String]](ctx.multipartIdentifier), ctx.identifier().getText, ctx.EXISTS() != null) + DropBranch( + typedVisit[Seq[String]](ctx.multipartIdentifier), + ctx.identifier().getText, + ctx.EXISTS() != null) } /** * Create an DROP TAG logical command. */ override def visitDropTag(ctx: DropTagContext): DropTag = withOrigin(ctx) { - DropTag(typedVisit[Seq[String]](ctx.multipartIdentifier), ctx.identifier().getText, ctx.EXISTS() != null) + DropTag( + typedVisit[Seq[String]](ctx.multipartIdentifier), + ctx.identifier().getText, + ctx.EXISTS() != null) } /** * Create an REPLACE PARTITION FIELD logical command. */ - override def visitReplacePartitionField(ctx: ReplacePartitionFieldContext): ReplacePartitionField = withOrigin(ctx) { + override def visitReplacePartitionField( + ctx: ReplacePartitionFieldContext): ReplacePartitionField = withOrigin(ctx) { ReplacePartitionField( typedVisit[Seq[String]](ctx.multipartIdentifier), typedVisit[Transform](ctx.transform(0)), @@ -182,20 +199,22 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS /** * Create an SET IDENTIFIER FIELDS logical command. */ - override def visitSetIdentifierFields(ctx: SetIdentifierFieldsContext): SetIdentifierFields = withOrigin(ctx) { - SetIdentifierFields( - typedVisit[Seq[String]](ctx.multipartIdentifier), - toSeq(ctx.fieldList.fields).map(_.getText)) - } + override def visitSetIdentifierFields(ctx: SetIdentifierFieldsContext): SetIdentifierFields = + withOrigin(ctx) { + SetIdentifierFields( + typedVisit[Seq[String]](ctx.multipartIdentifier), + toSeq(ctx.fieldList.fields).map(_.getText)) + } /** * Create an DROP IDENTIFIER FIELDS logical command. */ - override def visitDropIdentifierFields(ctx: DropIdentifierFieldsContext): DropIdentifierFields = withOrigin(ctx) { - DropIdentifierFields( - typedVisit[Seq[String]](ctx.multipartIdentifier), - toSeq(ctx.fieldList.fields).map(_.getText)) - } + override def visitDropIdentifierFields(ctx: DropIdentifierFieldsContext): DropIdentifierFields = + withOrigin(ctx) { + DropIdentifierFields( + typedVisit[Seq[String]](ctx.multipartIdentifier), + toSeq(ctx.fieldList.fields).map(_.getText)) + } /** * Create a [[SetWriteDistributionAndOrdering]] for changing the write distribution and ordering. @@ -253,12 +272,15 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS */ override def visitOrderField(ctx: OrderFieldContext): (Term, SortDirection, NullOrder) = { val term = Spark3Util.toIcebergTerm(typedVisit[Transform](ctx.transform)) - val direction = Option(ctx.ASC).map(_ => SortDirection.ASC) - .orElse(Option(ctx.DESC).map(_ => SortDirection.DESC)) - .getOrElse(SortDirection.ASC) - val nullOrder = Option(ctx.FIRST).map(_ => NullOrder.NULLS_FIRST) - .orElse(Option(ctx.LAST).map(_ => NullOrder.NULLS_LAST)) - .getOrElse(if (direction == SortDirection.ASC) NullOrder.NULLS_FIRST else NullOrder.NULLS_LAST) + val direction = Option(ctx.ASC) + .map(_ => SortDirection.ASC) + .orElse(Option(ctx.DESC).map(_ => SortDirection.DESC)) + .getOrElse(SortDirection.ASC) + val nullOrder = Option(ctx.FIRST) + .map(_ => NullOrder.NULLS_FIRST) + .orElse(Option(ctx.LAST).map(_ => NullOrder.NULLS_LAST)) + .getOrElse( + if (direction == SortDirection.ASC) NullOrder.NULLS_FIRST else NullOrder.NULLS_LAST) (term, direction, nullOrder) } @@ -280,27 +302,31 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS /** * Create a transform argument from a column reference or a constant. */ - override def visitTransformArgument(ctx: TransformArgumentContext): expressions.Expression = withOrigin(ctx) { - val reference = Option(ctx.multipartIdentifier()) + override def visitTransformArgument(ctx: TransformArgumentContext): expressions.Expression = + withOrigin(ctx) { + val reference = Option(ctx.multipartIdentifier()) .map(typedVisit[Seq[String]]) .map(FieldReference(_)) - val literal = Option(ctx.constant) + val literal = Option(ctx.constant) .map(visitConstant) .map(lit => LiteralValue(lit.value, lit.dataType)) - reference.orElse(literal) + reference + .orElse(literal) .getOrElse(throw new IcebergParseException(s"Invalid transform argument", ctx)) - } + } /** * Return a multi-part identifier as Seq[String]. */ - override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = withOrigin(ctx) { - toSeq(ctx.parts).map(_.getText) - } + override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = + withOrigin(ctx) { + toSeq(ctx.parts).map(_.getText) + } - override def visitSingleOrder(ctx: SingleOrderContext): Seq[(Term, SortDirection, NullOrder)] = withOrigin(ctx) { - toSeq(ctx.order.fields).map(typedVisit[(Term, SortDirection, NullOrder)]) - } + override def visitSingleOrder(ctx: SingleOrderContext): Seq[(Term, SortDirection, NullOrder)] = + withOrigin(ctx) { + toSeq(ctx.order.fields).map(typedVisit[(Term, SortDirection, NullOrder)]) + } override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) { visit(ctx.statement).asInstanceOf[LogicalPlan] @@ -320,10 +346,12 @@ class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) extends IcebergS } private def reconstructSqlString(ctx: ParserRuleContext): String = { - toBuffer(ctx.children).map { - case c: ParserRuleContext => reconstructSqlString(c) - case t: TerminalNode => t.getText - }.mkString(" ") + toBuffer(ctx.children) + .map { + case c: ParserRuleContext => reconstructSqlString(c) + case t: TerminalNode => t.getText + } + .mkString(" ") } private def typedVisit[T](ctx: ParseTree): T = { diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala index e8b1b2941161..0a830dbd4f6a 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala @@ -16,13 +16,13 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.expressions.Transform -case class AddPartitionField(table: Seq[String], transform: Transform, name: Option[String]) extends LeafCommand { +case class AddPartitionField(table: Seq[String], transform: Transform, name: Option[String]) + extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala index 4d7e0a086bda..15b908300213 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala @@ -16,8 +16,10 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical -case class BranchOptions (snapshotId: Option[Long], numSnapshots: Option[Long], - snapshotRetain: Option[Long], snapshotRefRetain: Option[Long]) +case class BranchOptions( + snapshotId: Option[Long], + numSnapshots: Option[Long], + snapshotRetain: Option[Long], + snapshotRefRetain: Option[Long]) diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala index b7981a3c7a0d..6900f6e8cc50 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute @@ -27,7 +26,8 @@ case class CreateOrReplaceBranch( branchOptions: BranchOptions, create: Boolean, replace: Boolean, - ifNotExists: Boolean) extends LeafCommand { + ifNotExists: Boolean) + extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala index 6e7db84a90fb..957c68e7a540 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute @@ -27,7 +26,8 @@ case class CreateOrReplaceTag( tagOptions: TagOptions, create: Boolean, replace: Boolean, - ifNotExists: Boolean) extends LeafCommand { + ifNotExists: Boolean) + extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala index bee0b0fae688..ed4f1f512b85 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala index 29dd686a0fba..1a91806280b3 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala @@ -16,14 +16,11 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute -case class DropIdentifierFields( - table: Seq[String], - fields: Seq[String]) extends LeafCommand { +case class DropIdentifierFields(table: Seq[String], fields: Seq[String]) extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala index fb1451324182..ec952c8c7118 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala index 7e4b38e74d2f..da69ca0383a1 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala index 8c660c6f37b1..c2525369e7c7 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute @@ -26,13 +25,14 @@ case class ReplacePartitionField( table: Seq[String], transformFrom: Transform, transformTo: Transform, - name: Option[String]) extends LeafCommand { + name: Option[String]) + extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil override def simpleString(maxFields: Int): String = { s"ReplacePartitionField ${table.quoted} ${transformFrom.describe} " + - s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" + s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" } } diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala index 1e5622e9c374..8cd2c0ddad05 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala @@ -16,14 +16,11 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute -case class SetIdentifierFields( - table: Seq[String], - fields: Seq[String]) extends LeafCommand { +case class SetIdentifierFields(table: Seq[String], fields: Seq[String]) extends LeafCommand { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala index 85e3b95f4aba..6afe1478d747 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical case class TagOptions(snapshotId: Option[Long], snapshotRefRetain: Option[Long]) diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala index 9366d5efe163..c5dadae70110 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala @@ -16,29 +16,30 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical.views import org.apache.spark.sql.catalyst.plans.logical.BinaryCommand import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan case class CreateIcebergView( - child: LogicalPlan, - queryText: String, - query: LogicalPlan, - columnAliases: Seq[String], - columnComments: Seq[Option[String]], - queryColumnNames: Seq[String] = Seq.empty, - comment: Option[String], - properties: Map[String, String], - allowExisting: Boolean, - replace: Boolean, - rewritten: Boolean = false) extends BinaryCommand { + child: LogicalPlan, + queryText: String, + query: LogicalPlan, + columnAliases: Seq[String], + columnComments: Seq[Option[String]], + queryColumnNames: Seq[String] = Seq.empty, + comment: Option[String], + properties: Map[String, String], + allowExisting: Boolean, + replace: Boolean, + rewritten: Boolean = false) + extends BinaryCommand { override def left: LogicalPlan = child override def right: LogicalPlan = query override protected def withNewChildrenInternal( - newLeft: LogicalPlan, newRight: LogicalPlan): LogicalPlan = + newLeft: LogicalPlan, + newRight: LogicalPlan): LogicalPlan = copy(child = newLeft, query = newRight) } diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala index 275dba6fbf5e..092b6b33fb0c 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala @@ -16,15 +16,12 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical.views import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.UnaryCommand -case class DropIcebergView( - child: LogicalPlan, - ifExists: Boolean) extends UnaryCommand { +case class DropIcebergView(child: LogicalPlan, ifExists: Boolean) extends UnaryCommand { override protected def withNewChildInternal(newChild: LogicalPlan): DropIcebergView = copy(child = newChild) } diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala index b9c05ff0061d..4d384e857703 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical.views import org.apache.spark.sql.catalyst.analysis.LeafNodeWithoutStats @@ -24,8 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog -case class ResolvedV2View( - catalog: ViewCatalog, - identifier: Identifier) extends LeafNodeWithoutStats { +case class ResolvedV2View(catalog: ViewCatalog, identifier: Identifier) + extends LeafNodeWithoutStats { override def output: Seq[Attribute] = Nil } diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala index b09c27acdc16..cbfe23d94cbe 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala @@ -16,8 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - - package org.apache.spark.sql.catalyst.plans.logical.views import org.apache.spark.sql.catalyst.expressions.Attribute @@ -26,9 +24,10 @@ import org.apache.spark.sql.catalyst.plans.logical.ShowViews import org.apache.spark.sql.catalyst.plans.logical.UnaryCommand case class ShowIcebergViews( - namespace: LogicalPlan, - pattern: Option[String], - override val output: Seq[Attribute] = ShowViews.getOutputAttrs) extends UnaryCommand { + namespace: LogicalPlan, + pattern: Option[String], + override val output: Seq[Attribute] = ShowViews.getOutputAttrs) + extends UnaryCommand { override def child: LogicalPlan = namespace override protected def withNewChildInternal(newChild: LogicalPlan): ShowIcebergViews = diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala index 55f327f7e45e..e28dcfb194b6 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.Spark3Util @@ -31,7 +30,8 @@ case class AddPartitionFieldExec( catalog: TableCatalog, ident: Identifier, transform: Transform, - name: Option[String]) extends LeafV2CommandExec { + name: Option[String]) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -39,12 +39,14 @@ case class AddPartitionFieldExec( override protected def run(): Seq[InternalRow] = { catalog.loadTable(ident) match { case iceberg: SparkTable => - iceberg.table.updateSpec() - .addField(name.orNull, Spark3Util.toIcebergTerm(transform)) - .commit() + iceberg.table + .updateSpec() + .addField(name.orNull, Spark3Util.toIcebergTerm(transform)) + .commit() case table => - throw new UnsupportedOperationException(s"Cannot add partition field to non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot add partition field to non-Iceberg table: $table") } Nil diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala index b103d1ee2c58..d6630e51ff5a 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -25,17 +24,17 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.connector.catalog.ViewChange - case class AlterV2ViewSetPropertiesExec( - catalog: ViewCatalog, - ident: Identifier, - properties: Map[String, String]) extends LeafV2CommandExec { + catalog: ViewCatalog, + ident: Identifier, + properties: Map[String, String]) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil override protected def run(): Seq[InternalRow] = { - val changes = properties.map { - case (property, value) => ViewChange.setProperty(property, value) + val changes = properties.map { case (property, value) => + ViewChange.setProperty(property, value) }.toSeq catalog.alterView(ident, changes: _*) diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala index d612173ca28d..aa57842f58b5 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -26,12 +25,12 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.connector.catalog.ViewChange - case class AlterV2ViewUnsetPropertiesExec( - catalog: ViewCatalog, - ident: Identifier, - propertyKeys: Seq[String], - ifExists: Boolean) extends LeafV2CommandExec { + catalog: ViewCatalog, + ident: Identifier, + propertyKeys: Seq[String], + ifExists: Boolean) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala index 2be406e7f344..baf985f53a22 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.relocated.com.google.common.base.Preconditions @@ -28,13 +27,14 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog case class CreateOrReplaceBranchExec( - catalog: TableCatalog, - ident: Identifier, - branch: String, - branchOptions: BranchOptions, - create: Boolean, - replace: Boolean, - ifNotExists: Boolean) extends LeafV2CommandExec { + catalog: TableCatalog, + ident: Identifier, + branch: String, + branchOptions: BranchOptions, + create: Boolean, + replace: Boolean, + ifNotExists: Boolean) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -62,8 +62,10 @@ case class CreateOrReplaceBranchExec( if (create && replace && !refExists) { safeCreateBranch() } else if (replace) { - Preconditions.checkArgument(snapshotId != null, - "Cannot complete replace branch operation on %s, main has no snapshot", ident) + Preconditions.checkArgument( + snapshotId != null, + "Cannot complete replace branch operation on %s, main has no snapshot", + ident) manageSnapshots.replaceBranch(branch, snapshotId) } else { if (refExists && ifNotExists) { @@ -88,7 +90,8 @@ case class CreateOrReplaceBranchExec( manageSnapshots.commit() case table => - throw new UnsupportedOperationException(s"Cannot create or replace branch on non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot create or replace branch on non-Iceberg table: $table") } Nil diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala index 372cd7548632..e486892614cb 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.relocated.com.google.common.base.Preconditions @@ -33,7 +32,8 @@ case class CreateOrReplaceTagExec( tagOptions: TagOptions, create: Boolean, replace: Boolean, - ifNotExists: Boolean) extends LeafV2CommandExec { + ifNotExists: Boolean) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -47,8 +47,10 @@ case class CreateOrReplaceTagExec( .map(java.lang.Long.valueOf) .orNull - Preconditions.checkArgument(snapshotId != null, - "Cannot complete create or replace tag operation on %s, main has no snapshot", ident) + Preconditions.checkArgument( + snapshotId != null, + "Cannot complete create or replace tag operation on %s, main has no snapshot", + ident) val manageSnapshot = iceberg.table.manageSnapshots() val refExists = null != iceberg.table().refs().get(tag) diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala index 97077a5ecf37..04f2b2af731d 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.SupportsReplaceView @@ -30,31 +29,33 @@ import org.apache.spark.sql.connector.catalog.ViewInfo import org.apache.spark.sql.types.StructType import scala.jdk.CollectionConverters._ - case class CreateV2ViewExec( - catalog: ViewCatalog, - ident: Identifier, - queryText: String, - viewSchema: StructType, - columnAliases: Seq[String], - columnComments: Seq[Option[String]], - queryColumnNames: Seq[String], - comment: Option[String], - properties: Map[String, String], - allowExisting: Boolean, - replace: Boolean) extends LeafV2CommandExec { + catalog: ViewCatalog, + ident: Identifier, + queryText: String, + viewSchema: StructType, + columnAliases: Seq[String], + columnComments: Seq[Option[String]], + queryColumnNames: Seq[String], + comment: Option[String], + properties: Map[String, String], + allowExisting: Boolean, + replace: Boolean) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil override protected def run(): Seq[InternalRow] = { val currentCatalogName = session.sessionState.catalogManager.currentCatalog.name - val currentCatalog = if (!catalog.name().equals(currentCatalogName)) currentCatalogName else null + val currentCatalog = + if (!catalog.name().equals(currentCatalogName)) currentCatalogName else null val currentNamespace = session.sessionState.catalogManager.currentNamespace val engineVersion = "Spark " + org.apache.spark.SPARK_VERSION val newProperties = properties ++ comment.map(ViewCatalog.PROP_COMMENT -> _) ++ - Map(ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion, + Map( + ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion, ViewCatalog.PROP_ENGINE_VERSION -> engineVersion) if (replace) { @@ -88,10 +89,10 @@ case class CreateV2ViewExec( } private def replaceView( - supportsReplaceView: SupportsReplaceView, - currentCatalog: String, - currentNamespace: Array[String], - newProperties: Map[String, String]) = { + supportsReplaceView: SupportsReplaceView, + currentCatalog: String, + currentNamespace: Array[String], + newProperties: Map[String, String]) = { supportsReplaceView.replaceView( ident, queryText, @@ -105,9 +106,9 @@ case class CreateV2ViewExec( } private def createView( - currentCatalog: String, - currentNamespace: Array[String], - newProperties: Map[String, String]) = { + currentCatalog: String, + currentNamespace: Array[String], + newProperties: Map[String, String]) = { val viewInfo: ViewInfo = new ViewInfo( ident, queryText, diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala index 38385d2979b7..106734d2078f 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -27,10 +26,9 @@ import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode import scala.jdk.CollectionConverters._ -case class DescribeV2ViewExec( - output: Seq[Attribute], - view: View, - isExtended: Boolean) extends V2CommandExec with LeafExecNode { +case class DescribeV2ViewExec(output: Seq[Attribute], view: View, isExtended: Boolean) + extends V2CommandExec + with LeafExecNode { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -44,23 +42,22 @@ case class DescribeV2ViewExec( private def describeSchema: Seq[InternalRow] = view.schema().map { column => - toCatalystRow( - column.name, - column.dataType.simpleString, - column.getComment().getOrElse("")) + toCatalystRow(column.name, column.dataType.simpleString, column.getComment().getOrElse("")) } private def emptyRow: InternalRow = toCatalystRow("", "", "") private def describeExtended: Seq[InternalRow] = { val outputColumns = view.queryColumnNames.mkString("[", ", ", "]") - val properties: Map[String, String] = view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala + val properties: Map[String, String] = + view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala val viewCatalogAndNamespace: Seq[String] = view.name.split("\\.").take(2).toIndexedSeq - val viewProperties = properties.toSeq.sortBy(_._1).map { - case (key, value) => + val viewProperties = properties.toSeq + .sortBy(_._1) + .map { case (key, value) => s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" - }.mkString("[", ", ", "]") - + } + .mkString("[", ", ", "]") // omitting view text here because it is shown as // part of SHOW CREATE TABLE and can result in weird formatting in the DESCRIBE output @@ -69,7 +66,10 @@ case class DescribeV2ViewExec( toCatalystRow("View Catalog and Namespace", viewCatalogAndNamespace.quoted, "") :: toCatalystRow("View Query Output Columns", outputColumns, "") :: toCatalystRow("View Properties", viewProperties, "") :: - toCatalystRow("Created By", view.properties.getOrDefault(ViewCatalog.PROP_CREATE_ENGINE_VERSION, ""), "") :: + toCatalystRow( + "Created By", + view.properties.getOrDefault(ViewCatalog.PROP_CREATE_ENGINE_VERSION, ""), + "") :: Nil } diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala index ff8f1820099a..e7d9c7b70d82 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.source.SparkTable @@ -29,7 +28,8 @@ case class DropBranchExec( catalog: TableCatalog, ident: Identifier, branch: String, - ifExists: Boolean) extends LeafV2CommandExec { + ifExists: Boolean) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala index dee778b474f9..87b18594d573 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.relocated.com.google.common.base.Preconditions @@ -27,10 +26,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog -case class DropIdentifierFieldsExec( - catalog: TableCatalog, - ident: Identifier, - fields: Seq[String]) extends LeafV2CommandExec { +case class DropIdentifierFieldsExec(catalog: TableCatalog, ident: Identifier, fields: Seq[String]) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -42,18 +39,24 @@ case class DropIdentifierFieldsExec( val identifierFieldNames = Sets.newHashSet(schema.identifierFieldNames) for (name <- fields) { - Preconditions.checkArgument(schema.findField(name) != null, - "Cannot complete drop identifier fields operation: field %s not found", name) - Preconditions.checkArgument(identifierFieldNames.contains(name), - "Cannot complete drop identifier fields operation: %s is not an identifier field", name) + Preconditions.checkArgument( + schema.findField(name) != null, + "Cannot complete drop identifier fields operation: field %s not found", + name) + Preconditions.checkArgument( + identifierFieldNames.contains(name), + "Cannot complete drop identifier fields operation: %s is not an identifier field", + name) identifierFieldNames.remove(name) } - iceberg.table.updateSchema() + iceberg.table + .updateSchema() .setIdentifierFields(identifierFieldNames) .commit(); case table => - throw new UnsupportedOperationException(s"Cannot drop identifier fields in non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot drop identifier fields in non-Iceberg table: $table") } Nil diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala index 9a153f0c004e..db43263e0e66 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.Spark3Util @@ -29,10 +28,8 @@ import org.apache.spark.sql.connector.expressions.FieldReference import org.apache.spark.sql.connector.expressions.IdentityTransform import org.apache.spark.sql.connector.expressions.Transform -case class DropPartitionFieldExec( - catalog: TableCatalog, - ident: Identifier, - transform: Transform) extends LeafV2CommandExec { +case class DropPartitionFieldExec(catalog: TableCatalog, ident: Identifier, transform: Transform) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -42,20 +39,24 @@ case class DropPartitionFieldExec( case iceberg: SparkTable => val schema = iceberg.table.schema transform match { - case IdentityTransform(FieldReference(parts)) if parts.size == 1 && schema.findField(parts.head) == null => + case IdentityTransform(FieldReference(parts)) + if parts.size == 1 && schema.findField(parts.head) == null => // the name is not present in the Iceberg schema, so it must be a partition field name, not a column name - iceberg.table.updateSpec() - .removeField(parts.head) - .commit() + iceberg.table + .updateSpec() + .removeField(parts.head) + .commit() case _ => - iceberg.table.updateSpec() - .removeField(Spark3Util.toIcebergTerm(transform)) - .commit() + iceberg.table + .updateSpec() + .removeField(Spark3Util.toIcebergTerm(transform)) + .commit() } case table => - throw new UnsupportedOperationException(s"Cannot drop partition field in non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot drop partition field in non-Iceberg table: $table") } Nil diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala index 8df88765a986..79b4a1525591 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.source.SparkTable @@ -25,11 +24,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog -case class DropTagExec( - catalog: TableCatalog, - ident: Identifier, - tag: String, - ifExists: Boolean) extends LeafV2CommandExec { +case class DropTagExec(catalog: TableCatalog, ident: Identifier, tag: String, ifExists: Boolean) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala index c35af1486fc7..6dd1188b78e8 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -25,11 +24,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog - -case class DropV2ViewExec( - catalog: ViewCatalog, - ident: Identifier, - ifExists: Boolean) extends LeafV2CommandExec { +case class DropV2ViewExec(catalog: ViewCatalog, ident: Identifier, ifExists: Boolean) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala index 9958d50840a4..898e6f3d41dc 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.Spark3Util @@ -64,11 +63,28 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi AddPartitionFieldExec(catalog, ident, transform, name) :: Nil case CreateOrReplaceBranch( - IcebergCatalogAndIdentifier(catalog, ident), branch, branchOptions, create, replace, ifNotExists) => - CreateOrReplaceBranchExec(catalog, ident, branch, branchOptions, create, replace, ifNotExists) :: Nil + IcebergCatalogAndIdentifier(catalog, ident), + branch, + branchOptions, + create, + replace, + ifNotExists) => + CreateOrReplaceBranchExec( + catalog, + ident, + branch, + branchOptions, + create, + replace, + ifNotExists) :: Nil case CreateOrReplaceTag( - IcebergCatalogAndIdentifier(catalog, ident), tag, tagOptions, create, replace, ifNotExists) => + IcebergCatalogAndIdentifier(catalog, ident), + tag, + tagOptions, + create, + replace, + ifNotExists) => CreateOrReplaceTagExec(catalog, ident, tag, tagOptions, create, replace, ifNotExists) :: Nil case DropBranch(IcebergCatalogAndIdentifier(catalog, ident), branch, ifExists) => @@ -80,7 +96,11 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi case DropPartitionField(IcebergCatalogAndIdentifier(catalog, ident), transform) => DropPartitionFieldExec(catalog, ident, transform) :: Nil - case ReplacePartitionField(IcebergCatalogAndIdentifier(catalog, ident), transformFrom, transformTo, name) => + case ReplacePartitionField( + IcebergCatalogAndIdentifier(catalog, ident), + transformFrom, + transformTo, + name) => ReplacePartitionFieldExec(catalog, ident, transformFrom, transformTo, name) :: Nil case SetIdentifierFields(IcebergCatalogAndIdentifier(catalog, ident), fields) => @@ -90,13 +110,15 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi DropIdentifierFieldsExec(catalog, ident, fields) :: Nil case SetWriteDistributionAndOrdering( - IcebergCatalogAndIdentifier(catalog, ident), distributionMode, ordering) => + IcebergCatalogAndIdentifier(catalog, ident), + distributionMode, + ordering) => SetWriteDistributionAndOrderingExec(catalog, ident, distributionMode, ordering) :: Nil case OrderAwareCoalesce(numPartitions, coalescer, child) => OrderAwareCoalesceExec(numPartitions, coalescer, planLater(child)) :: Nil - case RenameTable(ResolvedV2View(oldCatalog: ViewCatalog, oldIdent), newName, isView@true) => + case RenameTable(ResolvedV2View(oldCatalog: ViewCatalog, oldIdent), newName, isView @ true) => val newIdent = Spark3Util.catalogAndIdentifier(spark, newName.toList.asJava) if (oldCatalog.name != newIdent.catalog().name()) { throw new IcebergAnalysisException( @@ -107,8 +129,18 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi case DropIcebergView(ResolvedIdentifier(viewCatalog: ViewCatalog, ident), ifExists) => DropV2ViewExec(viewCatalog, ident, ifExists) :: Nil - case CreateIcebergView(ResolvedIdentifier(viewCatalog: ViewCatalog, ident), queryText, query, - columnAliases, columnComments, queryColumnNames, comment, properties, allowExisting, replace, _) => + case CreateIcebergView( + ResolvedIdentifier(viewCatalog: ViewCatalog, ident), + queryText, + query, + columnAliases, + columnComments, + queryColumnNames, + comment, + properties, + allowExisting, + replace, + _) => CreateV2ViewExec( catalog = viewCatalog, ident = ident, diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala index 61d362044c3c..5dada1cab0bb 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -24,11 +23,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog - -case class RenameV2ViewExec( - catalog: ViewCatalog, - oldIdent: Identifier, - newIdent: Identifier) extends LeafV2CommandExec { +case class RenameV2ViewExec(catalog: ViewCatalog, oldIdent: Identifier, newIdent: Identifier) + extends LeafV2CommandExec { override lazy val output: Seq[Attribute] = Nil @@ -38,7 +34,6 @@ case class RenameV2ViewExec( Seq.empty } - override def simpleString(maxFields: Int): String = { s"RenameV2View ${oldIdent} to {newIdent}" } diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala index fcae0a5defc4..00b998c49e83 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.Spark3Util @@ -34,7 +33,8 @@ case class ReplacePartitionFieldExec( ident: Identifier, transformFrom: Transform, transformTo: Transform, - name: Option[String]) extends LeafV2CommandExec { + name: Option[String]) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -44,22 +44,26 @@ case class ReplacePartitionFieldExec( case iceberg: SparkTable => val schema = iceberg.table.schema transformFrom match { - case IdentityTransform(FieldReference(parts)) if parts.size == 1 && schema.findField(parts.head) == null => + case IdentityTransform(FieldReference(parts)) + if parts.size == 1 && schema.findField(parts.head) == null => // the name is not present in the Iceberg schema, so it must be a partition field name, not a column name - iceberg.table.updateSpec() - .removeField(parts.head) - .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) - .commit() + iceberg.table + .updateSpec() + .removeField(parts.head) + .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) + .commit() case _ => - iceberg.table.updateSpec() - .removeField(Spark3Util.toIcebergTerm(transformFrom)) - .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) - .commit() + iceberg.table + .updateSpec() + .removeField(Spark3Util.toIcebergTerm(transformFrom)) + .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) + .commit() } case table => - throw new UnsupportedOperationException(s"Cannot replace partition field in non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot replace partition field in non-Iceberg table: $table") } Nil @@ -67,6 +71,6 @@ case class ReplacePartitionFieldExec( override def simpleString(maxFields: Int): String = { s"ReplacePartitionField ${catalog.name}.${ident.quoted} ${transformFrom.describe} " + - s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" + s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" } } diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala index b50550ad38ef..50c53473ab60 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.source.SparkTable @@ -26,10 +25,8 @@ import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.TableCatalog import scala.jdk.CollectionConverters._ -case class SetIdentifierFieldsExec( - catalog: TableCatalog, - ident: Identifier, - fields: Seq[String]) extends LeafV2CommandExec { +case class SetIdentifierFieldsExec(catalog: TableCatalog, ident: Identifier, fields: Seq[String]) + extends LeafV2CommandExec { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil @@ -37,11 +34,13 @@ case class SetIdentifierFieldsExec( override protected def run(): Seq[InternalRow] = { catalog.loadTable(ident) match { case iceberg: SparkTable => - iceberg.table.updateSchema() + iceberg.table + .updateSchema() .setIdentifierFields(fields.asJava) .commit(); case table => - throw new UnsupportedOperationException(s"Cannot set identifier fields in non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot set identifier fields in non-Iceberg table: $table") } Nil diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala index c9004ddc5bda..9a10949d5e9e 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.DistributionMode @@ -36,7 +35,8 @@ case class SetWriteDistributionAndOrderingExec( catalog: TableCatalog, ident: Identifier, distributionMode: Option[DistributionMode], - sortOrder: Seq[(Term, SortDirection, NullOrder)]) extends LeafV2CommandExec { + sortOrder: Seq[(Term, SortDirection, NullOrder)]) + extends LeafV2CommandExec { import CatalogV2Implicits._ @@ -57,7 +57,8 @@ case class SetWriteDistributionAndOrderingExec( orderBuilder.commit() distributionMode.foreach { mode => - txn.updateProperties() + txn + .updateProperties() .set(WRITE_DISTRIBUTION_MODE, mode.modeName()) .commit() } @@ -65,7 +66,8 @@ case class SetWriteDistributionAndOrderingExec( txn.commitTransaction() case table => - throw new UnsupportedOperationException(s"Cannot set write order of non-Iceberg table: $table") + throw new UnsupportedOperationException( + s"Cannot set write order of non-Iceberg table: $table") } Nil @@ -73,9 +75,11 @@ case class SetWriteDistributionAndOrderingExec( override def simpleString(maxFields: Int): String = { val tableIdent = s"${catalog.name}.${ident.quoted}" - val order = sortOrder.map { - case (term, direction, nullOrder) => s"$term $direction $nullOrder" - }.mkString(", ") + val order = sortOrder + .map { case (term, direction, nullOrder) => + s"$term $direction $nullOrder" + } + .mkString(", ") s"SetWriteDistributionAndOrdering $tableIdent $distributionMode $order" } } diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala index ecd3e0d3c9ba..07ac4aeda8fb 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -28,7 +27,8 @@ import org.apache.spark.sql.execution.LeafExecNode import scala.jdk.CollectionConverters._ case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View) - extends V2CommandExec with LeafExecNode { + extends V2CommandExec + with LeafExecNode { override protected def run(): Seq[InternalRow] = { val builder = new StringBuilder @@ -43,7 +43,9 @@ case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View) private def showColumns(view: View, builder: StringBuilder): Unit = { val columns = concatByMultiLines( - view.schema().fields + view + .schema() + .fields .map(x => s"${x.name}${x.getComment().map(c => s" COMMENT '$c'").getOrElse("")}")) builder ++= columns } @@ -54,14 +56,11 @@ case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View) .foreach(builder.append) } - private def showProperties( - view: View, - builder: StringBuilder): Unit = { + private def showProperties(view: View, builder: StringBuilder): Unit = { val showProps = view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala if (showProps.nonEmpty) { - val props = conf.redactOptions(showProps).toSeq.sortBy(_._1).map { - case (key, value) => - s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" + val props = conf.redactOptions(showProps).toSeq.sortBy(_._1).map { case (key, value) => + s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" } builder ++= "TBLPROPERTIES " diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala index 553b80833087..ace43eb6c07b 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -26,25 +25,22 @@ import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.execution.LeafExecNode import scala.jdk.CollectionConverters._ -case class ShowV2ViewPropertiesExec( - output: Seq[Attribute], - view: View, - propertyKey: Option[String]) extends V2CommandExec with LeafExecNode { +case class ShowV2ViewPropertiesExec(output: Seq[Attribute], view: View, propertyKey: Option[String]) + extends V2CommandExec + with LeafExecNode { override protected def run(): Seq[InternalRow] = { propertyKey match { case Some(p) => - val propValue = properties.getOrElse(p, - s"View ${view.name()} does not have property: $p") + val propValue = properties.getOrElse(p, s"View ${view.name()} does not have property: $p") Seq(toCatalystRow(p, propValue)) case None => - properties.map { - case (k, v) => toCatalystRow(k, v) + properties.map { case (k, v) => + toCatalystRow(k, v) }.toSeq } } - private def properties = { view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala } diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala index d82d13a3edd9..4e7700b43978 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources.v2 import org.apache.spark.sql.catalyst.InternalRow @@ -28,10 +27,12 @@ import org.apache.spark.sql.internal.SQLConf import scala.collection.mutable.ArrayBuffer case class ShowV2ViewsExec( - output: Seq[Attribute], - catalog: ViewCatalog, - namespace: Seq[String], - pattern: Option[String]) extends V2CommandExec with LeafExecNode { + output: Seq[Attribute], + catalog: ViewCatalog, + namespace: Seq[String], + pattern: Option[String]) + extends V2CommandExec + with LeafExecNode { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ @@ -41,7 +42,8 @@ case class ShowV2ViewsExec( // handle GLOBAL VIEWS val globalTemp = SQLConf.get.globalTempDatabase if (namespace.nonEmpty && globalTemp == namespace.head) { - pattern.map(p => session.sessionState.catalog.globalTempViewManager.listViewNames(p)) + pattern + .map(p => session.sessionState.catalog.globalTempViewManager.listViewNames(p)) .getOrElse(session.sessionState.catalog.globalTempViewManager.listViewNames("*")) .map(name => rows += toCatalystRow(globalTemp, name, true)) } else { @@ -54,7 +56,8 @@ case class ShowV2ViewsExec( } // include TEMP VIEWS - pattern.map(p => session.sessionState.catalog.listLocalTempViews(p)) + pattern + .map(p => session.sessionState.catalog.listLocalTempViews(p)) .getOrElse(session.sessionState.catalog.listLocalTempViews("*")) .map(v => rows += toCatalystRow(v.database.toArray.quoted, v.table, true)) diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala index 5acaa6800e68..5d5990c23b57 100644 --- a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.rdd.PartitionCoalescer @@ -26,10 +25,8 @@ import org.apache.spark.sql.catalyst.expressions.Attribute // this node doesn't extend RepartitionOperation on purpose to keep this logic isolated // and ignore it in optimizer rules such as CollapseRepartition -case class OrderAwareCoalesce( - numPartitions: Int, - coalescer: PartitionCoalescer, - child: LogicalPlan) extends OrderPreservingUnaryNode { +case class OrderAwareCoalesce(numPartitions: Int, coalescer: PartitionCoalescer, child: LogicalPlan) + extends OrderPreservingUnaryNode { override def output: Seq[Attribute] = child.output diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala index 7b599eb3da1d..3ce3d3e82ad8 100644 --- a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.iceberg.DistributionMode @@ -29,16 +28,19 @@ import org.apache.spark.sql.connector.catalog.CatalogV2Implicits case class SetWriteDistributionAndOrdering( table: Seq[String], distributionMode: Option[DistributionMode], - sortOrder: Seq[(Term, SortDirection, NullOrder)]) extends LeafCommand { + sortOrder: Seq[(Term, SortDirection, NullOrder)]) + extends LeafCommand { import CatalogV2Implicits._ override lazy val output: Seq[Attribute] = Nil override def simpleString(maxFields: Int): String = { - val order = sortOrder.map { - case (term, direction, nullOrder) => s"$term $direction $nullOrder" - }.mkString(", ") + val order = sortOrder + .map { case (term, direction, nullOrder) => + s"$term $direction $nullOrder" + } + .mkString(", ") s"SetWriteDistributionAndOrdering ${table.quoted} $distributionMode $order" } } diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala index bf19ef8a2167..71b6107d048e 100644 --- a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.plans.logical import org.apache.iceberg.NullOrder @@ -27,7 +26,9 @@ import org.apache.iceberg.expressions.Term class SortOrderParserUtil { - def collectSortOrder(tableSchema:Schema, sortOrder: Seq[(Term, SortDirection, NullOrder)]): SortOrder = { + def collectSortOrder( + tableSchema: Schema, + sortOrder: Seq[(Term, SortDirection, NullOrder)]): SortOrder = { val orderBuilder = SortOrder.builderFor(tableSchema) sortOrder.foreach { case (term, SortDirection.ASC, nullOrder) => diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala index aa9e9c553346..f46127f818e5 100644 --- a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.catalyst.utils import org.apache.iceberg.spark.source.SparkTable diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala index 2ef99550524a..83dec98c0d9a 100644 --- a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution import org.apache.spark.rdd.PartitionCoalescer @@ -31,7 +30,8 @@ import org.apache.spark.sql.catalyst.plans.physical.UnknownPartitioning case class OrderAwareCoalesceExec( numPartitions: Int, coalescer: PartitionCoalescer, - child: SparkPlan) extends UnaryExecNode { + child: SparkPlan) + extends UnaryExecNode { override def output: Seq[Attribute] = child.output diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala index f684ca113cf8..0be425f7d796 100644 --- a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.execution.datasources import org.apache.iceberg.spark.SparkV2Filters @@ -32,7 +31,8 @@ import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy object SparkExpressionConverter { - def convertToIcebergExpression(sparkExpression: Expression): org.apache.iceberg.expressions.Expression = { + def convertToIcebergExpression( + sparkExpression: Expression): org.apache.iceberg.expressions.Expression = { // Currently, it is a double conversion as we are converting Spark expression to Spark predicate // and then converting Spark predicate to Iceberg expression. // But these two conversions already exist and well tested. So, we are going with this approach. @@ -40,26 +40,33 @@ object SparkExpressionConverter { case Some(filter) => val converted = SparkV2Filters.convert(filter) if (converted == null) { - throw new IllegalArgumentException(s"Cannot convert Spark filter: $filter to Iceberg expression") + throw new IllegalArgumentException( + s"Cannot convert Spark filter: $filter to Iceberg expression") } converted case _ => - throw new IllegalArgumentException(s"Cannot translate Spark expression: $sparkExpression to data source filter") + throw new IllegalArgumentException( + s"Cannot translate Spark expression: $sparkExpression to data source filter") } } @throws[IcebergAnalysisException] - def collectResolvedSparkExpression(session: SparkSession, tableName: String, where: String): Expression = { + def collectResolvedSparkExpression( + session: SparkSession, + tableName: String, + where: String): Expression = { val tableAttrs = session.table(tableName).queryExecution.analyzed.output val unresolvedExpression = session.sessionState.sqlParser.parseExpression(where) val filter = Filter(unresolvedExpression, DummyRelation(tableAttrs)) val optimizedLogicalPlan = session.sessionState.executePlan(filter).optimizedPlan - optimizedLogicalPlan.collectFirst { - case filter: Filter => filter.condition - case _: DummyRelation => Literal.TrueLiteral - case _: LocalRelation => Literal.FalseLiteral - }.getOrElse(throw new IcebergAnalysisException("Failed to find filter expression")) + optimizedLogicalPlan + .collectFirst { + case filter: Filter => filter.condition + case _: DummyRelation => Literal.TrueLiteral + case _: LocalRelation => Literal.FalseLiteral + } + .getOrElse(throw new IcebergAnalysisException("Failed to find filter expression")) } case class DummyRelation(output: Seq[Attribute]) extends LeafNode diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala index b132f7457e72..c63f3c9c5cf6 100644 --- a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.spark.sql.stats import java.nio.ByteBuffer @@ -46,12 +45,13 @@ import org.apache.spark.unsafe.types.UTF8String * which should be deserialized to Compact sketch before using. * * See [[https://iceberg.apache.org/puffin-spec/]] for more information. - * */ case class ThetaSketchAgg( - child: Expression, - mutableAggBufferOffset: Int = 0, - inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[Sketch] with UnaryLike[Expression] { + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) + extends TypedImperativeAggregate[Sketch] + with UnaryLike[Expression] { private lazy val icebergType = SparkSchemaUtil.convert(child.dataType) @@ -102,7 +102,8 @@ case class ThetaSketchAgg( CompactSketch.wrap(Memory.wrap(storageFormat)) } - override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = { + override def withNewMutableAggBufferOffset( + newMutableAggBufferOffset: Int): ImperativeAggregate = { copy(mutableAggBufferOffset = newMutableAggBufferOffset) } From 29a144adca05344de2a8aab05e2b3385c4053c8f Mon Sep 17 00:00:00 2001 From: aiborodin Date: Wed, 3 Dec 2025 04:41:32 +1100 Subject: [PATCH 027/201] Refactor SnapshotAncestryValidator (#14650) Remove the extension of the Function interface to return a primitive boolean. --- .../java/org/apache/iceberg/SnapshotAncestryValidator.java | 6 ++---- core/src/main/java/org/apache/iceberg/SnapshotProducer.java | 2 +- .../test/java/org/apache/iceberg/TestSnapshotProducer.java | 2 +- .../apache/iceberg/flink/sink/dynamic/DynamicCommitter.java | 2 +- .../apache/iceberg/flink/sink/dynamic/DynamicCommitter.java | 2 +- .../apache/iceberg/flink/sink/dynamic/DynamicCommitter.java | 2 +- .../org/apache/iceberg/connect/channel/Coordinator.java | 2 +- 7 files changed, 8 insertions(+), 10 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/SnapshotAncestryValidator.java b/api/src/main/java/org/apache/iceberg/SnapshotAncestryValidator.java index 64b579a1a377..6cb486c65f5c 100644 --- a/api/src/main/java/org/apache/iceberg/SnapshotAncestryValidator.java +++ b/api/src/main/java/org/apache/iceberg/SnapshotAncestryValidator.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg; -import java.util.function.Function; import javax.annotation.Nonnull; /** @@ -28,7 +27,7 @@ * table state. */ @FunctionalInterface -public interface SnapshotAncestryValidator extends Function, Boolean> { +public interface SnapshotAncestryValidator { SnapshotAncestryValidator NON_VALIDATING = baseSnapshots -> true; @@ -38,8 +37,7 @@ public interface SnapshotAncestryValidator extends Function, * @param baseSnapshots ancestry of the base table metadata snapshots * @return boolean for whether the update is valid */ - @Override - Boolean apply(Iterable baseSnapshots); + boolean validate(Iterable baseSnapshots); /** * Validation message that will be included when throwing {@link diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index ce02637d98d3..876e0190d28c 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -354,7 +354,7 @@ private void runValidations(Snapshot parentSnapshot) { ? SnapshotUtil.ancestorsOf(parentSnapshot.snapshotId(), base::snapshot) : List.of(); - boolean valid = snapshotAncestryValidator.apply(snapshotAncestry); + boolean valid = snapshotAncestryValidator.validate(snapshotAncestry); ValidationException.check( valid, "Snapshot ancestry validation failed: %s", snapshotAncestryValidator.errorMessage()); } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java b/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java index 956242f66e33..ad25a30b56e7 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java @@ -95,7 +95,7 @@ public void testCommitValidationPreventsCommit() throws IOException { SnapshotAncestryValidator validator = new SnapshotAncestryValidator() { @Override - public Boolean apply(Iterable baseSnapshots) { + public boolean validate(Iterable baseSnapshots) { return false; } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java index 61b20cb27b4b..1cddc64d6016 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java @@ -369,7 +369,7 @@ private MaxCommittedCheckpointIdValidator( } @Override - public Boolean apply(Iterable baseSnapshots) { + public boolean validate(Iterable baseSnapshots) { long maxCommittedCheckpointId = getMaxCommittedCheckpointId(baseSnapshots, flinkJobId, flinkOperatorId); if (maxCommittedCheckpointId >= stagedCheckpointId) { diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java index 61b20cb27b4b..1cddc64d6016 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java @@ -369,7 +369,7 @@ private MaxCommittedCheckpointIdValidator( } @Override - public Boolean apply(Iterable baseSnapshots) { + public boolean validate(Iterable baseSnapshots) { long maxCommittedCheckpointId = getMaxCommittedCheckpointId(baseSnapshots, flinkJobId, flinkOperatorId); if (maxCommittedCheckpointId >= stagedCheckpointId) { diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java index 61b20cb27b4b..1cddc64d6016 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java @@ -369,7 +369,7 @@ private MaxCommittedCheckpointIdValidator( } @Override - public Boolean apply(Iterable baseSnapshots) { + public boolean validate(Iterable baseSnapshots) { long maxCommittedCheckpointId = getMaxCommittedCheckpointId(baseSnapshots, flinkJobId, flinkOperatorId); if (maxCommittedCheckpointId >= stagedCheckpointId) { diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java index 5c516311bd4c..1e0fa3286fe5 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java @@ -308,7 +308,7 @@ private SnapshotAncestryValidator offsetValidator( private Map lastCommittedOffsets; @Override - public Boolean apply(Iterable baseSnapshots) { + public boolean validate(Iterable baseSnapshots) { lastCommittedOffsets = lastCommittedOffsets(baseSnapshots); return expectedOffsets.equals(lastCommittedOffsets); From 9896e8cccac2b245ed20bfae602b29fc640dcd36 Mon Sep 17 00:00:00 2001 From: Drew Gallardo Date: Tue, 2 Dec 2025 10:33:02 -0800 Subject: [PATCH 028/201] Core: Align ContentFile partition JSON with REST spec (#14702) * Core: Align ContentFile partition JSON with REST spec * Handle backwards compat * address comments --- .../org/apache/iceberg/ContentFileParser.java | 70 ++++++++-- .../apache/iceberg/TestContentFileParser.java | 129 ++++++++++++++++-- .../apache/iceberg/TestDataTaskParser.java | 21 ++- .../iceberg/TestFileScanTaskParser.java | 38 +++++- ...TestFetchPlanningResultResponseParser.java | 6 +- .../TestFetchScanTasksResponseParser.java | 6 +- .../TestPlanTableScanResponseParser.java | 40 ++---- 7 files changed, 246 insertions(+), 64 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/ContentFileParser.java b/core/src/main/java/org/apache/iceberg/ContentFileParser.java index 3d0deba3dfcb..0033fa97725a 100644 --- a/core/src/main/java/org/apache/iceberg/ContentFileParser.java +++ b/core/src/main/java/org/apache/iceberg/ContentFileParser.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Types; import org.apache.iceberg.util.JsonUtil; public class ContentFileParser { @@ -89,7 +90,7 @@ public static void toJson(ContentFile contentFile, PartitionSpec spec, JsonGe if (contentFile.partition() != null) { generator.writeFieldName(PARTITION); - SingleValueParser.toJson(spec.partitionType(), contentFile.partition(), generator); + partitionToJson(spec.partitionType(), contentFile.partition(), generator); } generator.writeNumberField(FILE_SIZE, contentFile.fileSizeInBytes()); @@ -152,18 +153,7 @@ public static ContentFile fromJson(JsonNode jsonNode, Map javaClass = spec.partitionType().fields().get(pos).type().typeId().javaClass(); - partitionData.set(pos, structLike.get(pos, javaClass)); - } + partitionData = partitionFromJson(spec.partitionType(), jsonNode.get(PARTITION)); } long fileSizeInBytes = JsonUtil.getLong(FILE_SIZE, jsonNode); @@ -301,4 +291,58 @@ private static Metrics metricsFromJson(JsonNode jsonNode) { lowerBounds, upperBounds); } + + private static void partitionToJson( + Types.StructType partitionType, StructLike partitionData, JsonGenerator generator) + throws IOException { + generator.writeStartArray(); + List fields = partitionType.fields(); + for (int pos = 0; pos < fields.size(); ++pos) { + Types.NestedField field = fields.get(pos); + Object partitionValue = partitionData.get(pos, Object.class); + SingleValueParser.toJson(field.type(), partitionValue, generator); + } + generator.writeEndArray(); + } + + private static PartitionData partitionFromJson( + Types.StructType partitionType, JsonNode partitionNode) { + List fields = partitionType.fields(); + PartitionData partitionData = new PartitionData(partitionType); + + if (partitionNode.isArray()) { + Preconditions.checkArgument( + partitionNode.size() == fields.size(), + "Invalid partition data size: expected = %s, actual = %s", + fields.size(), + partitionNode.size()); + + for (int pos = 0; pos < fields.size(); ++pos) { + Types.NestedField field = fields.get(pos); + Object partitionValue = SingleValueParser.fromJson(field.type(), partitionNode.get(pos)); + partitionData.set(pos, partitionValue); + } + } else if (partitionNode.isObject()) { + // Handle partition struct object format, which serializes by field ID and skips + // null partition values + Preconditions.checkState( + partitionNode.size() <= fields.size(), + "Invalid partition data size: expected <= %s, actual = %s", + fields.size(), + partitionNode.size()); + + StructLike structLike = (StructLike) SingleValueParser.fromJson(partitionType, partitionNode); + for (int pos = 0; pos < partitionData.size(); ++pos) { + Class javaClass = fields.get(pos).type().typeId().javaClass(); + partitionData.set(pos, structLike.get(pos, javaClass)); + } + } else { + throw new IllegalArgumentException( + String.format( + "Invalid partition data for content file: expected array or object (%s)", + partitionNode)); + } + + return partitionData; + } } diff --git a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java index f8f0d77b0f0e..b3cca25bdcf0 100644 --- a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java +++ b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java @@ -87,7 +87,7 @@ public void testDataFile(PartitionSpec spec, DataFile dataFile, String expectedJ assertThat(jsonStr).isEqualTo(expectedJson); JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); ContentFile deserializedContentFile = - ContentFileParser.fromJson(jsonNode, Map.of(TestBase.SPEC.specId(), spec)); + ContentFileParser.fromJson(jsonNode, Map.of(spec.specId(), spec)); assertThat(deserializedContentFile).isInstanceOf(DataFile.class); assertContentFileEquals(dataFile, deserializedContentFile, spec); } @@ -100,11 +100,116 @@ public void testDeleteFile(PartitionSpec spec, DeleteFile deleteFile, String exp assertThat(jsonStr).isEqualTo(expectedJson); JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); ContentFile deserializedContentFile = - ContentFileParser.fromJson(jsonNode, Map.of(spec.specId(), TestBase.SPEC)); + ContentFileParser.fromJson(jsonNode, Map.of(spec.specId(), spec)); assertThat(deserializedContentFile).isInstanceOf(DeleteFile.class); assertContentFileEquals(deleteFile, deserializedContentFile, spec); } + @Test + public void testPartitionJsonArrayWrongSize() throws Exception { + PartitionSpec spec = PartitionSpec.builderFor(TestBase.SCHEMA).identity("data").build(); + String jsonStr = + "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":[],\"file-size-in-bytes\":10," + + "\"record-count\":1}"; + + JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); + + assertThatThrownBy(() -> ContentFileParser.fromJson(jsonNode, Map.of(spec.specId(), spec))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid partition data size"); + } + + @Test + public void testPartitionJsonInvalidType() throws Exception { + PartitionSpec spec = PartitionSpec.builderFor(TestBase.SCHEMA).identity("data").build(); + String jsonStr = + "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":\"invalid\",\"file-size-in-bytes\":10," + + "\"record-count\":1}"; + + JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); + + assertThatThrownBy(() -> ContentFileParser.fromJson(jsonNode, Map.of(spec.specId(), spec))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("expected array or object"); + } + + @Test + public void testParsesFieldIdPartitionMap() throws Exception { + PartitionSpec spec = PartitionSpec.builderFor(TestBase.SCHEMA).identity("data").build(); + String legacyJson = + "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":\"foo\"},\"file-size-in-bytes\":10," + + "\"record-count\":1}"; + + JsonNode jsonNode = JsonUtil.mapper().readTree(legacyJson); + ContentFile deserializedContentFile = + ContentFileParser.fromJson(jsonNode, Map.of(spec.specId(), spec)); + + assertThat(deserializedContentFile).isInstanceOf(DataFile.class); + assertThat(deserializedContentFile.partition().get(0, String.class)).isEqualTo("foo"); + } + + @Test + public void testPartitionStructObjectContainsExtraField() throws Exception { + PartitionSpec spec = PartitionSpec.builderFor(TestBase.SCHEMA).identity("data").build(); + String jsonStr = + "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":\"foo\",\"9999\":\"bar\"}," + + "\"file-size-in-bytes\":10,\"record-count\":1}"; + + JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); + assertThatThrownBy(() -> ContentFileParser.fromJson(jsonNode, Map.of(spec.specId(), spec))) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Invalid partition data size"); + } + + @Test + public void testPartitionStructObjectEmptyIsNull() throws Exception { + PartitionSpec spec = PartitionSpec.builderFor(TestBase.SCHEMA).identity("data").build(); + String jsonStr = + "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{},\"file-size-in-bytes\":10," + + "\"record-count\":1}"; + + JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); + ContentFile contentFile = ContentFileParser.fromJson(jsonNode, Map.of(spec.specId(), spec)); + assertThat(contentFile).isInstanceOf(DataFile.class); + assertThat(contentFile.partition().get(0, String.class)).isNull(); + } + + @Test + public void testPartitionArrayRespectsSpecOrder() throws Exception { + PartitionSpec spec = + PartitionSpec.builderFor(TestBase.SCHEMA).identity("id").identity("data").build(); + + PartitionData partitionData = new PartitionData(spec.partitionType()); + partitionData.set(0, 4); + partitionData.set(1, "foo"); + + DataFile dataFile = + DataFiles.builder(spec) + .withPath("/path/to/data.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .withPartition(partitionData) + .build(); + + String jsonStr = ContentFileParser.toJson(dataFile, spec); + + // Verify partition values are serialized as array in correct order + assertThat(jsonStr).contains("\"partition\":[4,\"foo\"]"); + + JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); + ContentFile deserializedContentFile = + ContentFileParser.fromJson(jsonNode, Map.of(spec.specId(), spec)); + + assertThat(deserializedContentFile).isInstanceOf(DataFile.class); + assertThat(deserializedContentFile.partition().get(0, Integer.class)).isEqualTo(4); + assertThat(deserializedContentFile.partition().get(1, String.class)).isEqualTo("foo"); + } + private static Stream provideSpecAndDataFile() { return Stream.of( Arguments.of( @@ -167,17 +272,17 @@ private static DataFile dataFileWithOnlyNanCounts(PartitionSpec spec) { private static String dataFileJsonWithRequiredOnly(PartitionSpec spec) { if (spec.isUnpartitioned()) { return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\",\"file-format\":\"PARQUET\"," - + "\"partition\":{},\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}"; + + "\"partition\":[],\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}"; } else { return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\",\"file-format\":\"PARQUET\"," - + "\"partition\":{\"1000\":1},\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}"; + + "\"partition\":[1],\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}"; } } private static String dataFileJsonWithAllOptional(PartitionSpec spec) { if (spec.isUnpartitioned()) { return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-with-stats.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{},\"file-size-in-bytes\":350,\"record-count\":10," + + "\"file-format\":\"PARQUET\",\"partition\":[],\"file-size-in-bytes\":350,\"record-count\":10," + "\"column-sizes\":{\"keys\":[3,4],\"values\":[100,200]}," + "\"value-counts\":{\"keys\":[3,4],\"values\":[90,180]}," + "\"null-value-counts\":{\"keys\":[3,4],\"values\":[10,20]}," @@ -188,7 +293,7 @@ private static String dataFileJsonWithAllOptional(PartitionSpec spec) { + "\"split-offsets\":[128,256],\"sort-order-id\":1}"; } else { return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-with-stats.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":1},\"file-size-in-bytes\":350,\"record-count\":10," + + "\"file-format\":\"PARQUET\",\"partition\":[1],\"file-size-in-bytes\":350,\"record-count\":10," + "\"column-sizes\":{\"keys\":[3,4],\"values\":[100,200]}," + "\"value-counts\":{\"keys\":[3,4],\"values\":[90,180]}," + "\"null-value-counts\":{\"keys\":[3,4],\"values\":[10,20]}," @@ -284,7 +389,7 @@ private static DeleteFile deleteFileWithDataRef(PartitionSpec spec) { private static String deleteFileWithDataRefJson() { return "{\"spec-id\":0,\"content\":\"POSITION_DELETES\",\"file-path\":\"/path/to/delete.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":4},\"file-size-in-bytes\":1234," + + "\"file-format\":\"PARQUET\",\"partition\":[4],\"file-size-in-bytes\":1234," + "\"record-count\":10,\"referenced-data-file\":\"/path/to/data/file.parquet\"}"; } @@ -310,7 +415,7 @@ private static DeleteFile dv(PartitionSpec spec) { private static String dvJson() { return "{\"spec-id\":0,\"content\":\"POSITION_DELETES\",\"file-path\":\"/path/to/delete.puffin\"," - + "\"file-format\":\"PUFFIN\",\"partition\":{\"1000\":4},\"file-size-in-bytes\":1234,\"record-count\":10," + + "\"file-format\":\"PUFFIN\",\"partition\":[4],\"file-size-in-bytes\":1234,\"record-count\":10," + "\"referenced-data-file\":\"/path/to/data/file.parquet\",\"content-offset\":4,\"content-size-in-bytes\":40}"; } @@ -383,17 +488,17 @@ private static DeleteFile deleteFileWithAllOptional(PartitionSpec spec) { private static String deleteFileJsonWithRequiredOnly(PartitionSpec spec) { if (spec.isUnpartitioned()) { return "{\"spec-id\":0,\"content\":\"POSITION_DELETES\",\"file-path\":\"/path/to/delete-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{},\"file-size-in-bytes\":1234,\"record-count\":9}"; + + "\"file-format\":\"PARQUET\",\"partition\":[],\"file-size-in-bytes\":1234,\"record-count\":9}"; } else { return "{\"spec-id\":0,\"content\":\"POSITION_DELETES\",\"file-path\":\"/path/to/delete-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":9},\"file-size-in-bytes\":1234,\"record-count\":9}"; + + "\"file-format\":\"PARQUET\",\"partition\":[9],\"file-size-in-bytes\":1234,\"record-count\":9}"; } } private static String deleteFileJsonWithAllOptional(PartitionSpec spec) { if (spec.isUnpartitioned()) { return "{\"spec-id\":0,\"content\":\"EQUALITY_DELETES\",\"file-path\":\"/path/to/delete-with-stats.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{},\"file-size-in-bytes\":1234,\"record-count\":10," + + "\"file-format\":\"PARQUET\",\"partition\":[],\"file-size-in-bytes\":1234,\"record-count\":10," + "\"column-sizes\":{\"keys\":[3,4],\"values\":[100,200]}," + "\"value-counts\":{\"keys\":[3,4],\"values\":[90,180]}," + "\"null-value-counts\":{\"keys\":[3,4],\"values\":[10,20]}," @@ -404,7 +509,7 @@ private static String deleteFileJsonWithAllOptional(PartitionSpec spec) { + "\"split-offsets\":[128],\"equality-ids\":[3],\"sort-order-id\":1}"; } else { return "{\"spec-id\":0,\"content\":\"EQUALITY_DELETES\",\"file-path\":\"/path/to/delete-with-stats.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":9},\"file-size-in-bytes\":1234,\"record-count\":10," + + "\"file-format\":\"PARQUET\",\"partition\":[9],\"file-size-in-bytes\":1234,\"record-count\":10," + "\"column-sizes\":{\"keys\":[3,4],\"values\":[100,200]}," + "\"value-counts\":{\"keys\":[3,4],\"values\":[90,180]}," + "\"null-value-counts\":{\"keys\":[3,4],\"values\":[10,20]}," diff --git a/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java b/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java index 8048b6d24e88..758c11db788a 100644 --- a/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java +++ b/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java @@ -146,7 +146,7 @@ public void missingFields() throws Exception { + "\"value\":\"string\",\"value-required\":true}}]}," + "\"metadata-file\":{\"spec-id\":0,\"content\":\"DATA\"," + "\"file-path\":\"/tmp/metadata2.json\"," - + "\"file-format\":\"METADATA\",\"partition\":{}," + + "\"file-format\":\"METADATA\",\"partition\":[]," + "\"file-size-in-bytes\":0,\"record-count\":2,\"sort-order-id\":0}" + "}"; JsonNode missingTableRowsNode = mapper.reader().readTree(missingTableRowsStr); @@ -164,6 +164,23 @@ public void roundTripSerde() { assertDataTaskEquals(dataTask, deserializedTask); } + @Test + public void testDataTaskParsesFieldIdPartitionMap() { + String jsonStr = + "{\"task-type\":\"data-task\"," + + "\"schema\":{\"type\":\"struct\",\"schema-id\":0," + + "\"fields\":[{\"id\":1,\"name\":\"committed_at\",\"required\":true,\"type\":\"timestamptz\"}]}," + + "\"projection\":{\"type\":\"struct\",\"schema-id\":0," + + "\"fields\":[{\"id\":1,\"name\":\"committed_at\",\"required\":true,\"type\":\"timestamptz\"}]}," + + "\"metadata-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/tmp/metadata.json\"," + + "\"file-format\":\"METADATA\",\"partition\":{},\"file-size-in-bytes\":0,\"record-count\":1,\"sort-order-id\":0}," + + "\"rows\":[{\"1\":\"2009-02-13T23:31:30+00:00\"}]}"; + + StaticDataTask deserializedTask = (StaticDataTask) ScanTaskParser.fromJson(jsonStr, true); + + assertThat(deserializedTask.metadataFile().partition().size()).isEqualTo(0); + } + private DataTask createDataTask() { Map summary1 = ImmutableMap.of( @@ -248,7 +265,7 @@ private String snapshotsDataTaskJson() { + "\"value\":\"string\",\"value-required\":true}}]}," + "\"metadata-file\":{\"spec-id\":0,\"content\":\"DATA\"," + "\"file-path\":\"/tmp/metadata2.json\"," - + "\"file-format\":\"METADATA\",\"partition\":{}," + + "\"file-format\":\"METADATA\",\"partition\":[]," + "\"file-size-in-bytes\":0,\"record-count\":2,\"sort-order-id\":0}," + "\"rows\":[{\"1\":\"2009-02-13T23:31:30+00:00\",\"2\":1,\"4\":\"append\"," + "\"5\":\"file:/tmp/manifest1.avro\"," diff --git a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java index c4a9fdf2340a..882c2b33496d 100644 --- a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java +++ b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java @@ -62,6 +62,18 @@ public void testScanTaskParserWithoutTaskTypeField(boolean caseSensitive) { assertFileScanTaskEquals(fileScanTask, deserializedTask, spec, caseSensitive); } + @Test + public void testFileScanTaskParsesFieldIdPartitionMap() { + boolean caseSensitive = true; + PartitionSpec spec = TestBase.SPEC; + FileScanTask expected = createFileScanTask(spec, caseSensitive); + + FileScanTask deserializedTask = + ScanTaskParser.fromJson(fileScanTaskFieldIdPartitionMapJson(), caseSensitive); + + assertFileScanTaskEquals(expected, deserializedTask, spec, caseSensitive); + } + private FileScanTask createFileScanTask(PartitionSpec spec, boolean caseSensitive) { ResidualEvaluator residualEvaluator; if (spec.isUnpartitioned()) { @@ -85,19 +97,39 @@ private String fileScanTaskJsonWithoutTaskType() { + "\"spec\":{\"spec-id\":0,\"fields\":[{\"name\":\"data_bucket\"," + "\"transform\":\"bucket[16]\",\"source-id\":4,\"field-id\":1000}]}," + "\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":0}," + + "\"file-format\":\"PARQUET\",\"partition\":[0]," + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + "\"start\":0,\"length\":10," + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," - + "\"partition\":{\"1000\":0},\"file-size-in-bytes\":10,\"record-count\":1}," + + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}," + "{\"spec-id\":0,\"content\":\"EQUALITY_DELETES\",\"file-path\":\"/path/to/data-a2-deletes.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":0},\"file-size-in-bytes\":10," + + "\"file-format\":\"PARQUET\",\"partition\":[0],\"file-size-in-bytes\":10," + "\"record-count\":1,\"equality-ids\":[1],\"sort-order-id\":0}]," + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}"; } private String fileScanTaskJson() { + return "{\"task-type\":\"file-scan-task\"," + + "\"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" + + "{\"id\":3,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":4,\"name\":\"data\",\"required\":true,\"type\":\"string\"}]}," + + "\"spec\":{\"spec-id\":0,\"fields\":[{\"name\":\"data_bucket\"," + + "\"transform\":\"bucket[16]\",\"source-id\":4,\"field-id\":1000}]}," + + "\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":[0]," + + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + + "\"start\":0,\"length\":10," + + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," + + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," + + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}," + + "{\"spec-id\":0,\"content\":\"EQUALITY_DELETES\",\"file-path\":\"/path/to/data-a2-deletes.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":[0],\"file-size-in-bytes\":10," + + "\"record-count\":1,\"equality-ids\":[1],\"sort-order-id\":0}]," + + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}"; + } + + private String fileScanTaskFieldIdPartitionMapJson() { return "{\"task-type\":\"file-scan-task\"," + "\"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" + "{\"id\":3,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java index 9db9e29530f1..8b82e9794d5d 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java @@ -157,7 +157,7 @@ public void roundTripSerdeWithInvalidPlanStatusSubmittedWithDeleteFilesNoFileSca "{\"status\":\"submitted\"," + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," - + "\"partition\":{\"1000\":0},\"file-size-in-bytes\":10,\"record-count\":1}]" + + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}]" + "}"; assertThatThrownBy( @@ -195,10 +195,10 @@ public void roundTripSerdeWithValidStatusAndFileScanTasks() throws JsonProcessin "{\"status\":\"completed\"," + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," - + "\"partition\":{\"1000\":0},\"file-size-in-bytes\":10,\"record-count\":1}]," + + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}]," + "\"file-scan-tasks\":[" + "{\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":0}," + + "\"file-format\":\"PARQUET\",\"partition\":[0]," + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + "\"delete-file-references\":[0]," + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}]" diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchScanTasksResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchScanTasksResponseParser.java index 01ca8288fb40..62de75e3818b 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchScanTasksResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchScanTasksResponseParser.java @@ -97,7 +97,7 @@ public void roundTripSerdeWithDeleteFilesNoFileScanTasksPresent() { "{\"plan-tasks\":[\"task1\",\"task2\"]," + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," - + "\"partition\":{\"1000\":0},\"file-size-in-bytes\":10,\"record-count\":1}]" + + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}]" + "}"; assertThatThrownBy( @@ -131,10 +131,10 @@ public void roundTripSerdeWithFileScanTasks() { "{" + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," - + "\"partition\":{\"1000\":0},\"file-size-in-bytes\":10,\"record-count\":1}]," + + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}]," + "\"file-scan-tasks\":[" + "{\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":0}," + + "\"file-format\":\"PARQUET\",\"partition\":[0]," + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + "\"delete-file-references\":[0]," + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}]" diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestPlanTableScanResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestPlanTableScanResponseParser.java index c2867ca3a25f..a59bbbd16b45 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestPlanTableScanResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestPlanTableScanResponseParser.java @@ -212,7 +212,7 @@ public void roundTripSerdeWithInvalidPlanStatusSubmittedWithDeleteFilesNoFileSca + "\"plan-id\":\"somePlanId\"," + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," - + "\"partition\":{\"1000\":0},\"file-size-in-bytes\":10,\"record-count\":1}]" + + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}]" + "}"; assertThatThrownBy( @@ -246,10 +246,10 @@ public void roundTripSerdeWithValidStatusAndFileScanTasks() { "{\"status\":\"completed\"," + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," - + "\"partition\":{\"1000\":0},\"file-size-in-bytes\":10,\"record-count\":1}]," + + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}]," + "\"file-scan-tasks\":[" + "{\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":0}," + + "\"file-format\":\"PARQUET\",\"partition\":[0]," + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + "\"delete-file-references\":[0]," + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}]" @@ -319,9 +319,7 @@ public void multipleTasksWithDifferentDeleteFilesDontAccumulateReferences() { + " \"content\" : \"POSITION_DELETES\",\n" + " \"file-path\" : \"/path/to/data-a-deletes.parquet\",\n" + " \"file-format\" : \"PARQUET\",\n" - + " \"partition\" : {\n" - + " \"1000\" : 0\n" - + " },\n" + + " \"partition\" : [ 0 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1\n" + " }, {\n" @@ -329,9 +327,7 @@ public void multipleTasksWithDifferentDeleteFilesDontAccumulateReferences() { + " \"content\" : \"POSITION_DELETES\",\n" + " \"file-path\" : \"/path/to/data-b-deletes.parquet\",\n" + " \"file-format\" : \"PARQUET\",\n" - + " \"partition\" : {\n" - + " \"1000\" : 1\n" - + " },\n" + + " \"partition\" : [ 1 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1\n" + " }, {\n" @@ -339,9 +335,7 @@ public void multipleTasksWithDifferentDeleteFilesDontAccumulateReferences() { + " \"content\" : \"EQUALITY_DELETES\",\n" + " \"file-path\" : \"/path/to/data-c-deletes.parquet\",\n" + " \"file-format\" : \"PARQUET\",\n" - + " \"partition\" : {\n" - + " \"1000\" : 2\n" - + " },\n" + + " \"partition\" : [ 2 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1,\n" + " \"equality-ids\" : [ 1 ],\n" @@ -353,9 +347,7 @@ public void multipleTasksWithDifferentDeleteFilesDontAccumulateReferences() { + " \"content\" : \"DATA\",\n" + " \"file-path\" : \"/path/to/data-a.parquet\",\n" + " \"file-format\" : \"PARQUET\",\n" - + " \"partition\" : {\n" - + " \"1000\" : 0\n" - + " },\n" + + " \"partition\" : [ 0 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1,\n" + " \"sort-order-id\" : 0\n" @@ -368,9 +360,7 @@ public void multipleTasksWithDifferentDeleteFilesDontAccumulateReferences() { + " \"content\" : \"DATA\",\n" + " \"file-path\" : \"/path/to/data-b.parquet\",\n" + " \"file-format\" : \"PARQUET\",\n" - + " \"partition\" : {\n" - + " \"1000\" : 1\n" - + " },\n" + + " \"partition\" : [ 1 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1,\n" + " \"split-offsets\" : [ 1 ],\n" @@ -384,9 +374,7 @@ public void multipleTasksWithDifferentDeleteFilesDontAccumulateReferences() { + " \"content\" : \"DATA\",\n" + " \"file-path\" : \"/path/to/data-c.parquet\",\n" + " \"file-format\" : \"PARQUET\",\n" - + " \"partition\" : {\n" - + " \"1000\" : 2\n" - + " },\n" + + " \"partition\" : [ 2 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1,\n" + " \"split-offsets\" : [ 2, 8 ],\n" @@ -422,7 +410,7 @@ public void roundTripSerdeWithoutDeleteFiles() { "{\"status\":\"completed\"," + "\"file-scan-tasks\":[" + "{\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":0}," + + "\"file-format\":\"PARQUET\",\"partition\":[0]," + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}]" + "}"; @@ -623,9 +611,7 @@ public void roundTripSerdeWithValidStatusAndFileScanTasksAndCredentials() { + " \"content\" : \"POSITION_DELETES\",\n" + " \"file-path\" : \"/path/to/data-a-deletes.parquet\",\n" + " \"file-format\" : \"PARQUET\",\n" - + " \"partition\" : {\n" - + " \"1000\" : 0\n" - + " },\n" + + " \"partition\" : [ 0 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1\n" + " } ],\n" @@ -635,9 +621,7 @@ public void roundTripSerdeWithValidStatusAndFileScanTasksAndCredentials() { + " \"content\" : \"DATA\",\n" + " \"file-path\" : \"/path/to/data-a.parquet\",\n" + " \"file-format\" : \"PARQUET\",\n" - + " \"partition\" : {\n" - + " \"1000\" : 0\n" - + " },\n" + + " \"partition\" : [ 0 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1,\n" + " \"sort-order-id\" : 0\n" From da76b873fad9b87c76eed1e91185e7b9fcb9a7f8 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Tue, 2 Dec 2025 10:49:07 -0800 Subject: [PATCH 029/201] open-api: use uv and python virtual env (#14684) * use uv * use 3.12 * pin to python 3.12 * fix github action * consolidate in make lint --- .github/workflows/open-api.yml | 10 +++------- open-api/Makefile | 12 +++++++----- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/.github/workflows/open-api.yml b/.github/workflows/open-api.yml index b20aef0fa63d..dedae02067e2 100644 --- a/.github/workflows/open-api.yml +++ b/.github/workflows/open-api.yml @@ -42,10 +42,9 @@ jobs: steps: - uses: actions/checkout@v6 - - uses: actions/setup-python@v6 - with: - python-version: 3.9 - - name: Install + - name: Install uv + uses: astral-sh/setup-uv@v7 + - name: Install dependencies working-directory: ./open-api run: make install - name: Validate REST catalog spec @@ -57,6 +56,3 @@ jobs: - name: Check if code is up to date working-directory: ./open-api run: git diff --exit-code - - name: Validate S3 REST Signer spec - working-directory: ./aws/src/main/resources - run: openapi-spec-validator s3-signer-open-api.yaml diff --git a/open-api/Makefile b/open-api/Makefile index 6bf7365f2afc..c9795ac22adb 100644 --- a/open-api/Makefile +++ b/open-api/Makefile @@ -16,15 +16,17 @@ # under the License. install: - pip install -r requirements.txt + uv venv --python 3.12 --allow-existing # Match --target-python-version in the `generate` target + uv pip install -r requirements.txt lint: - openapi-spec-validator --errors all rest-catalog-open-api.yaml + uv run openapi-spec-validator --errors all rest-catalog-open-api.yaml + uv run openapi-spec-validator --errors all ../aws/src/main/resources/s3-signer-open-api.yaml generate: - datamodel-codegen \ - --enum-field-as-literal all \ - --target-python-version 3.9 \ + uv run datamodel-codegen \ + --enum-field-as-literal all \ + --target-python-version 3.12 \ --use-schema-description \ --field-constraints \ --input rest-catalog-open-api.yaml \ From 35d66a3fe8ad76e1c76b39a898d9477f1da023ae Mon Sep 17 00:00:00 2001 From: Rulin Xing Date: Tue, 2 Dec 2025 13:06:29 -0800 Subject: [PATCH 030/201] Core: Support Custom Table/View Operations in RESTCatalog (#14465) --- .../org/apache/iceberg/rest/RESTCatalog.java | 16 +- .../iceberg/rest/RESTSessionCatalog.java | 93 +++++++++- .../apache/iceberg/rest/TestRESTCatalog.java | 173 ++++++++++++++++++ .../iceberg/rest/TestRESTViewCatalog.java | 110 +++++++++++ 4 files changed, 383 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java index 0176128ed576..aff8832c6bf4 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java @@ -69,13 +69,27 @@ public RESTCatalog(Function, RESTClient> clientBuilder) { public RESTCatalog( SessionCatalog.SessionContext context, Function, RESTClient> clientBuilder) { - this.sessionCatalog = new RESTSessionCatalog(clientBuilder, null); + this.sessionCatalog = newSessionCatalog(clientBuilder); this.delegate = sessionCatalog.asCatalog(context); this.nsDelegate = (SupportsNamespaces) delegate; this.context = context; this.viewSessionCatalog = sessionCatalog.asViewCatalog(context); } + /** + * Create a new {@link RESTSessionCatalog} instance. + * + *

This method can be overridden in subclasses to provide custom {@link RESTSessionCatalog} + * implementations. + * + * @param clientBuilder a function to build REST clients + * @return a new RESTSessionCatalog instance + */ + protected RESTSessionCatalog newSessionCatalog( + Function, RESTClient> clientBuilder) { + return new RESTSessionCatalog(clientBuilder, null); + } + @Override public void initialize(String name, Map props) { Preconditions.checkArgument(props != null, "Invalid configuration: null"); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index b903f13adc09..85b04f3868cd 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -27,6 +27,7 @@ import java.util.Set; import java.util.function.BiFunction; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.iceberg.BaseTable; import org.apache.iceberg.CatalogProperties; @@ -450,7 +451,7 @@ public Table loadTable(SessionContext context, TableIdentifier identifier) { RESTClient tableClient = client.withAuthSession(tableSession); RESTTableOperations ops = - new RESTTableOperations( + newTableOps( tableClient, paths.table(finalIdentifier), Map::of, @@ -529,7 +530,7 @@ public Table registerTable( AuthSession tableSession = authManager.tableSession(ident, tableConf, contextualSession); RESTClient tableClient = client.withAuthSession(tableSession); RESTTableOperations ops = - new RESTTableOperations( + newTableOps( tableClient, paths.table(ident), Map::of, @@ -788,7 +789,7 @@ public Table create() { AuthSession tableSession = authManager.tableSession(ident, tableConf, contextualSession); RESTClient tableClient = client.withAuthSession(tableSession); RESTTableOperations ops = - new RESTTableOperations( + newTableOps( tableClient, paths.table(ident), Map::of, @@ -815,7 +816,7 @@ public Transaction createTransaction() { RESTClient tableClient = client.withAuthSession(tableSession); RESTTableOperations ops = - new RESTTableOperations( + newTableOps( tableClient, paths.table(ident), Map::of, @@ -878,7 +879,7 @@ public Transaction replaceTransaction() { RESTClient tableClient = client.withAuthSession(tableSession); RESTTableOperations ops = - new RESTTableOperations( + newTableOps( tableClient, paths.table(ident), Map::of, @@ -1010,6 +1011,82 @@ private FileIO tableFileIO( return newFileIO(context, fullConf, storageCredentials); } + /** + * Create a new {@link RESTTableOperations} instance for simple table operations. + * + *

This method can be overridden in subclasses to provide custom table operations + * implementations. + * + * @param restClient the REST client to use for communicating with the catalog server + * @param path the REST path for the table + * @param headers a supplier for additional HTTP headers to include in requests + * @param fileIO the FileIO implementation for reading and writing table metadata and data files + * @param current the current table metadata + * @param supportedEndpoints the set of supported REST endpoints + * @return a new RESTTableOperations instance + */ + protected RESTTableOperations newTableOps( + RESTClient restClient, + String path, + Supplier> headers, + FileIO fileIO, + TableMetadata current, + Set supportedEndpoints) { + return new RESTTableOperations(restClient, path, headers, fileIO, current, supportedEndpoints); + } + + /** + * Create a new {@link RESTTableOperations} instance for transaction-based operations (create or + * replace). + * + *

This method can be overridden in subclasses to provide custom table operations + * implementations for transaction-based operations. + * + * @param restClient the REST client to use for communicating with the catalog server + * @param path the REST path for the table + * @param headers a supplier for additional HTTP headers to include in requests + * @param fileIO the FileIO implementation for reading and writing table metadata and data files + * @param updateType the {@link RESTTableOperations.UpdateType} being performed + * @param createChanges the list of metadata updates to apply during table creation or replacement + * @param current the current table metadata (may be null for CREATE operations) + * @param supportedEndpoints the set of supported REST endpoints + * @return a new RESTTableOperations instance + */ + protected RESTTableOperations newTableOps( + RESTClient restClient, + String path, + Supplier> headers, + FileIO fileIO, + RESTTableOperations.UpdateType updateType, + List createChanges, + TableMetadata current, + Set supportedEndpoints) { + return new RESTTableOperations( + restClient, path, headers, fileIO, updateType, createChanges, current, supportedEndpoints); + } + + /** + * Create a new {@link RESTViewOperations} instance. + * + *

This method can be overridden in subclasses to provide custom view operations + * implementations. + * + * @param restClient the REST client to use for communicating with the catalog server + * @param path the REST path for the view + * @param headers a supplier for additional HTTP headers to include in requests + * @param current the current view metadata + * @param supportedEndpoints the set of supported REST endpoints + * @return a new RESTViewOperations instance + */ + protected RESTViewOperations newViewOps( + RESTClient restClient, + String path, + Supplier> headers, + ViewMetadata current, + Set supportedEndpoints) { + return new RESTViewOperations(restClient, path, headers, current, supportedEndpoints); + } + private static ConfigResponse fetchConfig( RESTClient client, AuthSession initialAuth, Map properties) { // send the client's warehouse location to the service to keep in sync @@ -1154,7 +1231,7 @@ public View loadView(SessionContext context, TableIdentifier identifier) { ViewMetadata metadata = response.metadata(); RESTViewOperations ops = - new RESTViewOperations( + newViewOps( client.withAuthSession(tableSession), paths.view(identifier), Map::of, @@ -1333,7 +1410,7 @@ public View create() { Map tableConf = response.config(); AuthSession tableSession = authManager.tableSession(identifier, tableConf, contextualSession); RESTViewOperations ops = - new RESTViewOperations( + newViewOps( client.withAuthSession(tableSession), paths.view(identifier), Map::of, @@ -1424,7 +1501,7 @@ private View replace(LoadViewResponse response) { AuthSession contextualSession = authManager.contextualSession(context, catalogAuth); AuthSession tableSession = authManager.tableSession(identifier, tableConf, contextualSession); RESTViewOperations ops = - new RESTViewOperations( + newViewOps( client.withAuthSession(tableSession), paths.view(identifier), Map::of, diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index 59e91150eeae..efe76e2bf060 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -41,9 +41,15 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.UUID; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Supplier; import org.apache.hadoop.conf.Configuration; import org.apache.http.HttpHeaders; import org.apache.iceberg.BaseTable; @@ -72,6 +78,7 @@ import org.apache.iceberg.exceptions.ServiceFailureException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.io.FileIO; 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.Lists; @@ -3107,6 +3114,153 @@ public void testCommitStateUnknownNotReconciled() { .satisfies(ex -> assertThat(((CommitStateUnknownException) ex).getSuppressed()).isEmpty()); } + @Test + public void testCustomTableOperationsInjection() throws IOException { + AtomicBoolean customTableOpsCalled = new AtomicBoolean(); + AtomicBoolean customTransactionTableOpsCalled = new AtomicBoolean(); + AtomicReference capturedOps = new AtomicReference<>(); + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); + Map customHeaders = + ImmutableMap.of("X-Custom-Table-Header", "custom-value-12345"); + + // Custom RESTTableOperations that adds a custom header + class CustomRESTTableOperations extends RESTTableOperations { + CustomRESTTableOperations( + RESTClient client, + String path, + Supplier> headers, + FileIO fileIO, + TableMetadata current, + Set supportedEndpoints) { + super(client, path, () -> customHeaders, fileIO, current, supportedEndpoints); + customTableOpsCalled.set(true); + } + + CustomRESTTableOperations( + RESTClient client, + String path, + Supplier> headers, + FileIO fileIO, + RESTTableOperations.UpdateType updateType, + List createChanges, + TableMetadata current, + Set supportedEndpoints) { + super( + client, + path, + () -> customHeaders, + fileIO, + updateType, + createChanges, + current, + supportedEndpoints); + customTransactionTableOpsCalled.set(true); + } + } + + // Custom RESTSessionCatalog that overrides table operations creation + class CustomRESTSessionCatalog extends RESTSessionCatalog { + CustomRESTSessionCatalog( + Function, RESTClient> clientBuilder, + BiFunction, FileIO> ioBuilder) { + super(clientBuilder, ioBuilder); + } + + @Override + protected RESTTableOperations newTableOps( + RESTClient restClient, + String path, + Supplier> headers, + FileIO fileIO, + TableMetadata current, + Set supportedEndpoints) { + RESTTableOperations ops = + new CustomRESTTableOperations( + restClient, path, headers, fileIO, current, supportedEndpoints); + RESTTableOperations spy = Mockito.spy(ops); + capturedOps.set(spy); + return spy; + } + + @Override + protected RESTTableOperations newTableOps( + RESTClient restClient, + String path, + Supplier> headers, + FileIO fileIO, + RESTTableOperations.UpdateType updateType, + List createChanges, + TableMetadata current, + Set supportedEndpoints) { + RESTTableOperations ops = + new CustomRESTTableOperations( + restClient, + path, + headers, + fileIO, + updateType, + createChanges, + current, + supportedEndpoints); + RESTTableOperations spy = Mockito.spy(ops); + capturedOps.set(spy); + return spy; + } + } + + try (RESTCatalog catalog = + catalog(adapter, clientBuilder -> new CustomRESTSessionCatalog(clientBuilder, null))) { + catalog.createNamespace(NS); + + // Test table operations without UpdateType + assertThat(customTableOpsCalled).isFalse(); + assertThat(customTransactionTableOpsCalled).isFalse(); + Table table = catalog.createTable(TABLE, SCHEMA); + assertThat(customTableOpsCalled).isTrue(); + assertThat(customTransactionTableOpsCalled).isFalse(); + + // Trigger a commit through the custom operations + table.updateProperties().set("test-key", "test-value").commit(); + + // Verify the custom operations object was created and used + assertThat(capturedOps.get()).isNotNull(); + Mockito.verify(capturedOps.get(), Mockito.atLeastOnce()).current(); + Mockito.verify(capturedOps.get(), Mockito.atLeastOnce()).commit(any(), any()); + + // Verify the custom operations were used with custom headers + Mockito.verify(adapter, Mockito.atLeastOnce()) + .execute( + reqMatcher(HTTPMethod.POST, RESOURCE_PATHS.table(TABLE), customHeaders), + eq(LoadTableResponse.class), + any(), + any()); + + // Test table operations with UpdateType and createChanges + capturedOps.set(null); + customTableOpsCalled.set(false); + TableIdentifier table2 = TableIdentifier.of(NS, "table2"); + catalog.buildTable(table2, SCHEMA).createTransaction().commitTransaction(); + assertThat(customTableOpsCalled).isFalse(); + assertThat(customTransactionTableOpsCalled).isTrue(); + + // Trigger another commit to verify transaction operations also work + catalog.loadTable(table2).updateProperties().set("test-key-2", "test-value-2").commit(); + + // Verify the custom operations object was created and used + assertThat(capturedOps.get()).isNotNull(); + Mockito.verify(capturedOps.get(), Mockito.atLeastOnce()).current(); + Mockito.verify(capturedOps.get(), Mockito.atLeastOnce()).commit(any(), any()); + + // Verify the custom operations were used with custom headers + Mockito.verify(adapter, Mockito.atLeastOnce()) + .execute( + reqMatcher(HTTPMethod.POST, RESOURCE_PATHS.table(table2), customHeaders), + eq(LoadTableResponse.class), + any(), + any()); + } + } + private RESTCatalog catalog(RESTCatalogAdapter adapter) { RESTCatalog catalog = new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); @@ -3117,6 +3271,25 @@ private RESTCatalog catalog(RESTCatalogAdapter adapter) { return catalog; } + private RESTCatalog catalog( + RESTCatalogAdapter adapter, + Function, RESTClient>, RESTSessionCatalog> + sessionCatalogFactory) { + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter) { + @Override + protected RESTSessionCatalog newSessionCatalog( + Function, RESTClient> clientBuilder) { + return sessionCatalogFactory.apply(clientBuilder); + } + }; + catalog.initialize( + "test", + ImmutableMap.of( + CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.inmemory.InMemoryFileIO")); + return catalog; + } + static HTTPRequest reqMatcher(HTTPMethod method) { return argThat(req -> req.method() == method); } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java index f3ad68c0020a..6b39907098f9 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java @@ -32,22 +32,31 @@ import java.nio.file.Path; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiFunction; import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.Supplier; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SessionCatalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.io.FileIO; 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.Maps; import org.apache.iceberg.rest.HTTPRequest.HTTPMethod; import org.apache.iceberg.rest.responses.ConfigResponse; import org.apache.iceberg.rest.responses.ErrorResponse; import org.apache.iceberg.rest.responses.ListTablesResponse; import org.apache.iceberg.rest.responses.LoadViewResponse; import org.apache.iceberg.view.ViewCatalogTests; +import org.apache.iceberg.view.ViewMetadata; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.gzip.GzipHandler; import org.eclipse.jetty.servlet.ServletContextHandler; @@ -308,6 +317,107 @@ public void viewExistsFallbackToGETRequestWithLegacyServer() { ImmutableMap.of(RESTCatalogProperties.VIEW_ENDPOINTS_SUPPORTED, "true")); } + @Test + public void testCustomViewOperationsInjection() throws Exception { + AtomicBoolean customViewOpsCalled = new AtomicBoolean(); + AtomicReference capturedOps = new AtomicReference<>(); + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); + Map customHeaders = + ImmutableMap.of("X-Custom-View-Header", "custom-value-12345"); + + // Custom RESTViewOperations that adds a custom header + class CustomRESTViewOperations extends RESTViewOperations { + CustomRESTViewOperations( + RESTClient client, + String path, + Supplier> headers, + ViewMetadata current, + Set supportedEndpoints) { + super(client, path, () -> customHeaders, current, supportedEndpoints); + customViewOpsCalled.set(true); + } + } + + // Custom RESTSessionCatalog that overrides view operations creation + class CustomRESTSessionCatalog extends RESTSessionCatalog { + CustomRESTSessionCatalog( + Function, RESTClient> clientBuilder, + BiFunction, FileIO> ioBuilder) { + super(clientBuilder, ioBuilder); + } + + @Override + protected RESTViewOperations newViewOps( + RESTClient restClient, + String path, + Supplier> headers, + ViewMetadata current, + Set supportedEndpoints) { + RESTViewOperations ops = + new CustomRESTViewOperations(restClient, path, headers, current, supportedEndpoints); + RESTViewOperations spy = Mockito.spy(ops); + capturedOps.set(spy); + return spy; + } + } + + try (RESTCatalog catalog = + catalog(adapter, clientBuilder -> new CustomRESTSessionCatalog(clientBuilder, null))) { + Namespace namespace = Namespace.of("ns"); + catalog.createNamespace(namespace); + + // Test view operations + assertThat(customViewOpsCalled).isFalse(); + TableIdentifier viewIdentifier = TableIdentifier.of(namespace, "view1"); + org.apache.iceberg.view.View view = + catalog + .buildView(viewIdentifier) + .withSchema(SCHEMA) + .withDefaultNamespace(namespace) + .withQuery("spark", "select * from ns.table") + .create(); + + // Verify custom operations was created + assertThat(customViewOpsCalled).isTrue(); + + // Update view properties to trigger a commit through the custom operations + view.updateProperties().set("test-key", "test-value").commit(); + + // Verify the custom operations object was created and used + assertThat(capturedOps.get()).isNotNull(); + Mockito.verify(capturedOps.get(), Mockito.atLeastOnce()).current(); + Mockito.verify(capturedOps.get(), Mockito.atLeastOnce()).commit(any(), any()); + + // Verify the custom operations were used with custom headers + ResourcePaths resourcePaths = ResourcePaths.forCatalogProperties(Maps.newHashMap()); + Mockito.verify(adapter, Mockito.atLeastOnce()) + .execute( + reqMatcher(HTTPMethod.POST, resourcePaths.view(viewIdentifier), customHeaders), + eq(LoadViewResponse.class), + any(), + any()); + } + } + + private RESTCatalog catalog( + RESTCatalogAdapter adapter, + Function, RESTClient>, RESTSessionCatalog> + sessionCatalogFactory) { + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter) { + @Override + protected RESTSessionCatalog newSessionCatalog( + Function, RESTClient> clientBuilder) { + return sessionCatalogFactory.apply(clientBuilder); + } + }; + catalog.initialize( + "test", + ImmutableMap.of( + CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.inmemory.InMemoryFileIO")); + return catalog; + } + @Override protected RESTCatalog catalog() { return restCatalog; From fac485c56b6e24b9081fed9e001077632945613c Mon Sep 17 00:00:00 2001 From: jbewing Date: Wed, 3 Dec 2025 01:19:41 -0500 Subject: [PATCH 031/201] Spark: Analyze but don't optimize view body during creation (#14681) --- .../sql/catalyst/analysis/CheckViews.scala | 1 + .../sql/catalyst/analysis/ResolveViews.scala | 1 + .../logical/views/CreateIcebergView.scala | 24 ++++++++++++------- .../v2/ExtendedDataSourceV2Strategy.scala | 1 + .../sql/catalyst/analysis/CheckViews.scala | 1 + .../sql/catalyst/analysis/ResolveViews.scala | 1 + .../logical/views/CreateIcebergView.scala | 24 ++++++++++++------- .../v2/ExtendedDataSourceV2Strategy.scala | 1 + .../sql/catalyst/analysis/CheckViews.scala | 1 + .../sql/catalyst/analysis/ResolveViews.scala | 1 + .../logical/views/CreateIcebergView.scala | 24 ++++++++++++------- .../v2/ExtendedDataSourceV2Strategy.scala | 1 + 12 files changed, 57 insertions(+), 24 deletions(-) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala index a7c903dfdbc4..549aefaae28f 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -48,6 +48,7 @@ object CheckViews extends (LogicalPlan => Unit) { _, _, replace, + _, _) => verifyColumnCount(resolvedIdent, columnAliases, query) SchemaUtils.checkColumnNameDuplication( diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala index 1e0e5e8951ba..4f7e2b4d0f24 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala @@ -73,6 +73,7 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look _, _, _, + _, _) if query.resolved && !c.rewritten => val aliased = aliasColumns(query, columnAliases, columnComments) c.copy( diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala index c5dadae70110..84a00a4a9a88 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala @@ -18,9 +18,12 @@ */ package org.apache.spark.sql.catalyst.plans.logical.views -import org.apache.spark.sql.catalyst.plans.logical.BinaryCommand +import org.apache.spark.sql.catalyst.analysis.AnalysisContext +import org.apache.spark.sql.catalyst.plans.logical.AnalysisOnlyCommand import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +// Align Iceberg's CreateIcebergView with Spark’s CreateViewCommand by extending AnalysisOnlyCommand. +// The command’s children are analyzed then hidden, so the optimizer/planner won’t traverse the view body. case class CreateIcebergView( child: LogicalPlan, queryText: String, @@ -32,14 +35,19 @@ case class CreateIcebergView( properties: Map[String, String], allowExisting: Boolean, replace: Boolean, - rewritten: Boolean = false) - extends BinaryCommand { - override def left: LogicalPlan = child + rewritten: Boolean = false, + isAnalyzed: Boolean = false) + extends AnalysisOnlyCommand { - override def right: LogicalPlan = query + override def childrenToAnalyze: Seq[LogicalPlan] = child :: query :: Nil + + override def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan = { + copy(isAnalyzed = true) + } override protected def withNewChildrenInternal( - newLeft: LogicalPlan, - newRight: LogicalPlan): LogicalPlan = - copy(child = newLeft, query = newRight) + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { + assert(!isAnalyzed) + copy(child = newChildren.head, query = newChildren.last) + } } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala index afdddb7fab4e..53def0e95250 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala @@ -198,6 +198,7 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi properties, allowExisting, replace, + _, _) => CreateV2ViewExec( catalog = viewCatalog, diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala index aefc672dc878..319ab78a5348 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -48,6 +48,7 @@ object CheckViews extends (LogicalPlan => Unit) { _, _, replace, + _, _) => verifyColumnCount(resolvedIdent, columnAliases, query) SchemaUtils.checkColumnNameDuplication( diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala index 1e0e5e8951ba..4f7e2b4d0f24 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala @@ -73,6 +73,7 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look _, _, _, + _, _) if query.resolved && !c.rewritten => val aliased = aliasColumns(query, columnAliases, columnComments) c.copy( diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala index c5dadae70110..84a00a4a9a88 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala @@ -18,9 +18,12 @@ */ package org.apache.spark.sql.catalyst.plans.logical.views -import org.apache.spark.sql.catalyst.plans.logical.BinaryCommand +import org.apache.spark.sql.catalyst.analysis.AnalysisContext +import org.apache.spark.sql.catalyst.plans.logical.AnalysisOnlyCommand import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +// Align Iceberg's CreateIcebergView with Spark’s CreateViewCommand by extending AnalysisOnlyCommand. +// The command’s children are analyzed then hidden, so the optimizer/planner won’t traverse the view body. case class CreateIcebergView( child: LogicalPlan, queryText: String, @@ -32,14 +35,19 @@ case class CreateIcebergView( properties: Map[String, String], allowExisting: Boolean, replace: Boolean, - rewritten: Boolean = false) - extends BinaryCommand { - override def left: LogicalPlan = child + rewritten: Boolean = false, + isAnalyzed: Boolean = false) + extends AnalysisOnlyCommand { - override def right: LogicalPlan = query + override def childrenToAnalyze: Seq[LogicalPlan] = child :: query :: Nil + + override def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan = { + copy(isAnalyzed = true) + } override protected def withNewChildrenInternal( - newLeft: LogicalPlan, - newRight: LogicalPlan): LogicalPlan = - copy(child = newLeft, query = newRight) + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { + assert(!isAnalyzed) + copy(child = newChildren.head, query = newChildren.last) + } } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala index 64b228a4247b..6b340b72496e 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala @@ -148,6 +148,7 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi properties, allowExisting, replace, + _, _) => CreateV2ViewExec( catalog = viewCatalog, diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala index 343fbe7a91b6..5ad4b9c01409 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -48,6 +48,7 @@ object CheckViews extends (LogicalPlan => Unit) { _, _, replace, + _, _) => verifyColumnCount(resolvedIdent, columnAliases, query) SchemaUtils.checkColumnNameDuplication( diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala index 23d33352f134..ff7d20241bed 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala @@ -73,6 +73,7 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look _, _, _, + _, _) if query.resolved && !c.rewritten => val aliased = aliasColumns(query, columnAliases, columnComments) c.copy( diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala index c5dadae70110..84a00a4a9a88 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala @@ -18,9 +18,12 @@ */ package org.apache.spark.sql.catalyst.plans.logical.views -import org.apache.spark.sql.catalyst.plans.logical.BinaryCommand +import org.apache.spark.sql.catalyst.analysis.AnalysisContext +import org.apache.spark.sql.catalyst.plans.logical.AnalysisOnlyCommand import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +// Align Iceberg's CreateIcebergView with Spark’s CreateViewCommand by extending AnalysisOnlyCommand. +// The command’s children are analyzed then hidden, so the optimizer/planner won’t traverse the view body. case class CreateIcebergView( child: LogicalPlan, queryText: String, @@ -32,14 +35,19 @@ case class CreateIcebergView( properties: Map[String, String], allowExisting: Boolean, replace: Boolean, - rewritten: Boolean = false) - extends BinaryCommand { - override def left: LogicalPlan = child + rewritten: Boolean = false, + isAnalyzed: Boolean = false) + extends AnalysisOnlyCommand { - override def right: LogicalPlan = query + override def childrenToAnalyze: Seq[LogicalPlan] = child :: query :: Nil + + override def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan = { + copy(isAnalyzed = true) + } override protected def withNewChildrenInternal( - newLeft: LogicalPlan, - newRight: LogicalPlan): LogicalPlan = - copy(child = newLeft, query = newRight) + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { + assert(!isAnalyzed) + copy(child = newChildren.head, query = newChildren.last) + } } diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala index 898e6f3d41dc..da540f5891b7 100644 --- a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala @@ -140,6 +140,7 @@ case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy wi properties, allowExisting, replace, + _, _) => CreateV2ViewExec( catalog = viewCatalog, From 74a11607435ec332e557323b78c51726a7ef2fdf Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 3 Dec 2025 07:32:55 +0100 Subject: [PATCH 032/201] Spark 4.0, Core: Add Limit pushdown to Scan (#14615) --- .../org/apache/iceberg/BatchScanAdapter.java | 5 + .../main/java/org/apache/iceberg/Scan.java | 13 +++ .../java/org/apache/iceberg/BaseScan.java | 5 + .../org/apache/iceberg/TableScanContext.java | 7 ++ .../spark/source/SparkScanBuilder.java | 15 ++- .../spark/source/TestFilteredScan.java | 94 +++++++++++++++++++ .../apache/iceberg/spark/sql/TestSelect.java | 13 +++ 7 files changed, 151 insertions(+), 1 deletion(-) diff --git a/api/src/main/java/org/apache/iceberg/BatchScanAdapter.java b/api/src/main/java/org/apache/iceberg/BatchScanAdapter.java index 02b3d241d893..d8c5dda88558 100644 --- a/api/src/main/java/org/apache/iceberg/BatchScanAdapter.java +++ b/api/src/main/java/org/apache/iceberg/BatchScanAdapter.java @@ -151,4 +151,9 @@ public long splitOpenFileCost() { public BatchScan metricsReporter(MetricsReporter reporter) { return new BatchScanAdapter(scan.metricsReporter(reporter)); } + + @Override + public BatchScan minRowsRequested(long numRows) { + return new BatchScanAdapter(scan.minRowsRequested(numRows)); + } } diff --git a/api/src/main/java/org/apache/iceberg/Scan.java b/api/src/main/java/org/apache/iceberg/Scan.java index 339bc75336ba..9785ce6603ad 100644 --- a/api/src/main/java/org/apache/iceberg/Scan.java +++ b/api/src/main/java/org/apache/iceberg/Scan.java @@ -195,4 +195,17 @@ default ThisT metricsReporter(MetricsReporter reporter) { throw new UnsupportedOperationException( this.getClass().getName() + " doesn't implement metricsReporter"); } + + /** + * Create a new scan that returns files with at least the given number of rows. This is used as a + * hint and is entirely optional in order to not have to return more rows than necessary. This may + * return fewer rows if the scan does not contain that many, or it may return more than requested. + * + * @param numRows The minimum number of rows requested + * @return A new scan based on this with at least the given number of rows + */ + default ThisT minRowsRequested(long numRows) { + throw new UnsupportedOperationException( + this.getClass().getName() + " doesn't implement minRowsRequested"); + } } diff --git a/core/src/main/java/org/apache/iceberg/BaseScan.java b/core/src/main/java/org/apache/iceberg/BaseScan.java index 618b2e95f29f..e40b1b743c76 100644 --- a/core/src/main/java/org/apache/iceberg/BaseScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseScan.java @@ -293,6 +293,11 @@ public ThisT metricsReporter(MetricsReporter reporter) { return newRefinedScan(table, schema, context.reportWith(reporter)); } + @Override + public ThisT minRowsRequested(long numRows) { + return newRefinedScan(table, schema, context.minRowsRequested(numRows)); + } + /** * Retrieves a list of column names based on the type of manifest content provided. * diff --git a/core/src/main/java/org/apache/iceberg/TableScanContext.java b/core/src/main/java/org/apache/iceberg/TableScanContext.java index 5722ed7d8c1c..faa1c264d5f1 100644 --- a/core/src/main/java/org/apache/iceberg/TableScanContext.java +++ b/core/src/main/java/org/apache/iceberg/TableScanContext.java @@ -103,6 +103,9 @@ public MetricsReporter metricsReporter() { @Nullable public abstract String branch(); + @Nullable + public abstract Long minRowsRequested(); + TableScanContext useSnapshotId(Long scanSnapshotId) { return ImmutableTableScanContext.builder().from(this).snapshotId(scanSnapshotId).build(); } @@ -193,6 +196,10 @@ TableScanContext useBranch(String ref) { return ImmutableTableScanContext.builder().from(this).branch(ref).build(); } + TableScanContext minRowsRequested(long numRows) { + return ImmutableTableScanContext.builder().from(this).minRowsRequested(numRows).build(); + } + public static TableScanContext empty() { return ImmutableTableScanContext.builder().build(); } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 968e6eeaaa45..11b0ba58af51 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -71,6 +71,7 @@ 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; @@ -85,7 +86,8 @@ public class SparkScanBuilder SupportsPushDownAggregates, SupportsPushDownV2Filters, SupportsPushDownRequiredColumns, - SupportsReportStatistics { + SupportsReportStatistics, + SupportsPushDownLimit { private static final Logger LOG = LoggerFactory.getLogger(SparkScanBuilder.class); private static final Predicate[] NO_PREDICATES = new Predicate[0]; @@ -102,6 +104,7 @@ public class SparkScanBuilder private boolean caseSensitive; private List filterExpressions = null; private Predicate[] pushedPredicates = NO_PREDICATES; + private Integer limit = null; SparkScanBuilder( SparkSession spark, @@ -739,6 +742,10 @@ public Scan buildCopyOnWriteScan() { TableProperties.SPLIT_OPEN_FILE_COST, String.valueOf(splitOpenFileCost)); } + if (null != limit) { + configuredScan = configuredScan.minRowsRequested(limit.longValue()); + } + return configuredScan; } @@ -759,4 +766,10 @@ private BatchScan newBatchScan() { return table.newBatchScan(); } } + + @Override + public boolean pushLimit(int pushedLimit) { + this.limit = pushedLimit; + return true; + } } diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java index 8d7053efb1e0..61d6501a6847 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java @@ -23,6 +23,7 @@ import static org.apache.iceberg.PlanningMode.LOCAL; import static org.apache.iceberg.data.FileHelpers.encrypt; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -73,6 +74,7 @@ import org.apache.spark.sql.sources.Not; import org.apache.spark.sql.sources.StringStartsWith; import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.assertj.core.api.AbstractObjectAssert; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -267,6 +269,98 @@ public void testUnpartitionedTimestampFilter() { "ts < cast('2017-12-22 00:00:00+00:00' as timestamp)")); } + @TestTemplate + public void limitPushedDownToSparkScan() { + assumeThat(fileFormat) + .as("no need to run this across the entire test matrix") + .isEqualTo(FileFormat.PARQUET); + + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); + + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + long limit = 23; + // simulate Spark pushing down the limit to the scan builder + builder.pushLimit((int) limit); + assertThat(builder).extracting("limit").isEqualTo((int) limit); + + // verify batch scan + AbstractObjectAssert scanAssert = assertThat(builder.build()).extracting("scan"); + if (LOCAL == planningMode) { + scanAssert = scanAssert.extracting("scan"); + } + + scanAssert.extracting("context").extracting("minRowsRequested").isEqualTo(limit); + + // verify changelog scan + assertThat(builder.buildChangelogScan()) + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify CoW scan + assertThat(builder.buildCopyOnWriteScan()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify MoR scan + scanAssert = assertThat(builder.buildMergeOnReadScan()).extracting("scan"); + if (LOCAL == planningMode) { + scanAssert = scanAssert.extracting("scan"); + } + + scanAssert.extracting("context").extracting("minRowsRequested").isEqualTo(limit); + } + + @TestTemplate + public void limitPushedDownToSparkScanForMetadataTable() { + assumeThat(fileFormat) + .as("no need to run this across the entire test matrix") + .isEqualTo(FileFormat.PARQUET); + + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); + + // load the snapshots metadata table + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path") + "#snapshots"), options); + + long limit = 23; + // simulate Spark pushing down the limit to the scan builder + builder.pushLimit((int) limit); + assertThat(builder).extracting("limit").isEqualTo((int) limit); + + // verify batch scan + assertThat(builder.build()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify CoW scan + assertThat(builder.buildCopyOnWriteScan()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify MoR scan + assertThat(builder.buildMergeOnReadScan()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + } + @TestTemplate public void testBucketPartitionedIDFilters() { Table table = buildPartitionedTable("bucketed_by_id", BUCKET_BY_ID); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 68b93be47960..8fa8406278ea 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -150,6 +150,19 @@ public void testSelectRewrite() { .isEqualTo("(float IS NOT NULL AND is_nan(float))"); } + @TestTemplate + public void selectWithLimit() { + Object[] first = row(1L, "a", 1.0F); + Object[] second = row(2L, "b", 2.0F); + Object[] third = row(3L, "c", Float.NaN); + + // verify that LIMIT is properly applied in case SupportsPushDownLimit.isPartiallyPushed() is + // ever overridden in SparkScanBuilder + assertThat(sql("SELECT * FROM %s LIMIT 1", tableName)).containsExactly(first); + assertThat(sql("SELECT * FROM %s LIMIT 2", tableName)).containsExactly(first, second); + assertThat(sql("SELECT * FROM %s LIMIT 3", tableName)).containsExactly(first, second, third); + } + @TestTemplate public void testProjection() { List expected = ImmutableList.of(row(1L), row(2L), row(3L)); From fc7cd7db00132b32cfd75f609d98916d83032209 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 3 Dec 2025 11:21:10 +0100 Subject: [PATCH 033/201] Spark 3.4,3.5: Add LIMIT pushdown to Scan (#14741) --- .../spark/source/SparkScanBuilder.java | 15 ++- .../spark/source/TestFilteredScan.java | 94 +++++++++++++++++++ .../apache/iceberg/spark/sql/TestSelect.java | 13 +++ .../spark/source/SparkScanBuilder.java | 15 ++- .../spark/source/TestFilteredScan.java | 94 +++++++++++++++++++ .../apache/iceberg/spark/sql/TestSelect.java | 13 +++ 6 files changed, 242 insertions(+), 2 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 968e6eeaaa45..11b0ba58af51 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -71,6 +71,7 @@ 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; @@ -85,7 +86,8 @@ public class SparkScanBuilder SupportsPushDownAggregates, SupportsPushDownV2Filters, SupportsPushDownRequiredColumns, - SupportsReportStatistics { + SupportsReportStatistics, + SupportsPushDownLimit { private static final Logger LOG = LoggerFactory.getLogger(SparkScanBuilder.class); private static final Predicate[] NO_PREDICATES = new Predicate[0]; @@ -102,6 +104,7 @@ public class SparkScanBuilder private boolean caseSensitive; private List filterExpressions = null; private Predicate[] pushedPredicates = NO_PREDICATES; + private Integer limit = null; SparkScanBuilder( SparkSession spark, @@ -739,6 +742,10 @@ public Scan buildCopyOnWriteScan() { TableProperties.SPLIT_OPEN_FILE_COST, String.valueOf(splitOpenFileCost)); } + if (null != limit) { + configuredScan = configuredScan.minRowsRequested(limit.longValue()); + } + return configuredScan; } @@ -759,4 +766,10 @@ private BatchScan newBatchScan() { return table.newBatchScan(); } } + + @Override + public boolean pushLimit(int pushedLimit) { + this.limit = pushedLimit; + return true; + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java index e30f7e44af25..308b1bd2c646 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java @@ -26,6 +26,7 @@ import static org.apache.spark.sql.functions.callUDF; import static org.apache.spark.sql.functions.column; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -82,6 +83,7 @@ import org.apache.spark.sql.types.LongType$; import org.apache.spark.sql.types.StringType$; import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.assertj.core.api.AbstractObjectAssert; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -299,6 +301,98 @@ public void testUnpartitionedTimestampFilter() { "ts < cast('2017-12-22 00:00:00+00:00' as timestamp)")); } + @TestTemplate + public void limitPushedDownToSparkScan() { + assumeThat(fileFormat) + .as("no need to run this across the entire test matrix") + .isEqualTo(FileFormat.PARQUET); + + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); + + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + long limit = 23; + // simulate Spark pushing down the limit to the scan builder + builder.pushLimit((int) limit); + assertThat(builder).extracting("limit").isEqualTo((int) limit); + + // verify batch scan + AbstractObjectAssert scanAssert = assertThat(builder.build()).extracting("scan"); + if (LOCAL == planningMode) { + scanAssert = scanAssert.extracting("scan"); + } + + scanAssert.extracting("context").extracting("minRowsRequested").isEqualTo(limit); + + // verify changelog scan + assertThat(builder.buildChangelogScan()) + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify CoW scan + assertThat(builder.buildCopyOnWriteScan()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify MoR scan + scanAssert = assertThat(builder.buildMergeOnReadScan()).extracting("scan"); + if (LOCAL == planningMode) { + scanAssert = scanAssert.extracting("scan"); + } + + scanAssert.extracting("context").extracting("minRowsRequested").isEqualTo(limit); + } + + @TestTemplate + public void limitPushedDownToSparkScanForMetadataTable() { + assumeThat(fileFormat) + .as("no need to run this across the entire test matrix") + .isEqualTo(FileFormat.PARQUET); + + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); + + // load the snapshots metadata table + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path") + "#snapshots"), options); + + long limit = 23; + // simulate Spark pushing down the limit to the scan builder + builder.pushLimit((int) limit); + assertThat(builder).extracting("limit").isEqualTo((int) limit); + + // verify batch scan + assertThat(builder.build()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify CoW scan + assertThat(builder.buildCopyOnWriteScan()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify MoR scan + assertThat(builder.buildMergeOnReadScan()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + } + @TestTemplate public void testBucketPartitionedIDFilters() { Table table = buildPartitionedTable("bucketed_by_id", BUCKET_BY_ID, "bucket4", "id"); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index a21dfd388d1b..d4187a266b94 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -150,6 +150,19 @@ public void testSelectRewrite() { .isEqualTo("(float IS NOT NULL AND is_nan(float))"); } + @TestTemplate + public void selectWithLimit() { + Object[] first = row(1L, "a", 1.0F); + Object[] second = row(2L, "b", 2.0F); + Object[] third = row(3L, "c", Float.NaN); + + // verify that LIMIT is properly applied in case SupportsPushDownLimit.isPartiallyPushed() is + // ever overridden in SparkScanBuilder + assertThat(sql("SELECT * FROM %s LIMIT 1", tableName)).containsExactly(first); + assertThat(sql("SELECT * FROM %s LIMIT 2", tableName)).containsExactly(first, second); + assertThat(sql("SELECT * FROM %s LIMIT 3", tableName)).containsExactly(first, second, third); + } + @TestTemplate public void testProjection() { List expected = ImmutableList.of(row(1L), row(2L), row(3L)); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 968e6eeaaa45..11b0ba58af51 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -71,6 +71,7 @@ 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; @@ -85,7 +86,8 @@ public class SparkScanBuilder SupportsPushDownAggregates, SupportsPushDownV2Filters, SupportsPushDownRequiredColumns, - SupportsReportStatistics { + SupportsReportStatistics, + SupportsPushDownLimit { private static final Logger LOG = LoggerFactory.getLogger(SparkScanBuilder.class); private static final Predicate[] NO_PREDICATES = new Predicate[0]; @@ -102,6 +104,7 @@ public class SparkScanBuilder private boolean caseSensitive; private List filterExpressions = null; private Predicate[] pushedPredicates = NO_PREDICATES; + private Integer limit = null; SparkScanBuilder( SparkSession spark, @@ -739,6 +742,10 @@ public Scan buildCopyOnWriteScan() { TableProperties.SPLIT_OPEN_FILE_COST, String.valueOf(splitOpenFileCost)); } + if (null != limit) { + configuredScan = configuredScan.minRowsRequested(limit.longValue()); + } + return configuredScan; } @@ -759,4 +766,10 @@ private BatchScan newBatchScan() { return table.newBatchScan(); } } + + @Override + public boolean pushLimit(int pushedLimit) { + this.limit = pushedLimit; + return true; + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java index 8d7053efb1e0..61d6501a6847 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java @@ -23,6 +23,7 @@ import static org.apache.iceberg.PlanningMode.LOCAL; import static org.apache.iceberg.data.FileHelpers.encrypt; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -73,6 +74,7 @@ import org.apache.spark.sql.sources.Not; import org.apache.spark.sql.sources.StringStartsWith; import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.assertj.core.api.AbstractObjectAssert; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -267,6 +269,98 @@ public void testUnpartitionedTimestampFilter() { "ts < cast('2017-12-22 00:00:00+00:00' as timestamp)")); } + @TestTemplate + public void limitPushedDownToSparkScan() { + assumeThat(fileFormat) + .as("no need to run this across the entire test matrix") + .isEqualTo(FileFormat.PARQUET); + + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); + + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + long limit = 23; + // simulate Spark pushing down the limit to the scan builder + builder.pushLimit((int) limit); + assertThat(builder).extracting("limit").isEqualTo((int) limit); + + // verify batch scan + AbstractObjectAssert scanAssert = assertThat(builder.build()).extracting("scan"); + if (LOCAL == planningMode) { + scanAssert = scanAssert.extracting("scan"); + } + + scanAssert.extracting("context").extracting("minRowsRequested").isEqualTo(limit); + + // verify changelog scan + assertThat(builder.buildChangelogScan()) + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify CoW scan + assertThat(builder.buildCopyOnWriteScan()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify MoR scan + scanAssert = assertThat(builder.buildMergeOnReadScan()).extracting("scan"); + if (LOCAL == planningMode) { + scanAssert = scanAssert.extracting("scan"); + } + + scanAssert.extracting("context").extracting("minRowsRequested").isEqualTo(limit); + } + + @TestTemplate + public void limitPushedDownToSparkScanForMetadataTable() { + assumeThat(fileFormat) + .as("no need to run this across the entire test matrix") + .isEqualTo(FileFormat.PARQUET); + + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); + + // load the snapshots metadata table + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path") + "#snapshots"), options); + + long limit = 23; + // simulate Spark pushing down the limit to the scan builder + builder.pushLimit((int) limit); + assertThat(builder).extracting("limit").isEqualTo((int) limit); + + // verify batch scan + assertThat(builder.build()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify CoW scan + assertThat(builder.buildCopyOnWriteScan()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify MoR scan + assertThat(builder.buildMergeOnReadScan()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + } + @TestTemplate public void testBucketPartitionedIDFilters() { Table table = buildPartitionedTable("bucketed_by_id", BUCKET_BY_ID); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 68b93be47960..8fa8406278ea 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -150,6 +150,19 @@ public void testSelectRewrite() { .isEqualTo("(float IS NOT NULL AND is_nan(float))"); } + @TestTemplate + public void selectWithLimit() { + Object[] first = row(1L, "a", 1.0F); + Object[] second = row(2L, "b", 2.0F); + Object[] third = row(3L, "c", Float.NaN); + + // verify that LIMIT is properly applied in case SupportsPushDownLimit.isPartiallyPushed() is + // ever overridden in SparkScanBuilder + assertThat(sql("SELECT * FROM %s LIMIT 1", tableName)).containsExactly(first); + assertThat(sql("SELECT * FROM %s LIMIT 2", tableName)).containsExactly(first, second); + assertThat(sql("SELECT * FROM %s LIMIT 3", tableName)).containsExactly(first, second, third); + } + @TestTemplate public void testProjection() { List expected = ImmutableList.of(row(1L), row(2L), row(3L)); From bfe06a540d9d31d89b5161b20dcdcf6b1b6e2ef4 Mon Sep 17 00:00:00 2001 From: pvary Date: Wed, 3 Dec 2025 14:03:40 +0100 Subject: [PATCH 034/201] Spark: Move DeleteFiltering out from the vectorized reader (#14652) --- .../checkstyle/checkstyle-suppressions.xml | 2 + .../data/vectorized/ColumnarBatchReader.java | 57 +------ .../vectorized/CometColumnarBatchReader.java | 64 +------- .../vectorized/CometDeleteColumnReader.java | 27 ++- .../vectorized/CometDeletedColumnVector.java | 155 ++++++++++++++++++ .../CometVectorizedReaderBuilder.java | 13 +- .../data/vectorized/DeletedColumnVector.java | 3 +- .../UpdatableDeletedColumnVector.java | 23 +++ .../VectorizedSparkParquetReaders.java | 32 +--- .../iceberg/spark/source/BaseBatchReader.java | 101 ++++++++++-- .../iceberg/spark/source/BatchDataReader.java | 8 +- .../TestSparkParquetReadMetadataColumns.java | 71 ++++++-- ...rquetDictionaryEncodedVectorizedReads.java | 2 +- .../parquet/TestParquetVectorizedReads.java | 5 +- .../iceberg/spark/source/BatchReaderUtil.java | 34 ++++ 15 files changed, 391 insertions(+), 206 deletions(-) create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java diff --git a/.baseline/checkstyle/checkstyle-suppressions.xml b/.baseline/checkstyle/checkstyle-suppressions.xml index 1f180e40a123..1e79b1a7aa4b 100644 --- a/.baseline/checkstyle/checkstyle-suppressions.xml +++ b/.baseline/checkstyle/checkstyle-suppressions.xml @@ -55,6 +55,8 @@ + + diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java index a0670c2bbe71..38d505d250d0 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java @@ -22,15 +22,11 @@ import java.util.Map; import org.apache.iceberg.arrow.vectorized.BaseBatchReader; import org.apache.iceberg.arrow.vectorized.VectorizedArrowReader; -import org.apache.iceberg.arrow.vectorized.VectorizedArrowReader.DeletedVectorReader; -import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.util.Pair; import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; @@ -40,31 +36,15 @@ * populated via delegated read calls to {@linkplain VectorizedArrowReader VectorReader(s)}. */ public class ColumnarBatchReader extends BaseBatchReader { - private final boolean hasIsDeletedColumn; - private DeleteFilter deletes = null; - private long rowStartPosInBatch = 0; public ColumnarBatchReader(List> readers) { super(readers); - this.hasIsDeletedColumn = - readers.stream().anyMatch(reader -> reader instanceof DeletedVectorReader); } @Override public void setRowGroupInfo( PageReadStore pageStore, Map metaData) { super.setRowGroupInfo(pageStore, metaData); - this.rowStartPosInBatch = - pageStore - .getRowIndexOffset() - .orElseThrow( - () -> - new IllegalArgumentException( - "PageReadStore does not contain row index offset")); - } - - public void setDeleteFilter(DeleteFilter deleteFilter) { - this.deletes = deleteFilter; } @Override @@ -73,9 +53,7 @@ public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) { closeVectors(); } - ColumnarBatch columnarBatch = new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); - rowStartPosInBatch += numRowsToRead; - return columnarBatch; + return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); } private class ColumnBatchLoader { @@ -89,43 +67,12 @@ private class ColumnBatchLoader { ColumnarBatch loadDataToColumnBatch() { ColumnVector[] vectors = readDataToColumnVectors(); - int numLiveRows = batchSize; - - if (hasIsDeletedColumn) { - boolean[] isDeleted = buildIsDeleted(vectors); - for (ColumnVector vector : vectors) { - if (vector instanceof DeletedColumnVector) { - ((DeletedColumnVector) vector).setValue(isDeleted); - } - } - } else { - Pair pair = buildRowIdMapping(vectors); - if (pair != null) { - int[] rowIdMapping = pair.first(); - numLiveRows = pair.second(); - for (int i = 0; i < vectors.length; i++) { - vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping); - } - } - } - - if (deletes != null && deletes.hasEqDeletes()) { - vectors = ColumnarBatchUtil.removeExtraColumns(deletes, vectors); - } ColumnarBatch batch = new ColumnarBatch(vectors); - batch.setNumRows(numLiveRows); + batch.setNumRows(batchSize); return batch; } - private boolean[] buildIsDeleted(ColumnVector[] vectors) { - return ColumnarBatchUtil.buildIsDeleted(vectors, deletes, rowStartPosInBatch, batchSize); - } - - private Pair buildRowIdMapping(ColumnVector[] vectors) { - return ColumnarBatchUtil.buildRowIdMapping(vectors, deletes, rowStartPosInBatch, batchSize); - } - ColumnVector[] readDataToColumnVectors() { ColumnVector[] arrowColumnVectors = new ColumnVector[readers.length]; diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java index 04ac69476add..3d3e9aca24de 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java @@ -25,15 +25,12 @@ import org.apache.comet.parquet.AbstractColumnReader; import org.apache.comet.parquet.BatchReader; import org.apache.iceberg.Schema; -import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.util.Pair; import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; @@ -46,7 +43,6 @@ class CometColumnarBatchReader implements VectorizedReader { private final CometColumnReader[] readers; - private final boolean hasIsDeletedColumn; // The delegated BatchReader on the Comet side does the real work of loading a batch of rows. // The Comet BatchReader contains an array of ColumnReader. There is no need to explicitly call @@ -56,14 +52,10 @@ class CometColumnarBatchReader implements VectorizedReader { // DeleteColumnReader.readBatch must be called explicitly later, after the isDeleted value is // available. private final BatchReader delegate; - private DeleteFilter deletes = null; - private long rowStartPosInBatch = 0; CometColumnarBatchReader(List> readers, Schema schema) { this.readers = readers.stream().map(CometColumnReader.class::cast).toArray(CometColumnReader[]::new); - this.hasIsDeletedColumn = - readers.stream().anyMatch(reader -> reader instanceof CometDeleteColumnReader); AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; this.delegate = new BatchReader(abstractColumnReaders); @@ -89,25 +81,11 @@ public void setRowGroupInfo( for (int i = 0; i < readers.length; i++) { delegate.getColumnReaders()[i] = this.readers[i].delegate(); } - - this.rowStartPosInBatch = - pageStore - .getRowIndexOffset() - .orElseThrow( - () -> - new IllegalArgumentException( - "PageReadStore does not contain row index offset")); - } - - public void setDeleteFilter(DeleteFilter deleteFilter) { - this.deletes = deleteFilter; } @Override public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) { - ColumnarBatch columnarBatch = new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); - rowStartPosInBatch += numRowsToRead; - return columnarBatch; + return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); } @Override @@ -139,39 +117,12 @@ private class ColumnBatchLoader { ColumnarBatch loadDataToColumnBatch() { ColumnVector[] vectors = readDataToColumnVectors(); - int numLiveRows = batchSize; - - if (hasIsDeletedColumn) { - boolean[] isDeleted = buildIsDeleted(vectors); - readDeletedColumn(vectors, isDeleted); - } else { - Pair pair = buildRowIdMapping(vectors); - if (pair != null) { - int[] rowIdMapping = pair.first(); - numLiveRows = pair.second(); - for (int i = 0; i < vectors.length; i++) { - vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping); - } - } - } - - if (deletes != null && deletes.hasEqDeletes()) { - vectors = ColumnarBatchUtil.removeExtraColumns(deletes, vectors); - } ColumnarBatch batch = new ColumnarBatch(vectors); - batch.setNumRows(numLiveRows); + batch.setNumRows(batchSize); return batch; } - private boolean[] buildIsDeleted(ColumnVector[] vectors) { - return ColumnarBatchUtil.buildIsDeleted(vectors, deletes, rowStartPosInBatch, batchSize); - } - - private Pair buildRowIdMapping(ColumnVector[] vectors) { - return ColumnarBatchUtil.buildRowIdMapping(vectors, deletes, rowStartPosInBatch, batchSize); - } - ColumnVector[] readDataToColumnVectors() { ColumnVector[] columnVectors = new ColumnVector[readers.length]; // Fetch rows for all readers in the delegate @@ -182,16 +133,5 @@ ColumnVector[] readDataToColumnVectors() { return columnVectors; } - - void readDeletedColumn(ColumnVector[] columnVectors, boolean[] isDeleted) { - for (int i = 0; i < readers.length; i++) { - if (readers[i] instanceof CometDeleteColumnReader) { - CometDeleteColumnReader deleteColumnReader = new CometDeleteColumnReader<>(isDeleted); - deleteColumnReader.setBatchSize(batchSize); - deleteColumnReader.delegate().readBatch(batchSize); - columnVectors[i] = deleteColumnReader.delegate().currentBatch(); - } - } - } } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java index 6235bfe4865e..26219014f777 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java @@ -21,7 +21,7 @@ import org.apache.comet.parquet.MetadataColumnReader; import org.apache.comet.parquet.Native; import org.apache.comet.parquet.TypeUtil; -import org.apache.iceberg.MetadataColumns; +import org.apache.comet.vector.CometVector; import org.apache.iceberg.types.Types; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; @@ -33,11 +33,6 @@ class CometDeleteColumnReader extends CometColumnReader { setDelegate(new DeleteColumnReader()); } - CometDeleteColumnReader(boolean[] isDeleted) { - super(MetadataColumns.IS_DELETED); - setDelegate(new DeleteColumnReader(isDeleted)); - } - @Override public void setBatchSize(int batchSize) { super.setBatchSize(batchSize); @@ -46,30 +41,34 @@ public void setBatchSize(int batchSize) { } private static class DeleteColumnReader extends MetadataColumnReader { - private boolean[] isDeleted; + private final CometDeletedColumnVector deletedVector; DeleteColumnReader() { + this(new boolean[0]); + } + + DeleteColumnReader(boolean[] isDeleted) { super( DataTypes.BooleanType, TypeUtil.convertToParquet( new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), false /* useDecimal128 = false */, false /* isConstant = false */); - this.isDeleted = new boolean[0]; - } - - DeleteColumnReader(boolean[] isDeleted) { - this(); - this.isDeleted = isDeleted; + this.deletedVector = new CometDeletedColumnVector(isDeleted); } @Override public void readBatch(int total) { Native.resetBatch(nativeHandle); // set isDeleted on the native side to be consumed by native execution - Native.setIsDeleted(nativeHandle, isDeleted); + Native.setIsDeleted(nativeHandle, deletedVector.isDeleted()); super.readBatch(total); } + + @Override + public CometVector currentBatch() { + return deletedVector; + } } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java new file mode 100644 index 000000000000..5817f2c20a4f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java @@ -0,0 +1,155 @@ +/* + * 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.data.vectorized; + +import org.apache.comet.shaded.arrow.vector.ValueVector; +import org.apache.comet.vector.CometVector; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +public class CometDeletedColumnVector extends CometVector implements UpdatableDeletedColumnVector { + private boolean[] isDeleted; + + public CometDeletedColumnVector(boolean[] isDeleted) { + super(SparkSchemaUtil.convert(Types.BooleanType.get()), false); + this.isDeleted = isDeleted; + } + + @Override + public void setValue(boolean[] deleted) { + this.isDeleted = deleted; + } + + boolean[] isDeleted() { + return isDeleted; + } + + @Override + public void setNumNulls(int numNulls) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void setNumValues(int numValues) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public int numValues() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public ValueVector getValueVector() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public CometVector slice(int offset, int length) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void close() {} + + @Override + public boolean hasNull() { + return false; + } + + @Override + public int numNulls() { + return 0; + } + + @Override + public boolean isNullAt(int rowId) { + return false; + } + + @Override + public boolean getBoolean(int rowId) { + return isDeleted[rowId]; + } + + @Override + public byte getByte(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public short getShort(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public int getInt(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public long getLong(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloat(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public double getDouble(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarArray getArray(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarMap getMap(int ordinal) { + throw new UnsupportedOperationException(); + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + throw new UnsupportedOperationException(); + } + + @Override + public UTF8String getUTF8String(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] getBinary(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnVector getChild(int ordinal) { + throw new UnsupportedOperationException(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java index d36f1a727477..779dc240d4f6 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java @@ -24,7 +24,6 @@ import java.util.stream.IntStream; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; -import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.TypeWithSchemaVisitor; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -37,7 +36,6 @@ import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; -import org.apache.spark.sql.catalyst.InternalRow; class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor> { @@ -45,19 +43,16 @@ class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor idToConstant; private final Function>, VectorizedReader> readerFactory; - private final DeleteFilter deleteFilter; CometVectorizedReaderBuilder( Schema expectedSchema, MessageType parquetSchema, Map idToConstant, - Function>, VectorizedReader> readerFactory, - DeleteFilter deleteFilter) { + Function>, VectorizedReader> readerFactory) { this.parquetSchema = parquetSchema; this.icebergSchema = expectedSchema; this.idToConstant = idToConstant; this.readerFactory = readerFactory; - this.deleteFilter = deleteFilter; } @Override @@ -107,11 +102,7 @@ public VectorizedReader message( } protected VectorizedReader vectorizedReader(List> reorderedFields) { - VectorizedReader reader = readerFactory.apply(reorderedFields); - if (deleteFilter != null) { - ((CometColumnarBatchReader) reader).setDeleteFilter(deleteFilter); - } - return reader; + return readerFactory.apply(reorderedFields); } @Override diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java index a453247068d8..fa3bcfdd004e 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java @@ -26,13 +26,14 @@ import org.apache.spark.sql.vectorized.ColumnarMap; import org.apache.spark.unsafe.types.UTF8String; -public class DeletedColumnVector extends ColumnVector { +public class DeletedColumnVector extends ColumnVector implements UpdatableDeletedColumnVector { private boolean[] isDeleted; public DeletedColumnVector(Type type) { super(SparkSchemaUtil.convert(type)); } + @Override public void setValue(boolean[] deleted) { this.isDeleted = deleted; } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java new file mode 100644 index 000000000000..99bedc42bf3c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java @@ -0,0 +1,23 @@ +/* + * 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.data.vectorized; + +public interface UpdatableDeletedColumnVector { + void setValue(boolean[] isDeleted); +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java index d95baa724b82..8e25e81a05b2 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -26,12 +26,10 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.arrow.ArrowAllocation; import org.apache.iceberg.arrow.vectorized.VectorizedReaderBuilder; -import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.TypeWithSchemaVisitor; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.spark.SparkUtil; import org.apache.parquet.schema.MessageType; -import org.apache.spark.sql.catalyst.InternalRow; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,7 +56,6 @@ public static ColumnarBatchReader buildReader( Schema expectedSchema, MessageType fileSchema, Map idToConstant, - DeleteFilter deleteFilter, BufferAllocator bufferAllocator) { return (ColumnarBatchReader) TypeWithSchemaVisitor.visit( @@ -70,24 +67,16 @@ public static ColumnarBatchReader buildReader( NullCheckingForGet.NULL_CHECKING_ENABLED, idToConstant, ColumnarBatchReader::new, - deleteFilter, bufferAllocator)); } public static ColumnarBatchReader buildReader( - Schema expectedSchema, - MessageType fileSchema, - Map idToConstant, - DeleteFilter deleteFilter) { - return buildReader( - expectedSchema, fileSchema, idToConstant, deleteFilter, ArrowAllocation.rootAllocator()); + Schema expectedSchema, MessageType fileSchema, Map idToConstant) { + return buildReader(expectedSchema, fileSchema, idToConstant, ArrowAllocation.rootAllocator()); } public static CometColumnarBatchReader buildCometReader( - Schema expectedSchema, - MessageType fileSchema, - Map idToConstant, - DeleteFilter deleteFilter) { + Schema expectedSchema, MessageType fileSchema, Map idToConstant) { return (CometColumnarBatchReader) TypeWithSchemaVisitor.visit( expectedSchema.asStruct(), @@ -96,8 +85,7 @@ public static CometColumnarBatchReader buildCometReader( expectedSchema, fileSchema, idToConstant, - readers -> new CometColumnarBatchReader(readers, expectedSchema), - deleteFilter)); + readers -> new CometColumnarBatchReader(readers, expectedSchema))); } // enables unsafe memory access to avoid costly checks to see if index is within bounds @@ -134,7 +122,6 @@ private static String confValue(String propName, String envName) { } private static class ReaderBuilder extends VectorizedReaderBuilder { - private final DeleteFilter deleteFilter; ReaderBuilder( Schema expectedSchema, @@ -142,7 +129,6 @@ private static class ReaderBuilder extends VectorizedReaderBuilder { boolean setArrowValidityVector, Map idToConstant, Function>, VectorizedReader> readerFactory, - DeleteFilter deleteFilter, BufferAllocator bufferAllocator) { super( expectedSchema, @@ -152,16 +138,6 @@ private static class ReaderBuilder extends VectorizedReaderBuilder { readerFactory, SparkUtil::internalToSpark, bufferAllocator); - this.deleteFilter = deleteFilter; - } - - @Override - protected VectorizedReader vectorizedReader(List> reorderedFields) { - VectorizedReader reader = super.vectorizedReader(reorderedFields); - if (deleteFilter != null) { - ((ColumnarBatchReader) reader).setDeleteFilter(deleteFilter); - } - return reader; } } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index a0f45e7610a2..ff30f29aeae6 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -20,24 +20,33 @@ import java.util.Map; import java.util.Set; +import javax.annotation.Nonnull; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.OrcBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; import org.apache.iceberg.spark.ParquetReaderType; +import org.apache.iceberg.spark.data.vectorized.ColumnVectorWithFilter; +import org.apache.iceberg.spark.data.vectorized.ColumnarBatchUtil; +import org.apache.iceberg.spark.data.vectorized.UpdatableDeletedColumnVector; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; abstract class BaseBatchReader extends BaseReader { @@ -66,18 +75,23 @@ protected CloseableIterable newBatchIterable( long length, Expression residual, Map idToConstant, - SparkDeleteFilter deleteFilter) { + @Nonnull SparkDeleteFilter deleteFilter) { + CloseableIterable iterable; switch (format) { case PARQUET: - return newParquetIterable(inputFile, start, length, residual, idToConstant, deleteFilter); - + iterable = + newParquetIterable( + inputFile, start, length, residual, idToConstant, deleteFilter.requiredSchema()); + break; case ORC: - return newOrcIterable(inputFile, start, length, residual, idToConstant); - + iterable = newOrcIterable(inputFile, start, length, residual, idToConstant); + break; default: throw new UnsupportedOperationException( "Format: " + format + " not supported for batched reads"); } + + return CloseableIterable.transform(iterable, new BatchDeleteFilter(deleteFilter)::filterBatch); } private CloseableIterable newParquetIterable( @@ -86,10 +100,7 @@ private CloseableIterable newParquetIterable( long length, Expression residual, Map idToConstant, - SparkDeleteFilter deleteFilter) { - // get required schema if there are deletes - Schema requiredSchema = deleteFilter != null ? deleteFilter.requiredSchema() : expectedSchema(); - + Schema requiredSchema) { return Parquet.read(inputFile) .project(requiredSchema) .split(start, length) @@ -97,10 +108,10 @@ private CloseableIterable newParquetIterable( fileSchema -> { if (parquetConf.readerType() == ParquetReaderType.COMET) { return VectorizedSparkParquetReaders.buildCometReader( - requiredSchema, fileSchema, idToConstant, deleteFilter); + requiredSchema, fileSchema, idToConstant); } else { return VectorizedSparkParquetReaders.buildReader( - requiredSchema, fileSchema, idToConstant, deleteFilter); + requiredSchema, fileSchema, idToConstant); } }) .recordsPerBatch(parquetConf.batchSize()) @@ -139,4 +150,72 @@ private CloseableIterable newOrcIterable( .withNameMapping(nameMapping()) .build(); } + + @VisibleForTesting + static class BatchDeleteFilter { + private final DeleteFilter deletes; + private boolean hasIsDeletedColumn; + private int rowPositionColumnIndex = -1; + + BatchDeleteFilter(DeleteFilter deletes) { + this.deletes = deletes; + + Schema schema = deletes.requiredSchema(); + for (int i = 0; i < schema.columns().size(); i++) { + if (schema.columns().get(i).fieldId() == MetadataColumns.ROW_POSITION.fieldId()) { + this.rowPositionColumnIndex = i; + } else if (schema.columns().get(i).fieldId() == MetadataColumns.IS_DELETED.fieldId()) { + this.hasIsDeletedColumn = true; + } + } + } + + ColumnarBatch filterBatch(ColumnarBatch batch) { + if (!needDeletes()) { + return batch; + } + + ColumnVector[] vectors = new ColumnVector[batch.numCols()]; + for (int i = 0; i < batch.numCols(); i++) { + vectors[i] = batch.column(i); + } + + int numLiveRows = batch.numRows(); + long rowStartPosInBatch = + rowPositionColumnIndex == -1 ? -1 : vectors[rowPositionColumnIndex].getLong(0); + + if (hasIsDeletedColumn) { + boolean[] isDeleted = + ColumnarBatchUtil.buildIsDeleted(vectors, deletes, rowStartPosInBatch, numLiveRows); + for (ColumnVector vector : vectors) { + if (vector instanceof UpdatableDeletedColumnVector) { + ((UpdatableDeletedColumnVector) vector).setValue(isDeleted); + } + } + } else { + Pair pair = + ColumnarBatchUtil.buildRowIdMapping(vectors, deletes, rowStartPosInBatch, numLiveRows); + if (pair != null) { + int[] rowIdMapping = pair.first(); + numLiveRows = pair.second(); + for (int i = 0; i < vectors.length; i++) { + vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping); + } + } + } + + if (deletes != null && deletes.hasEqDeletes()) { + vectors = ColumnarBatchUtil.removeExtraColumns(deletes, vectors); + } + + ColumnarBatch output = new ColumnarBatch(vectors); + output.setNumRows(numLiveRows); + return output; + } + + private boolean needDeletes() { + return hasIsDeletedColumn + || (deletes != null && (deletes.hasEqDeletes() || deletes.hasPosDeletes())); + } + } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index 20128ccef8b2..9ec0f885775f 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -105,15 +105,13 @@ protected CloseableIterator open(FileScanTask task) { // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); - Map idToConstant = constantsMap(task, expectedSchema()); - InputFile inputFile = getInputFile(filePath); Preconditions.checkNotNull(inputFile, "Could not find InputFile associated with FileScanTask"); SparkDeleteFilter deleteFilter = - task.deletes().isEmpty() - ? null - : new SparkDeleteFilter(filePath, task.deletes(), counter(), false); + new SparkDeleteFilter(filePath, task.deletes(), counter(), true); + + Map idToConstant = constantsMap(task, deleteFilter.requiredSchema()); return newBatchIterable( inputFile, diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java index 044ea3d93c0b..ccd783915c62 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java @@ -21,8 +21,6 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; import java.io.File; import java.io.IOException; @@ -37,12 +35,15 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.deletes.DeleteCounter; import org.apache.iceberg.deletes.PositionDeleteIndex; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.InputFile; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.parquet.ParquetSchemaUtil; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -51,6 +52,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; +import org.apache.iceberg.spark.source.BatchReaderUtil; import org.apache.iceberg.types.Types; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.hadoop.ParquetFileReader; @@ -181,22 +183,52 @@ public void testReadRowNumbersWithDelete() throws IOException { Parquet.ReadBuilder builder = Parquet.read(Files.localInput(testFile)).project(PROJECTION_SCHEMA); - DeleteFilter deleteFilter = mock(DeleteFilter.class); - when(deleteFilter.hasPosDeletes()).thenReturn(true); - PositionDeleteIndex deletedRowPos = new CustomizedPositionDeleteIndex(); - deletedRowPos.delete(98, 103); - when(deleteFilter.deletedRowPositions()).thenReturn(deletedRowPos); + DeleteFilter deleteFilter = new TestDeleteFilter(true); builder.createBatchedReaderFunc( fileSchema -> VectorizedSparkParquetReaders.buildReader( - PROJECTION_SCHEMA, fileSchema, Maps.newHashMap(), deleteFilter)); + PROJECTION_SCHEMA, fileSchema, Maps.newHashMap())); builder.recordsPerBatch(RECORDS_PER_BATCH); - validate(expectedRowsAfterDelete, builder); + validate(expectedRowsAfterDelete, builder, deleteFilter); } - private class CustomizedPositionDeleteIndex implements PositionDeleteIndex { + private static class TestDeleteFilter extends DeleteFilter { + private final boolean hasDeletes; + + protected TestDeleteFilter(boolean hasDeletes) { + super("", List.of(), DATA_SCHEMA, PROJECTION_SCHEMA, new DeleteCounter(), true); + this.hasDeletes = hasDeletes; + } + + @Override + protected StructLike asStructLike(InternalRow record) { + return null; + } + + @Override + protected InputFile getInputFile(String location) { + return null; + } + + @Override + public boolean hasPosDeletes() { + return hasDeletes; + } + + @Override + public PositionDeleteIndex deletedRowPositions() { + PositionDeleteIndex deletedRowPos = new CustomizedPositionDeleteIndex(); + if (hasDeletes) { + deletedRowPos.delete(98, 103); + } + + return deletedRowPos; + } + } + + private static class CustomizedPositionDeleteIndex implements PositionDeleteIndex { private final Set deleteIndex; private CustomizedPositionDeleteIndex() { @@ -266,7 +298,7 @@ private void readAndValidate( builder.createBatchedReaderFunc( fileSchema -> VectorizedSparkParquetReaders.buildReader( - PROJECTION_SCHEMA, fileSchema, Maps.newHashMap(), null)); + PROJECTION_SCHEMA, fileSchema, Maps.newHashMap())); builder.recordsPerBatch(RECORDS_PER_BATCH); } else { builder = @@ -282,13 +314,13 @@ private void readAndValidate( builder = builder.split(splitStart, splitLength); } - validate(expected, builder); + validate(expected, builder, new TestDeleteFilter(false)); } - private void validate(List expected, Parquet.ReadBuilder builder) + private void validate( + List expected, Parquet.ReadBuilder builder, DeleteFilter filter) throws IOException { - try (CloseableIterable reader = - vectorized ? batchesToRows(builder.build()) : builder.build()) { + try (CloseableIterable reader = reader(builder, filter)) { final Iterator actualRows = reader.iterator(); for (InternalRow internalRow : expected) { @@ -300,6 +332,15 @@ private void validate(List expected, Parquet.ReadBuilder builder) } } + private CloseableIterable reader( + Parquet.ReadBuilder builder, DeleteFilter filter) { + if (!vectorized) { + return builder.build(); + } else { + return batchesToRows(BatchReaderUtil.applyDeleteFilter(builder.build(), filter)); + } + } + private CloseableIterable batchesToRows(CloseableIterable batches) { return CloseableIterable.combine( Iterables.concat(Iterables.transform(batches, b -> (Iterable) b::rowIterator)), diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java index 973a17c9a388..284fa0b0552f 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java @@ -179,7 +179,7 @@ public void testDecimalNotAllPagesDictionaryEncoded() throws Exception { .createBatchedReaderFunc( type -> VectorizedSparkParquetReaders.buildReader( - schema, type, ImmutableMap.of(), null, allocator)); + schema, type, ImmutableMap.of(), allocator)); try (CloseableIterable batchReader = readBuilder.build()) { Iterator expectedIter = expected.iterator(); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java index 2e46088cfe6d..46a6a302e1c4 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java @@ -277,7 +277,7 @@ void assertRecordsMatch( .createBatchedReaderFunc( type -> VectorizedSparkParquetReaders.buildReader( - schema, type, idToConstant, null, allocator)); + schema, type, idToConstant, allocator)); if (reuseContainers) { readBuilder.reuseContainers(); } @@ -308,8 +308,7 @@ public void testNestedStruct() { new Schema(required(1, "struct", SUPPORTED_PRIMITIVES))), new MessageType( "struct", new GroupType(Type.Repetition.OPTIONAL, "struct").withId(1)), - Maps.newHashMap(), - null)) + Maps.newHashMap())) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Vectorized reads are not supported yet for struct fields"); } diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java new file mode 100644 index 000000000000..e5d03a4efba5 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java @@ -0,0 +1,34 @@ +/* + * 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.data.DeleteFilter; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +public class BatchReaderUtil { + private BatchReaderUtil() {} + + public static CloseableIterable applyDeleteFilter( + CloseableIterable batches, DeleteFilter filter) { + return CloseableIterable.transform( + batches, new BaseBatchReader.BatchDeleteFilter(filter)::filterBatch); + } +} From 8626ef5137024c1a69daaff97a832af6b0ae37ea Mon Sep 17 00:00:00 2001 From: ggershinsky Date: Thu, 4 Dec 2025 00:27:52 +0200 Subject: [PATCH 035/201] Docs: encryption (#14621) * initial commit * clean up * brief how it works section * clean up * add refs * discussion updates * address review comments * add ref to custom catalogs doc * add line break --- docs/docs/configuration.md | 10 +++ docs/docs/custom-catalog.md | 2 + docs/docs/encryption.md | 153 ++++++++++++++++++++++++++++++++++++ docs/mkdocs.yml | 1 + 4 files changed, 166 insertions(+) create mode 100644 docs/docs/encryption.md diff --git a/docs/docs/configuration.md b/docs/docs/configuration.md index 161fe8b6e78a..3f730a4f4c13 100644 --- a/docs/docs/configuration.md +++ b/docs/docs/configuration.md @@ -90,6 +90,15 @@ Iceberg tables support table properties to configure table behavior, like the de | write.merge.isolation-level | serializable | Isolation level for merge commands: serializable or snapshot | | write.delete.granularity | partition | Controls the granularity of generated delete files: partition or file | +### Encryption properties + +| Property | Default | Description | +| --------------------------------- | ------------------ | ------------------------------------------------------------------------------------- | +| encryption.key-id | (not set) | ID of the master key of the table | +| encryption.data-key-length | 16 (bytes) | Length of keys used for encryption of table files. Valid values are 16, 24, 32 bytes | + +See the [Encryption](encryption.md) document for additional details. + ### Table behavior properties | Property | Default | Description | @@ -138,6 +147,7 @@ Iceberg catalogs support using catalog properties to configure catalog behaviors | cache-enabled | true | Whether to cache catalog entries | | cache.expiration-interval-ms | 30000 | How long catalog entries are locally cached, in milliseconds; 0 disables caching, negative values disable expiration | | metrics-reporter-impl | org.apache.iceberg.metrics.LoggingMetricsReporter | Custom `MetricsReporter` implementation to use in a catalog. See the [Metrics reporting](metrics-reporting.md) section for additional details | +| encryption.kms-impl | null | a custom `KeyManagementClient` implementation to use in a catalog for interactions with KMS (key management service). See the [Encryption](encryption.md) document for additional details | `HadoopCatalog` and `HiveCatalog` can access the properties in their constructors. Any other custom catalog can access the properties by implementing `Catalog.initialize(catalogName, catalogProperties)`. diff --git a/docs/docs/custom-catalog.md b/docs/docs/custom-catalog.md index a812046589e8..f0a6b5718a6c 100644 --- a/docs/docs/custom-catalog.md +++ b/docs/docs/custom-catalog.md @@ -28,6 +28,8 @@ It's possible to read an iceberg table either from an hdfs path or from a hive t - [Custom LocationProvider](#custom-location-provider-implementation) - [Custom IcebergSource](#custom-icebergsource) +Note: To work with encrypted tables, custom catalogs must address a number of security [requirements](encryption.md#catalog-security-requirements). + ### Custom table operations implementation Extend `BaseMetastoreTableOperations` to provide implementation on how to read and write metadata diff --git a/docs/docs/encryption.md b/docs/docs/encryption.md new file mode 100644 index 000000000000..0b694c44005b --- /dev/null +++ b/docs/docs/encryption.md @@ -0,0 +1,153 @@ +--- +title: "Encryption" +--- + + +# Encryption + +Iceberg table encryption protects confidentiality and integrity of table data in an untrusted storage. The `data`, `delete`, `manifest` and `manifest list` files are encrypted and tamper-proofed before being sent to the storage backend. + +The `metadata.json` file does not contain data or stats, and is therefore not encrypted. + +Currently, encryption is supported in the Hive and REST catalogs for tables with Parquet and Avro data formats. + +Two parameters are required to activate encryption of a table +1. Catalog property `encryption.kms-impl`, that specifies the class path for a client of a KMS ("key management service"). +2. Table property `encryption.key-id`, that specifies the ID of a master key used to encrypt and decrypt the table. Master keys are stored and managed in the KMS. + +For more details on table encryption, see the "Appendix: Internals Overview" [subsection](#appendix-internals-overview). + +## Example + +```sh +spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-{{ sparkVersionMajor }}:{{ icebergVersion }}\ + --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \ + --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog \ + --conf spark.sql.catalog.spark_catalog.type=hive \ + --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.local.type=hive \ + --conf spark.sql.catalog.local.encryption.kms-impl=org.apache.iceberg.aws.AwsKeyManagementClient +``` + +```sql +CREATE TABLE local.db.table (id bigint, data string) USING iceberg +TBLPROPERTIES ('encryption.key-id'='{{ master key id }}'); +``` + +Inserted data will be automatically encrypted, + +```sql +INSERT INTO local.db.table VALUES (1, 'a'), (2, 'b'), (3, 'c'); +``` + +To verify encryption, the contents of data, manifest and manifest list files can be dumped in the command line with + +```sh +hexdump -C {{ /path/to/file }} | more +``` + +The Parquet files must start with the "PARE" magic string (PARquet Encrypted footer mode), and manifest/list files must start with "AGS1" magic string (Aes Gcm Stream version 1). + +Queried data will be automatically decrypted, + +```sql +SELECT * FROM local.db.table; +``` + +## Catalog security requirements + +1. Catalogs must ensure the `encryption.key-id` property is not modified or removed during table lifetime. + +2. To function properly, Iceberg table encryption requires the catalog implementations not to retrieve the metadata +directly from metadata.json files, if these files are kept unprotected in a storage vulnerable to tampering. + +* Catalogs may keep the metadata in a trusted independent object store. +* Catalogs may work with metadata.json files in a tamper-proof storage. +* Catalogs may use checksum techniques to verify integrity of metadata.json files in a storage vulnerable to tampering +(the checksums must be kept in a separate trusted storage). + +## Key Management Clients + +Currently, Iceberg has clients for the AWS, GCP and Azure KMS systems. A custom client can be built for other key management systems by implementing the `org.apache.iceberg.encryption.KeyManagementClient` interface. + +This interface has the following main methods, + +```java + /** + * Initialize the KMS client with given properties. + * + * @param properties kms client properties (taken from catalog properties) + */ + void initialize(Map properties); + + /** + * Wrap a secret key, using a wrapping/master key which is stored in KMS and referenced by an ID. + * Wrapping means encryption of the secret key with the master key, and adding optional + * KMS-specific metadata that allows the KMS to decrypt the secret key in an unwrapping call. + * + * @param key a secret key being wrapped + * @param wrappingKeyId a key ID that represents a wrapping key stored in KMS + * @return wrapped key material + */ + ByteBuffer wrapKey(ByteBuffer key, String wrappingKeyId); + + /** + * Unwrap a secret key, using a wrapping/master key which is stored in KMS and referenced by an + * ID. + * + * @param wrappedKey wrapped key material (encrypted key and optional KMS metadata, returned by + * the wrapKey method) + * @param wrappingKeyId a key ID that represents a wrapping key stored in KMS + * @return raw key bytes + */ + ByteBuffer unwrapKey(ByteBuffer wrappedKey, String wrappingKeyId); +``` + +## Appendix: Internals Overview + +The standard Iceberg encryption manager generates an encryption key and a unique file ID ("AAD prefix") +for each data and delete file. The generation is performed in the worker nodes, by using a secure random +number generator. For Parquet data files, these parameters are passed to the native Parquet Modular +Encryption [mechanism](https://parquet.apache.org/docs/file-format/data-pages/encryption). For Avro data files, +these parameters are passed to the AES GCM Stream encryption [mechanism](../../format/gcm-stream-spec.md). + +The parent manifest file stores the encryption key and AAD prefix for each data and delete file in the +`key_metadata` [field](../../format/spec.md#data-file-fields). For Avro data tables, the data file length +is also added to the `key_metadata`. +The manifest file is encrypted by the AES GCM Stream encryption mechanism, using an encryption key and an +AAD prefix generated by the standard encryption manager. The generation is performed in the driver nodes, +by using a secure random number generator. + +The parent manifest list file stores the encryption key, AAD prefix and file length for each manifest file +in the `key_metadata` [field](../../format/spec.md#manifest-lists). The manifest list file is encrypted by +the AES GCM Stream encryption mechanism, +using an encryption key and an AAD prefix generated by the standard encryption manager. + +The manifest list encryption key, AAD prefix and file length are packed in a key metadata object. This object +is serialized and encrypted with a "key encryption key" (KEK), using the KEK creation timestamp as the AES +GCM AAD. A KEK and its unique KEK_ID are generated by using a secure random number generator. For each +snapshot, the KEK_ID of the encryption key that encrypts the manifest list key metadata is kept in the +`key-id` field in the table metadata snapshot [structure](../../format/spec.md#snapshots). The encrypted +manifest list key metadata is kept in the `encryption-keys` list in the table metadata +[structure](../../format/spec.md#table-metadata-fields). + +The KEK is encrypted by the table master key via the KMS client. The result is kept in the `encryption-keys` +list in the table metadata structure. The KEK is re-used for a period allowed by the NIST SP 800-57 +specification. Then, it is rotated - a new KEK and KEK_ID are generated for encryption of new manifest list +key metadata objects. The new KEK is encrypted by the table master key and stored in the `encryption-keys` +list in the table metadata structure. The previous KEKs are retained for the existing table snapshots. diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index 13b187909208..c1807a6b8542 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -26,6 +26,7 @@ nav: - Tables: - branching.md - configuration.md + - encryption.md - evolution.md - maintenance.md - metrics-reporting.md From 52d6a79e93f2f820787ad27e67ac795991ff8fa4 Mon Sep 17 00:00:00 2001 From: Sreesh Maheshwar Date: Thu, 4 Dec 2025 07:03:46 +0000 Subject: [PATCH 036/201] Nit: Prefer `Preconditions` in `StandardEncryptionManager` (#14753) --- .../encryption/StandardEncryptionManager.java | 47 ++++++++----------- 1 file changed, 20 insertions(+), 27 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java b/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java index 1e6ac961d3be..8edb6a136dc7 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java +++ b/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java @@ -145,10 +145,9 @@ private SecureRandom workerRNG() { */ @Deprecated public ByteBuffer wrapKey(ByteBuffer secretKey) { - if (transientState == null) { - throw new IllegalStateException( - "Cannot wrap key after called after serialization (missing KMS client)"); - } + Preconditions.checkState( + transientState != null, + "Cannot wrap key after called after serialization (missing KMS client)"); return transientState.kmsClient.wrapKey(secretKey, tableKeyId); } @@ -158,25 +157,21 @@ public ByteBuffer wrapKey(ByteBuffer secretKey) { */ @Deprecated public ByteBuffer unwrapKey(ByteBuffer wrappedSecretKey) { - if (transientState == null) { - throw new IllegalStateException("Cannot unwrap key after serialization"); - } + Preconditions.checkState(transientState != null, "Cannot unwrap key after serialization"); return transientState.kmsClient.unwrapKey(wrappedSecretKey, tableKeyId); } Map encryptionKeys() { - if (transientState == null) { - throw new IllegalStateException("Cannot return the encryption keys after serialization"); - } + Preconditions.checkState( + transientState != null, "Cannot return the encryption keys after serialization"); return transientState.encryptionKeys; } String keyEncryptionKeyID() { - if (transientState == null) { - throw new IllegalStateException("Cannot return the current key after serialization"); - } + Preconditions.checkState( + transientState != null, "Cannot return the current key after serialization"); // Find unexpired key encryption key for (String keyID : transientState.encryptionKeys.keySet()) { @@ -214,28 +209,26 @@ private long currentTimeMillis() { } ByteBuffer encryptedByKey(String manifestListKeyID) { - if (transientState == null) { - throw new IllegalStateException("Cannot find key encryption key after serialization"); - } + Preconditions.checkState( + transientState != null, "Cannot find key encryption key after serialization"); EncryptedKey encryptedKeyMetadata = transientState.encryptionKeys.get(manifestListKeyID); - if (encryptedKeyMetadata == null) { - throw new IllegalStateException( - "Cannot find manifest list key metadata with id " + manifestListKeyID); - } - if (encryptedKeyMetadata.encryptedById().equals(tableKeyId)) { - throw new IllegalArgumentException( - manifestListKeyID + " is a key encryption key, not manifest list key metadata"); - } + Preconditions.checkState( + encryptedKeyMetadata != null, + "Cannot find manifest list key metadata with id %s", + manifestListKeyID); + + Preconditions.checkArgument( + !encryptedKeyMetadata.encryptedById().equals(tableKeyId), + "%s is a key encryption key, not manifest list key metadata", + manifestListKeyID); return transientState.unwrappedKeyCache.get(encryptedKeyMetadata.encryptedById()); } public String addManifestListKeyMetadata(NativeEncryptionKeyMetadata keyMetadata) { - if (transientState == null) { - throw new IllegalStateException("Cannot add key metadata after serialization"); - } + Preconditions.checkState(transientState != null, "Cannot add key metadata after serialization"); String manifestListKeyID = generateKeyId(); String keyEncryptionKeyID = keyEncryptionKeyID(); From 65280c0153ec53f5a1a9ff1b2a608a75018dc147 Mon Sep 17 00:00:00 2001 From: pvary Date: Thu, 4 Dec 2025 10:14:36 +0100 Subject: [PATCH 037/201] Spark: Backport move DeleteFiltering out from the vectorized reader (#14745) --- .../data/vectorized/ColumnarBatchReader.java | 57 +------ .../vectorized/CometColumnarBatchReader.java | 64 +------- .../vectorized/CometDeleteColumnReader.java | 29 ++-- .../vectorized/CometDeletedColumnVector.java | 155 ++++++++++++++++++ .../CometVectorizedReaderBuilder.java | 13 +- .../data/vectorized/DeletedColumnVector.java | 3 +- .../UpdatableDeletedColumnVector.java | 23 +++ .../VectorizedSparkParquetReaders.java | 32 +--- .../iceberg/spark/source/BaseBatchReader.java | 101 ++++++++++-- .../iceberg/spark/source/BatchDataReader.java | 8 +- .../TestSparkParquetReadMetadataColumns.java | 71 ++++++-- .../TestParquetVectorizedReads.java | 5 +- .../iceberg/spark/source/BatchReaderUtil.java | 34 ++++ .../data/vectorized/ColumnarBatchReader.java | 57 +------ .../vectorized/CometColumnarBatchReader.java | 64 +------- .../vectorized/CometDeleteColumnReader.java | 27 ++- .../vectorized/CometDeletedColumnVector.java | 155 ++++++++++++++++++ .../CometVectorizedReaderBuilder.java | 13 +- .../data/vectorized/DeletedColumnVector.java | 3 +- .../UpdatableDeletedColumnVector.java | 23 +++ .../VectorizedSparkParquetReaders.java | 32 +--- .../iceberg/spark/source/BaseBatchReader.java | 101 ++++++++++-- .../iceberg/spark/source/BatchDataReader.java | 8 +- .../TestSparkParquetReadMetadataColumns.java | 71 ++++++-- ...rquetDictionaryEncodedVectorizedReads.java | 2 +- .../parquet/TestParquetVectorizedReads.java | 5 +- .../iceberg/spark/source/BatchReaderUtil.java | 34 ++++ 27 files changed, 778 insertions(+), 412 deletions(-) create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java create mode 100644 spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java index a0670c2bbe71..38d505d250d0 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java @@ -22,15 +22,11 @@ import java.util.Map; import org.apache.iceberg.arrow.vectorized.BaseBatchReader; import org.apache.iceberg.arrow.vectorized.VectorizedArrowReader; -import org.apache.iceberg.arrow.vectorized.VectorizedArrowReader.DeletedVectorReader; -import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.util.Pair; import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; @@ -40,31 +36,15 @@ * populated via delegated read calls to {@linkplain VectorizedArrowReader VectorReader(s)}. */ public class ColumnarBatchReader extends BaseBatchReader { - private final boolean hasIsDeletedColumn; - private DeleteFilter deletes = null; - private long rowStartPosInBatch = 0; public ColumnarBatchReader(List> readers) { super(readers); - this.hasIsDeletedColumn = - readers.stream().anyMatch(reader -> reader instanceof DeletedVectorReader); } @Override public void setRowGroupInfo( PageReadStore pageStore, Map metaData) { super.setRowGroupInfo(pageStore, metaData); - this.rowStartPosInBatch = - pageStore - .getRowIndexOffset() - .orElseThrow( - () -> - new IllegalArgumentException( - "PageReadStore does not contain row index offset")); - } - - public void setDeleteFilter(DeleteFilter deleteFilter) { - this.deletes = deleteFilter; } @Override @@ -73,9 +53,7 @@ public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) { closeVectors(); } - ColumnarBatch columnarBatch = new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); - rowStartPosInBatch += numRowsToRead; - return columnarBatch; + return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); } private class ColumnBatchLoader { @@ -89,43 +67,12 @@ private class ColumnBatchLoader { ColumnarBatch loadDataToColumnBatch() { ColumnVector[] vectors = readDataToColumnVectors(); - int numLiveRows = batchSize; - - if (hasIsDeletedColumn) { - boolean[] isDeleted = buildIsDeleted(vectors); - for (ColumnVector vector : vectors) { - if (vector instanceof DeletedColumnVector) { - ((DeletedColumnVector) vector).setValue(isDeleted); - } - } - } else { - Pair pair = buildRowIdMapping(vectors); - if (pair != null) { - int[] rowIdMapping = pair.first(); - numLiveRows = pair.second(); - for (int i = 0; i < vectors.length; i++) { - vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping); - } - } - } - - if (deletes != null && deletes.hasEqDeletes()) { - vectors = ColumnarBatchUtil.removeExtraColumns(deletes, vectors); - } ColumnarBatch batch = new ColumnarBatch(vectors); - batch.setNumRows(numLiveRows); + batch.setNumRows(batchSize); return batch; } - private boolean[] buildIsDeleted(ColumnVector[] vectors) { - return ColumnarBatchUtil.buildIsDeleted(vectors, deletes, rowStartPosInBatch, batchSize); - } - - private Pair buildRowIdMapping(ColumnVector[] vectors) { - return ColumnarBatchUtil.buildRowIdMapping(vectors, deletes, rowStartPosInBatch, batchSize); - } - ColumnVector[] readDataToColumnVectors() { ColumnVector[] arrowColumnVectors = new ColumnVector[readers.length]; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java index 04ac69476add..3d3e9aca24de 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java @@ -25,15 +25,12 @@ import org.apache.comet.parquet.AbstractColumnReader; import org.apache.comet.parquet.BatchReader; import org.apache.iceberg.Schema; -import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.util.Pair; import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; @@ -46,7 +43,6 @@ class CometColumnarBatchReader implements VectorizedReader { private final CometColumnReader[] readers; - private final boolean hasIsDeletedColumn; // The delegated BatchReader on the Comet side does the real work of loading a batch of rows. // The Comet BatchReader contains an array of ColumnReader. There is no need to explicitly call @@ -56,14 +52,10 @@ class CometColumnarBatchReader implements VectorizedReader { // DeleteColumnReader.readBatch must be called explicitly later, after the isDeleted value is // available. private final BatchReader delegate; - private DeleteFilter deletes = null; - private long rowStartPosInBatch = 0; CometColumnarBatchReader(List> readers, Schema schema) { this.readers = readers.stream().map(CometColumnReader.class::cast).toArray(CometColumnReader[]::new); - this.hasIsDeletedColumn = - readers.stream().anyMatch(reader -> reader instanceof CometDeleteColumnReader); AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; this.delegate = new BatchReader(abstractColumnReaders); @@ -89,25 +81,11 @@ public void setRowGroupInfo( for (int i = 0; i < readers.length; i++) { delegate.getColumnReaders()[i] = this.readers[i].delegate(); } - - this.rowStartPosInBatch = - pageStore - .getRowIndexOffset() - .orElseThrow( - () -> - new IllegalArgumentException( - "PageReadStore does not contain row index offset")); - } - - public void setDeleteFilter(DeleteFilter deleteFilter) { - this.deletes = deleteFilter; } @Override public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) { - ColumnarBatch columnarBatch = new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); - rowStartPosInBatch += numRowsToRead; - return columnarBatch; + return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); } @Override @@ -139,39 +117,12 @@ private class ColumnBatchLoader { ColumnarBatch loadDataToColumnBatch() { ColumnVector[] vectors = readDataToColumnVectors(); - int numLiveRows = batchSize; - - if (hasIsDeletedColumn) { - boolean[] isDeleted = buildIsDeleted(vectors); - readDeletedColumn(vectors, isDeleted); - } else { - Pair pair = buildRowIdMapping(vectors); - if (pair != null) { - int[] rowIdMapping = pair.first(); - numLiveRows = pair.second(); - for (int i = 0; i < vectors.length; i++) { - vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping); - } - } - } - - if (deletes != null && deletes.hasEqDeletes()) { - vectors = ColumnarBatchUtil.removeExtraColumns(deletes, vectors); - } ColumnarBatch batch = new ColumnarBatch(vectors); - batch.setNumRows(numLiveRows); + batch.setNumRows(batchSize); return batch; } - private boolean[] buildIsDeleted(ColumnVector[] vectors) { - return ColumnarBatchUtil.buildIsDeleted(vectors, deletes, rowStartPosInBatch, batchSize); - } - - private Pair buildRowIdMapping(ColumnVector[] vectors) { - return ColumnarBatchUtil.buildRowIdMapping(vectors, deletes, rowStartPosInBatch, batchSize); - } - ColumnVector[] readDataToColumnVectors() { ColumnVector[] columnVectors = new ColumnVector[readers.length]; // Fetch rows for all readers in the delegate @@ -182,16 +133,5 @@ ColumnVector[] readDataToColumnVectors() { return columnVectors; } - - void readDeletedColumn(ColumnVector[] columnVectors, boolean[] isDeleted) { - for (int i = 0; i < readers.length; i++) { - if (readers[i] instanceof CometDeleteColumnReader) { - CometDeleteColumnReader deleteColumnReader = new CometDeleteColumnReader<>(isDeleted); - deleteColumnReader.setBatchSize(batchSize); - deleteColumnReader.delegate().readBatch(batchSize); - columnVectors[i] = deleteColumnReader.delegate().currentBatch(); - } - } - } } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java index 4a28fc51da9b..26219014f777 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java @@ -21,7 +21,7 @@ import org.apache.comet.parquet.MetadataColumnReader; import org.apache.comet.parquet.Native; import org.apache.comet.parquet.TypeUtil; -import org.apache.iceberg.MetadataColumns; +import org.apache.comet.vector.CometVector; import org.apache.iceberg.types.Types; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; @@ -33,11 +33,6 @@ class CometDeleteColumnReader extends CometColumnReader { setDelegate(new DeleteColumnReader()); } - CometDeleteColumnReader(boolean[] isDeleted) { - super(MetadataColumns.IS_DELETED); - setDelegate(new DeleteColumnReader(isDeleted)); - } - @Override public void setBatchSize(int batchSize) { super.setBatchSize(batchSize); @@ -46,30 +41,34 @@ public void setBatchSize(int batchSize) { } private static class DeleteColumnReader extends MetadataColumnReader { - private boolean[] isDeleted; + private final CometDeletedColumnVector deletedVector; DeleteColumnReader() { + this(new boolean[0]); + } + + DeleteColumnReader(boolean[] isDeleted) { super( DataTypes.BooleanType, TypeUtil.convertToParquet( new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), false /* useDecimal128 = false */, - false /* isConstant */); - this.isDeleted = new boolean[0]; - } - - DeleteColumnReader(boolean[] isDeleted) { - this(); - this.isDeleted = isDeleted; + false /* isConstant = false */); + this.deletedVector = new CometDeletedColumnVector(isDeleted); } @Override public void readBatch(int total) { Native.resetBatch(nativeHandle); // set isDeleted on the native side to be consumed by native execution - Native.setIsDeleted(nativeHandle, isDeleted); + Native.setIsDeleted(nativeHandle, deletedVector.isDeleted()); super.readBatch(total); } + + @Override + public CometVector currentBatch() { + return deletedVector; + } } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java new file mode 100644 index 000000000000..5817f2c20a4f --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java @@ -0,0 +1,155 @@ +/* + * 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.data.vectorized; + +import org.apache.comet.shaded.arrow.vector.ValueVector; +import org.apache.comet.vector.CometVector; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +public class CometDeletedColumnVector extends CometVector implements UpdatableDeletedColumnVector { + private boolean[] isDeleted; + + public CometDeletedColumnVector(boolean[] isDeleted) { + super(SparkSchemaUtil.convert(Types.BooleanType.get()), false); + this.isDeleted = isDeleted; + } + + @Override + public void setValue(boolean[] deleted) { + this.isDeleted = deleted; + } + + boolean[] isDeleted() { + return isDeleted; + } + + @Override + public void setNumNulls(int numNulls) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void setNumValues(int numValues) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public int numValues() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public ValueVector getValueVector() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public CometVector slice(int offset, int length) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void close() {} + + @Override + public boolean hasNull() { + return false; + } + + @Override + public int numNulls() { + return 0; + } + + @Override + public boolean isNullAt(int rowId) { + return false; + } + + @Override + public boolean getBoolean(int rowId) { + return isDeleted[rowId]; + } + + @Override + public byte getByte(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public short getShort(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public int getInt(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public long getLong(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloat(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public double getDouble(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarArray getArray(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarMap getMap(int ordinal) { + throw new UnsupportedOperationException(); + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + throw new UnsupportedOperationException(); + } + + @Override + public UTF8String getUTF8String(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] getBinary(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnVector getChild(int ordinal) { + throw new UnsupportedOperationException(); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java index d36f1a727477..779dc240d4f6 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java @@ -24,7 +24,6 @@ import java.util.stream.IntStream; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; -import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.TypeWithSchemaVisitor; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -37,7 +36,6 @@ import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; -import org.apache.spark.sql.catalyst.InternalRow; class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor> { @@ -45,19 +43,16 @@ class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor idToConstant; private final Function>, VectorizedReader> readerFactory; - private final DeleteFilter deleteFilter; CometVectorizedReaderBuilder( Schema expectedSchema, MessageType parquetSchema, Map idToConstant, - Function>, VectorizedReader> readerFactory, - DeleteFilter deleteFilter) { + Function>, VectorizedReader> readerFactory) { this.parquetSchema = parquetSchema; this.icebergSchema = expectedSchema; this.idToConstant = idToConstant; this.readerFactory = readerFactory; - this.deleteFilter = deleteFilter; } @Override @@ -107,11 +102,7 @@ public VectorizedReader message( } protected VectorizedReader vectorizedReader(List> reorderedFields) { - VectorizedReader reader = readerFactory.apply(reorderedFields); - if (deleteFilter != null) { - ((CometColumnarBatchReader) reader).setDeleteFilter(deleteFilter); - } - return reader; + return readerFactory.apply(reorderedFields); } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java index a453247068d8..fa3bcfdd004e 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java @@ -26,13 +26,14 @@ import org.apache.spark.sql.vectorized.ColumnarMap; import org.apache.spark.unsafe.types.UTF8String; -public class DeletedColumnVector extends ColumnVector { +public class DeletedColumnVector extends ColumnVector implements UpdatableDeletedColumnVector { private boolean[] isDeleted; public DeletedColumnVector(Type type) { super(SparkSchemaUtil.convert(type)); } + @Override public void setValue(boolean[] deleted) { this.isDeleted = deleted; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java new file mode 100644 index 000000000000..99bedc42bf3c --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java @@ -0,0 +1,23 @@ +/* + * 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.data.vectorized; + +public interface UpdatableDeletedColumnVector { + void setValue(boolean[] isDeleted); +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java index d95baa724b82..8e25e81a05b2 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -26,12 +26,10 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.arrow.ArrowAllocation; import org.apache.iceberg.arrow.vectorized.VectorizedReaderBuilder; -import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.TypeWithSchemaVisitor; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.spark.SparkUtil; import org.apache.parquet.schema.MessageType; -import org.apache.spark.sql.catalyst.InternalRow; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,7 +56,6 @@ public static ColumnarBatchReader buildReader( Schema expectedSchema, MessageType fileSchema, Map idToConstant, - DeleteFilter deleteFilter, BufferAllocator bufferAllocator) { return (ColumnarBatchReader) TypeWithSchemaVisitor.visit( @@ -70,24 +67,16 @@ public static ColumnarBatchReader buildReader( NullCheckingForGet.NULL_CHECKING_ENABLED, idToConstant, ColumnarBatchReader::new, - deleteFilter, bufferAllocator)); } public static ColumnarBatchReader buildReader( - Schema expectedSchema, - MessageType fileSchema, - Map idToConstant, - DeleteFilter deleteFilter) { - return buildReader( - expectedSchema, fileSchema, idToConstant, deleteFilter, ArrowAllocation.rootAllocator()); + Schema expectedSchema, MessageType fileSchema, Map idToConstant) { + return buildReader(expectedSchema, fileSchema, idToConstant, ArrowAllocation.rootAllocator()); } public static CometColumnarBatchReader buildCometReader( - Schema expectedSchema, - MessageType fileSchema, - Map idToConstant, - DeleteFilter deleteFilter) { + Schema expectedSchema, MessageType fileSchema, Map idToConstant) { return (CometColumnarBatchReader) TypeWithSchemaVisitor.visit( expectedSchema.asStruct(), @@ -96,8 +85,7 @@ public static CometColumnarBatchReader buildCometReader( expectedSchema, fileSchema, idToConstant, - readers -> new CometColumnarBatchReader(readers, expectedSchema), - deleteFilter)); + readers -> new CometColumnarBatchReader(readers, expectedSchema))); } // enables unsafe memory access to avoid costly checks to see if index is within bounds @@ -134,7 +122,6 @@ private static String confValue(String propName, String envName) { } private static class ReaderBuilder extends VectorizedReaderBuilder { - private final DeleteFilter deleteFilter; ReaderBuilder( Schema expectedSchema, @@ -142,7 +129,6 @@ private static class ReaderBuilder extends VectorizedReaderBuilder { boolean setArrowValidityVector, Map idToConstant, Function>, VectorizedReader> readerFactory, - DeleteFilter deleteFilter, BufferAllocator bufferAllocator) { super( expectedSchema, @@ -152,16 +138,6 @@ private static class ReaderBuilder extends VectorizedReaderBuilder { readerFactory, SparkUtil::internalToSpark, bufferAllocator); - this.deleteFilter = deleteFilter; - } - - @Override - protected VectorizedReader vectorizedReader(List> reorderedFields) { - VectorizedReader reader = super.vectorizedReader(reorderedFields); - if (deleteFilter != null) { - ((ColumnarBatchReader) reader).setDeleteFilter(deleteFilter); - } - return reader; } } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index a0f45e7610a2..ff30f29aeae6 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -20,24 +20,33 @@ import java.util.Map; import java.util.Set; +import javax.annotation.Nonnull; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.OrcBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; import org.apache.iceberg.spark.ParquetReaderType; +import org.apache.iceberg.spark.data.vectorized.ColumnVectorWithFilter; +import org.apache.iceberg.spark.data.vectorized.ColumnarBatchUtil; +import org.apache.iceberg.spark.data.vectorized.UpdatableDeletedColumnVector; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; abstract class BaseBatchReader extends BaseReader { @@ -66,18 +75,23 @@ protected CloseableIterable newBatchIterable( long length, Expression residual, Map idToConstant, - SparkDeleteFilter deleteFilter) { + @Nonnull SparkDeleteFilter deleteFilter) { + CloseableIterable iterable; switch (format) { case PARQUET: - return newParquetIterable(inputFile, start, length, residual, idToConstant, deleteFilter); - + iterable = + newParquetIterable( + inputFile, start, length, residual, idToConstant, deleteFilter.requiredSchema()); + break; case ORC: - return newOrcIterable(inputFile, start, length, residual, idToConstant); - + iterable = newOrcIterable(inputFile, start, length, residual, idToConstant); + break; default: throw new UnsupportedOperationException( "Format: " + format + " not supported for batched reads"); } + + return CloseableIterable.transform(iterable, new BatchDeleteFilter(deleteFilter)::filterBatch); } private CloseableIterable newParquetIterable( @@ -86,10 +100,7 @@ private CloseableIterable newParquetIterable( long length, Expression residual, Map idToConstant, - SparkDeleteFilter deleteFilter) { - // get required schema if there are deletes - Schema requiredSchema = deleteFilter != null ? deleteFilter.requiredSchema() : expectedSchema(); - + Schema requiredSchema) { return Parquet.read(inputFile) .project(requiredSchema) .split(start, length) @@ -97,10 +108,10 @@ private CloseableIterable newParquetIterable( fileSchema -> { if (parquetConf.readerType() == ParquetReaderType.COMET) { return VectorizedSparkParquetReaders.buildCometReader( - requiredSchema, fileSchema, idToConstant, deleteFilter); + requiredSchema, fileSchema, idToConstant); } else { return VectorizedSparkParquetReaders.buildReader( - requiredSchema, fileSchema, idToConstant, deleteFilter); + requiredSchema, fileSchema, idToConstant); } }) .recordsPerBatch(parquetConf.batchSize()) @@ -139,4 +150,72 @@ private CloseableIterable newOrcIterable( .withNameMapping(nameMapping()) .build(); } + + @VisibleForTesting + static class BatchDeleteFilter { + private final DeleteFilter deletes; + private boolean hasIsDeletedColumn; + private int rowPositionColumnIndex = -1; + + BatchDeleteFilter(DeleteFilter deletes) { + this.deletes = deletes; + + Schema schema = deletes.requiredSchema(); + for (int i = 0; i < schema.columns().size(); i++) { + if (schema.columns().get(i).fieldId() == MetadataColumns.ROW_POSITION.fieldId()) { + this.rowPositionColumnIndex = i; + } else if (schema.columns().get(i).fieldId() == MetadataColumns.IS_DELETED.fieldId()) { + this.hasIsDeletedColumn = true; + } + } + } + + ColumnarBatch filterBatch(ColumnarBatch batch) { + if (!needDeletes()) { + return batch; + } + + ColumnVector[] vectors = new ColumnVector[batch.numCols()]; + for (int i = 0; i < batch.numCols(); i++) { + vectors[i] = batch.column(i); + } + + int numLiveRows = batch.numRows(); + long rowStartPosInBatch = + rowPositionColumnIndex == -1 ? -1 : vectors[rowPositionColumnIndex].getLong(0); + + if (hasIsDeletedColumn) { + boolean[] isDeleted = + ColumnarBatchUtil.buildIsDeleted(vectors, deletes, rowStartPosInBatch, numLiveRows); + for (ColumnVector vector : vectors) { + if (vector instanceof UpdatableDeletedColumnVector) { + ((UpdatableDeletedColumnVector) vector).setValue(isDeleted); + } + } + } else { + Pair pair = + ColumnarBatchUtil.buildRowIdMapping(vectors, deletes, rowStartPosInBatch, numLiveRows); + if (pair != null) { + int[] rowIdMapping = pair.first(); + numLiveRows = pair.second(); + for (int i = 0; i < vectors.length; i++) { + vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping); + } + } + } + + if (deletes != null && deletes.hasEqDeletes()) { + vectors = ColumnarBatchUtil.removeExtraColumns(deletes, vectors); + } + + ColumnarBatch output = new ColumnarBatch(vectors); + output.setNumRows(numLiveRows); + return output; + } + + private boolean needDeletes() { + return hasIsDeletedColumn + || (deletes != null && (deletes.hasEqDeletes() || deletes.hasPosDeletes())); + } + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index 20128ccef8b2..9ec0f885775f 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -105,15 +105,13 @@ protected CloseableIterator open(FileScanTask task) { // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); - Map idToConstant = constantsMap(task, expectedSchema()); - InputFile inputFile = getInputFile(filePath); Preconditions.checkNotNull(inputFile, "Could not find InputFile associated with FileScanTask"); SparkDeleteFilter deleteFilter = - task.deletes().isEmpty() - ? null - : new SparkDeleteFilter(filePath, task.deletes(), counter(), false); + new SparkDeleteFilter(filePath, task.deletes(), counter(), true); + + Map idToConstant = constantsMap(task, deleteFilter.requiredSchema()); return newBatchIterable( inputFile, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java index 044ea3d93c0b..ccd783915c62 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java @@ -21,8 +21,6 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; import java.io.File; import java.io.IOException; @@ -37,12 +35,15 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.deletes.DeleteCounter; import org.apache.iceberg.deletes.PositionDeleteIndex; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.InputFile; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.parquet.ParquetSchemaUtil; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -51,6 +52,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; +import org.apache.iceberg.spark.source.BatchReaderUtil; import org.apache.iceberg.types.Types; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.hadoop.ParquetFileReader; @@ -181,22 +183,52 @@ public void testReadRowNumbersWithDelete() throws IOException { Parquet.ReadBuilder builder = Parquet.read(Files.localInput(testFile)).project(PROJECTION_SCHEMA); - DeleteFilter deleteFilter = mock(DeleteFilter.class); - when(deleteFilter.hasPosDeletes()).thenReturn(true); - PositionDeleteIndex deletedRowPos = new CustomizedPositionDeleteIndex(); - deletedRowPos.delete(98, 103); - when(deleteFilter.deletedRowPositions()).thenReturn(deletedRowPos); + DeleteFilter deleteFilter = new TestDeleteFilter(true); builder.createBatchedReaderFunc( fileSchema -> VectorizedSparkParquetReaders.buildReader( - PROJECTION_SCHEMA, fileSchema, Maps.newHashMap(), deleteFilter)); + PROJECTION_SCHEMA, fileSchema, Maps.newHashMap())); builder.recordsPerBatch(RECORDS_PER_BATCH); - validate(expectedRowsAfterDelete, builder); + validate(expectedRowsAfterDelete, builder, deleteFilter); } - private class CustomizedPositionDeleteIndex implements PositionDeleteIndex { + private static class TestDeleteFilter extends DeleteFilter { + private final boolean hasDeletes; + + protected TestDeleteFilter(boolean hasDeletes) { + super("", List.of(), DATA_SCHEMA, PROJECTION_SCHEMA, new DeleteCounter(), true); + this.hasDeletes = hasDeletes; + } + + @Override + protected StructLike asStructLike(InternalRow record) { + return null; + } + + @Override + protected InputFile getInputFile(String location) { + return null; + } + + @Override + public boolean hasPosDeletes() { + return hasDeletes; + } + + @Override + public PositionDeleteIndex deletedRowPositions() { + PositionDeleteIndex deletedRowPos = new CustomizedPositionDeleteIndex(); + if (hasDeletes) { + deletedRowPos.delete(98, 103); + } + + return deletedRowPos; + } + } + + private static class CustomizedPositionDeleteIndex implements PositionDeleteIndex { private final Set deleteIndex; private CustomizedPositionDeleteIndex() { @@ -266,7 +298,7 @@ private void readAndValidate( builder.createBatchedReaderFunc( fileSchema -> VectorizedSparkParquetReaders.buildReader( - PROJECTION_SCHEMA, fileSchema, Maps.newHashMap(), null)); + PROJECTION_SCHEMA, fileSchema, Maps.newHashMap())); builder.recordsPerBatch(RECORDS_PER_BATCH); } else { builder = @@ -282,13 +314,13 @@ private void readAndValidate( builder = builder.split(splitStart, splitLength); } - validate(expected, builder); + validate(expected, builder, new TestDeleteFilter(false)); } - private void validate(List expected, Parquet.ReadBuilder builder) + private void validate( + List expected, Parquet.ReadBuilder builder, DeleteFilter filter) throws IOException { - try (CloseableIterable reader = - vectorized ? batchesToRows(builder.build()) : builder.build()) { + try (CloseableIterable reader = reader(builder, filter)) { final Iterator actualRows = reader.iterator(); for (InternalRow internalRow : expected) { @@ -300,6 +332,15 @@ private void validate(List expected, Parquet.ReadBuilder builder) } } + private CloseableIterable reader( + Parquet.ReadBuilder builder, DeleteFilter filter) { + if (!vectorized) { + return builder.build(); + } else { + return batchesToRows(BatchReaderUtil.applyDeleteFilter(builder.build(), filter)); + } + } + private CloseableIterable batchesToRows(CloseableIterable batches) { return CloseableIterable.combine( Iterables.concat(Iterables.transform(batches, b -> (Iterable) b::rowIterator)), diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java index 9062050f544d..6bff9010eb4f 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java @@ -174,7 +174,7 @@ void assertRecordsMatch( .createBatchedReaderFunc( type -> VectorizedSparkParquetReaders.buildReader( - schema, type, Maps.newHashMap(), null, allocator)); + schema, type, Maps.newHashMap(), allocator)); if (reuseContainers) { readBuilder.reuseContainers(); } @@ -204,8 +204,7 @@ public void testNestedStruct() { new Schema(required(1, "struct", SUPPORTED_PRIMITIVES))), new MessageType( "struct", new GroupType(Type.Repetition.OPTIONAL, "struct").withId(1)), - Maps.newHashMap(), - null)) + Maps.newHashMap())) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Vectorized reads are not supported yet for struct fields"); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java new file mode 100644 index 000000000000..e5d03a4efba5 --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java @@ -0,0 +1,34 @@ +/* + * 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.data.DeleteFilter; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +public class BatchReaderUtil { + private BatchReaderUtil() {} + + public static CloseableIterable applyDeleteFilter( + CloseableIterable batches, DeleteFilter filter) { + return CloseableIterable.transform( + batches, new BaseBatchReader.BatchDeleteFilter(filter)::filterBatch); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java index a0670c2bbe71..38d505d250d0 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java @@ -22,15 +22,11 @@ import java.util.Map; import org.apache.iceberg.arrow.vectorized.BaseBatchReader; import org.apache.iceberg.arrow.vectorized.VectorizedArrowReader; -import org.apache.iceberg.arrow.vectorized.VectorizedArrowReader.DeletedVectorReader; -import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.util.Pair; import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; @@ -40,31 +36,15 @@ * populated via delegated read calls to {@linkplain VectorizedArrowReader VectorReader(s)}. */ public class ColumnarBatchReader extends BaseBatchReader { - private final boolean hasIsDeletedColumn; - private DeleteFilter deletes = null; - private long rowStartPosInBatch = 0; public ColumnarBatchReader(List> readers) { super(readers); - this.hasIsDeletedColumn = - readers.stream().anyMatch(reader -> reader instanceof DeletedVectorReader); } @Override public void setRowGroupInfo( PageReadStore pageStore, Map metaData) { super.setRowGroupInfo(pageStore, metaData); - this.rowStartPosInBatch = - pageStore - .getRowIndexOffset() - .orElseThrow( - () -> - new IllegalArgumentException( - "PageReadStore does not contain row index offset")); - } - - public void setDeleteFilter(DeleteFilter deleteFilter) { - this.deletes = deleteFilter; } @Override @@ -73,9 +53,7 @@ public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) { closeVectors(); } - ColumnarBatch columnarBatch = new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); - rowStartPosInBatch += numRowsToRead; - return columnarBatch; + return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); } private class ColumnBatchLoader { @@ -89,43 +67,12 @@ private class ColumnBatchLoader { ColumnarBatch loadDataToColumnBatch() { ColumnVector[] vectors = readDataToColumnVectors(); - int numLiveRows = batchSize; - - if (hasIsDeletedColumn) { - boolean[] isDeleted = buildIsDeleted(vectors); - for (ColumnVector vector : vectors) { - if (vector instanceof DeletedColumnVector) { - ((DeletedColumnVector) vector).setValue(isDeleted); - } - } - } else { - Pair pair = buildRowIdMapping(vectors); - if (pair != null) { - int[] rowIdMapping = pair.first(); - numLiveRows = pair.second(); - for (int i = 0; i < vectors.length; i++) { - vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping); - } - } - } - - if (deletes != null && deletes.hasEqDeletes()) { - vectors = ColumnarBatchUtil.removeExtraColumns(deletes, vectors); - } ColumnarBatch batch = new ColumnarBatch(vectors); - batch.setNumRows(numLiveRows); + batch.setNumRows(batchSize); return batch; } - private boolean[] buildIsDeleted(ColumnVector[] vectors) { - return ColumnarBatchUtil.buildIsDeleted(vectors, deletes, rowStartPosInBatch, batchSize); - } - - private Pair buildRowIdMapping(ColumnVector[] vectors) { - return ColumnarBatchUtil.buildRowIdMapping(vectors, deletes, rowStartPosInBatch, batchSize); - } - ColumnVector[] readDataToColumnVectors() { ColumnVector[] arrowColumnVectors = new ColumnVector[readers.length]; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java index 04ac69476add..3d3e9aca24de 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java @@ -25,15 +25,12 @@ import org.apache.comet.parquet.AbstractColumnReader; import org.apache.comet.parquet.BatchReader; import org.apache.iceberg.Schema; -import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.util.Pair; import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; @@ -46,7 +43,6 @@ class CometColumnarBatchReader implements VectorizedReader { private final CometColumnReader[] readers; - private final boolean hasIsDeletedColumn; // The delegated BatchReader on the Comet side does the real work of loading a batch of rows. // The Comet BatchReader contains an array of ColumnReader. There is no need to explicitly call @@ -56,14 +52,10 @@ class CometColumnarBatchReader implements VectorizedReader { // DeleteColumnReader.readBatch must be called explicitly later, after the isDeleted value is // available. private final BatchReader delegate; - private DeleteFilter deletes = null; - private long rowStartPosInBatch = 0; CometColumnarBatchReader(List> readers, Schema schema) { this.readers = readers.stream().map(CometColumnReader.class::cast).toArray(CometColumnReader[]::new); - this.hasIsDeletedColumn = - readers.stream().anyMatch(reader -> reader instanceof CometDeleteColumnReader); AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; this.delegate = new BatchReader(abstractColumnReaders); @@ -89,25 +81,11 @@ public void setRowGroupInfo( for (int i = 0; i < readers.length; i++) { delegate.getColumnReaders()[i] = this.readers[i].delegate(); } - - this.rowStartPosInBatch = - pageStore - .getRowIndexOffset() - .orElseThrow( - () -> - new IllegalArgumentException( - "PageReadStore does not contain row index offset")); - } - - public void setDeleteFilter(DeleteFilter deleteFilter) { - this.deletes = deleteFilter; } @Override public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) { - ColumnarBatch columnarBatch = new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); - rowStartPosInBatch += numRowsToRead; - return columnarBatch; + return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); } @Override @@ -139,39 +117,12 @@ private class ColumnBatchLoader { ColumnarBatch loadDataToColumnBatch() { ColumnVector[] vectors = readDataToColumnVectors(); - int numLiveRows = batchSize; - - if (hasIsDeletedColumn) { - boolean[] isDeleted = buildIsDeleted(vectors); - readDeletedColumn(vectors, isDeleted); - } else { - Pair pair = buildRowIdMapping(vectors); - if (pair != null) { - int[] rowIdMapping = pair.first(); - numLiveRows = pair.second(); - for (int i = 0; i < vectors.length; i++) { - vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping); - } - } - } - - if (deletes != null && deletes.hasEqDeletes()) { - vectors = ColumnarBatchUtil.removeExtraColumns(deletes, vectors); - } ColumnarBatch batch = new ColumnarBatch(vectors); - batch.setNumRows(numLiveRows); + batch.setNumRows(batchSize); return batch; } - private boolean[] buildIsDeleted(ColumnVector[] vectors) { - return ColumnarBatchUtil.buildIsDeleted(vectors, deletes, rowStartPosInBatch, batchSize); - } - - private Pair buildRowIdMapping(ColumnVector[] vectors) { - return ColumnarBatchUtil.buildRowIdMapping(vectors, deletes, rowStartPosInBatch, batchSize); - } - ColumnVector[] readDataToColumnVectors() { ColumnVector[] columnVectors = new ColumnVector[readers.length]; // Fetch rows for all readers in the delegate @@ -182,16 +133,5 @@ ColumnVector[] readDataToColumnVectors() { return columnVectors; } - - void readDeletedColumn(ColumnVector[] columnVectors, boolean[] isDeleted) { - for (int i = 0; i < readers.length; i++) { - if (readers[i] instanceof CometDeleteColumnReader) { - CometDeleteColumnReader deleteColumnReader = new CometDeleteColumnReader<>(isDeleted); - deleteColumnReader.setBatchSize(batchSize); - deleteColumnReader.delegate().readBatch(batchSize); - columnVectors[i] = deleteColumnReader.delegate().currentBatch(); - } - } - } } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java index 6235bfe4865e..26219014f777 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java @@ -21,7 +21,7 @@ import org.apache.comet.parquet.MetadataColumnReader; import org.apache.comet.parquet.Native; import org.apache.comet.parquet.TypeUtil; -import org.apache.iceberg.MetadataColumns; +import org.apache.comet.vector.CometVector; import org.apache.iceberg.types.Types; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.Metadata; @@ -33,11 +33,6 @@ class CometDeleteColumnReader extends CometColumnReader { setDelegate(new DeleteColumnReader()); } - CometDeleteColumnReader(boolean[] isDeleted) { - super(MetadataColumns.IS_DELETED); - setDelegate(new DeleteColumnReader(isDeleted)); - } - @Override public void setBatchSize(int batchSize) { super.setBatchSize(batchSize); @@ -46,30 +41,34 @@ public void setBatchSize(int batchSize) { } private static class DeleteColumnReader extends MetadataColumnReader { - private boolean[] isDeleted; + private final CometDeletedColumnVector deletedVector; DeleteColumnReader() { + this(new boolean[0]); + } + + DeleteColumnReader(boolean[] isDeleted) { super( DataTypes.BooleanType, TypeUtil.convertToParquet( new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), false /* useDecimal128 = false */, false /* isConstant = false */); - this.isDeleted = new boolean[0]; - } - - DeleteColumnReader(boolean[] isDeleted) { - this(); - this.isDeleted = isDeleted; + this.deletedVector = new CometDeletedColumnVector(isDeleted); } @Override public void readBatch(int total) { Native.resetBatch(nativeHandle); // set isDeleted on the native side to be consumed by native execution - Native.setIsDeleted(nativeHandle, isDeleted); + Native.setIsDeleted(nativeHandle, deletedVector.isDeleted()); super.readBatch(total); } + + @Override + public CometVector currentBatch() { + return deletedVector; + } } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java new file mode 100644 index 000000000000..5817f2c20a4f --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java @@ -0,0 +1,155 @@ +/* + * 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.data.vectorized; + +import org.apache.comet.shaded.arrow.vector.ValueVector; +import org.apache.comet.vector.CometVector; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +public class CometDeletedColumnVector extends CometVector implements UpdatableDeletedColumnVector { + private boolean[] isDeleted; + + public CometDeletedColumnVector(boolean[] isDeleted) { + super(SparkSchemaUtil.convert(Types.BooleanType.get()), false); + this.isDeleted = isDeleted; + } + + @Override + public void setValue(boolean[] deleted) { + this.isDeleted = deleted; + } + + boolean[] isDeleted() { + return isDeleted; + } + + @Override + public void setNumNulls(int numNulls) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void setNumValues(int numValues) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public int numValues() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public ValueVector getValueVector() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public CometVector slice(int offset, int length) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void close() {} + + @Override + public boolean hasNull() { + return false; + } + + @Override + public int numNulls() { + return 0; + } + + @Override + public boolean isNullAt(int rowId) { + return false; + } + + @Override + public boolean getBoolean(int rowId) { + return isDeleted[rowId]; + } + + @Override + public byte getByte(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public short getShort(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public int getInt(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public long getLong(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloat(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public double getDouble(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarArray getArray(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarMap getMap(int ordinal) { + throw new UnsupportedOperationException(); + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + throw new UnsupportedOperationException(); + } + + @Override + public UTF8String getUTF8String(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] getBinary(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnVector getChild(int ordinal) { + throw new UnsupportedOperationException(); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java index d36f1a727477..779dc240d4f6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java @@ -24,7 +24,6 @@ import java.util.stream.IntStream; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; -import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.TypeWithSchemaVisitor; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -37,7 +36,6 @@ import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; -import org.apache.spark.sql.catalyst.InternalRow; class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor> { @@ -45,19 +43,16 @@ class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor idToConstant; private final Function>, VectorizedReader> readerFactory; - private final DeleteFilter deleteFilter; CometVectorizedReaderBuilder( Schema expectedSchema, MessageType parquetSchema, Map idToConstant, - Function>, VectorizedReader> readerFactory, - DeleteFilter deleteFilter) { + Function>, VectorizedReader> readerFactory) { this.parquetSchema = parquetSchema; this.icebergSchema = expectedSchema; this.idToConstant = idToConstant; this.readerFactory = readerFactory; - this.deleteFilter = deleteFilter; } @Override @@ -107,11 +102,7 @@ public VectorizedReader message( } protected VectorizedReader vectorizedReader(List> reorderedFields) { - VectorizedReader reader = readerFactory.apply(reorderedFields); - if (deleteFilter != null) { - ((CometColumnarBatchReader) reader).setDeleteFilter(deleteFilter); - } - return reader; + return readerFactory.apply(reorderedFields); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java index a453247068d8..fa3bcfdd004e 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java @@ -26,13 +26,14 @@ import org.apache.spark.sql.vectorized.ColumnarMap; import org.apache.spark.unsafe.types.UTF8String; -public class DeletedColumnVector extends ColumnVector { +public class DeletedColumnVector extends ColumnVector implements UpdatableDeletedColumnVector { private boolean[] isDeleted; public DeletedColumnVector(Type type) { super(SparkSchemaUtil.convert(type)); } + @Override public void setValue(boolean[] deleted) { this.isDeleted = deleted; } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java new file mode 100644 index 000000000000..99bedc42bf3c --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java @@ -0,0 +1,23 @@ +/* + * 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.data.vectorized; + +public interface UpdatableDeletedColumnVector { + void setValue(boolean[] isDeleted); +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java index d95baa724b82..8e25e81a05b2 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -26,12 +26,10 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.arrow.ArrowAllocation; import org.apache.iceberg.arrow.vectorized.VectorizedReaderBuilder; -import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.TypeWithSchemaVisitor; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.spark.SparkUtil; import org.apache.parquet.schema.MessageType; -import org.apache.spark.sql.catalyst.InternalRow; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,7 +56,6 @@ public static ColumnarBatchReader buildReader( Schema expectedSchema, MessageType fileSchema, Map idToConstant, - DeleteFilter deleteFilter, BufferAllocator bufferAllocator) { return (ColumnarBatchReader) TypeWithSchemaVisitor.visit( @@ -70,24 +67,16 @@ public static ColumnarBatchReader buildReader( NullCheckingForGet.NULL_CHECKING_ENABLED, idToConstant, ColumnarBatchReader::new, - deleteFilter, bufferAllocator)); } public static ColumnarBatchReader buildReader( - Schema expectedSchema, - MessageType fileSchema, - Map idToConstant, - DeleteFilter deleteFilter) { - return buildReader( - expectedSchema, fileSchema, idToConstant, deleteFilter, ArrowAllocation.rootAllocator()); + Schema expectedSchema, MessageType fileSchema, Map idToConstant) { + return buildReader(expectedSchema, fileSchema, idToConstant, ArrowAllocation.rootAllocator()); } public static CometColumnarBatchReader buildCometReader( - Schema expectedSchema, - MessageType fileSchema, - Map idToConstant, - DeleteFilter deleteFilter) { + Schema expectedSchema, MessageType fileSchema, Map idToConstant) { return (CometColumnarBatchReader) TypeWithSchemaVisitor.visit( expectedSchema.asStruct(), @@ -96,8 +85,7 @@ public static CometColumnarBatchReader buildCometReader( expectedSchema, fileSchema, idToConstant, - readers -> new CometColumnarBatchReader(readers, expectedSchema), - deleteFilter)); + readers -> new CometColumnarBatchReader(readers, expectedSchema))); } // enables unsafe memory access to avoid costly checks to see if index is within bounds @@ -134,7 +122,6 @@ private static String confValue(String propName, String envName) { } private static class ReaderBuilder extends VectorizedReaderBuilder { - private final DeleteFilter deleteFilter; ReaderBuilder( Schema expectedSchema, @@ -142,7 +129,6 @@ private static class ReaderBuilder extends VectorizedReaderBuilder { boolean setArrowValidityVector, Map idToConstant, Function>, VectorizedReader> readerFactory, - DeleteFilter deleteFilter, BufferAllocator bufferAllocator) { super( expectedSchema, @@ -152,16 +138,6 @@ private static class ReaderBuilder extends VectorizedReaderBuilder { readerFactory, SparkUtil::internalToSpark, bufferAllocator); - this.deleteFilter = deleteFilter; - } - - @Override - protected VectorizedReader vectorizedReader(List> reorderedFields) { - VectorizedReader reader = super.vectorizedReader(reorderedFields); - if (deleteFilter != null) { - ((ColumnarBatchReader) reader).setDeleteFilter(deleteFilter); - } - return reader; } } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index a0f45e7610a2..ff30f29aeae6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -20,24 +20,33 @@ import java.util.Map; import java.util.Set; +import javax.annotation.Nonnull; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.OrcBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; import org.apache.iceberg.spark.ParquetReaderType; +import org.apache.iceberg.spark.data.vectorized.ColumnVectorWithFilter; +import org.apache.iceberg.spark.data.vectorized.ColumnarBatchUtil; +import org.apache.iceberg.spark.data.vectorized.UpdatableDeletedColumnVector; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; abstract class BaseBatchReader extends BaseReader { @@ -66,18 +75,23 @@ protected CloseableIterable newBatchIterable( long length, Expression residual, Map idToConstant, - SparkDeleteFilter deleteFilter) { + @Nonnull SparkDeleteFilter deleteFilter) { + CloseableIterable iterable; switch (format) { case PARQUET: - return newParquetIterable(inputFile, start, length, residual, idToConstant, deleteFilter); - + iterable = + newParquetIterable( + inputFile, start, length, residual, idToConstant, deleteFilter.requiredSchema()); + break; case ORC: - return newOrcIterable(inputFile, start, length, residual, idToConstant); - + iterable = newOrcIterable(inputFile, start, length, residual, idToConstant); + break; default: throw new UnsupportedOperationException( "Format: " + format + " not supported for batched reads"); } + + return CloseableIterable.transform(iterable, new BatchDeleteFilter(deleteFilter)::filterBatch); } private CloseableIterable newParquetIterable( @@ -86,10 +100,7 @@ private CloseableIterable newParquetIterable( long length, Expression residual, Map idToConstant, - SparkDeleteFilter deleteFilter) { - // get required schema if there are deletes - Schema requiredSchema = deleteFilter != null ? deleteFilter.requiredSchema() : expectedSchema(); - + Schema requiredSchema) { return Parquet.read(inputFile) .project(requiredSchema) .split(start, length) @@ -97,10 +108,10 @@ private CloseableIterable newParquetIterable( fileSchema -> { if (parquetConf.readerType() == ParquetReaderType.COMET) { return VectorizedSparkParquetReaders.buildCometReader( - requiredSchema, fileSchema, idToConstant, deleteFilter); + requiredSchema, fileSchema, idToConstant); } else { return VectorizedSparkParquetReaders.buildReader( - requiredSchema, fileSchema, idToConstant, deleteFilter); + requiredSchema, fileSchema, idToConstant); } }) .recordsPerBatch(parquetConf.batchSize()) @@ -139,4 +150,72 @@ private CloseableIterable newOrcIterable( .withNameMapping(nameMapping()) .build(); } + + @VisibleForTesting + static class BatchDeleteFilter { + private final DeleteFilter deletes; + private boolean hasIsDeletedColumn; + private int rowPositionColumnIndex = -1; + + BatchDeleteFilter(DeleteFilter deletes) { + this.deletes = deletes; + + Schema schema = deletes.requiredSchema(); + for (int i = 0; i < schema.columns().size(); i++) { + if (schema.columns().get(i).fieldId() == MetadataColumns.ROW_POSITION.fieldId()) { + this.rowPositionColumnIndex = i; + } else if (schema.columns().get(i).fieldId() == MetadataColumns.IS_DELETED.fieldId()) { + this.hasIsDeletedColumn = true; + } + } + } + + ColumnarBatch filterBatch(ColumnarBatch batch) { + if (!needDeletes()) { + return batch; + } + + ColumnVector[] vectors = new ColumnVector[batch.numCols()]; + for (int i = 0; i < batch.numCols(); i++) { + vectors[i] = batch.column(i); + } + + int numLiveRows = batch.numRows(); + long rowStartPosInBatch = + rowPositionColumnIndex == -1 ? -1 : vectors[rowPositionColumnIndex].getLong(0); + + if (hasIsDeletedColumn) { + boolean[] isDeleted = + ColumnarBatchUtil.buildIsDeleted(vectors, deletes, rowStartPosInBatch, numLiveRows); + for (ColumnVector vector : vectors) { + if (vector instanceof UpdatableDeletedColumnVector) { + ((UpdatableDeletedColumnVector) vector).setValue(isDeleted); + } + } + } else { + Pair pair = + ColumnarBatchUtil.buildRowIdMapping(vectors, deletes, rowStartPosInBatch, numLiveRows); + if (pair != null) { + int[] rowIdMapping = pair.first(); + numLiveRows = pair.second(); + for (int i = 0; i < vectors.length; i++) { + vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping); + } + } + } + + if (deletes != null && deletes.hasEqDeletes()) { + vectors = ColumnarBatchUtil.removeExtraColumns(deletes, vectors); + } + + ColumnarBatch output = new ColumnarBatch(vectors); + output.setNumRows(numLiveRows); + return output; + } + + private boolean needDeletes() { + return hasIsDeletedColumn + || (deletes != null && (deletes.hasEqDeletes() || deletes.hasPosDeletes())); + } + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index 20128ccef8b2..9ec0f885775f 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -105,15 +105,13 @@ protected CloseableIterator open(FileScanTask task) { // update the current file for Spark's filename() function InputFileBlockHolder.set(filePath, task.start(), task.length()); - Map idToConstant = constantsMap(task, expectedSchema()); - InputFile inputFile = getInputFile(filePath); Preconditions.checkNotNull(inputFile, "Could not find InputFile associated with FileScanTask"); SparkDeleteFilter deleteFilter = - task.deletes().isEmpty() - ? null - : new SparkDeleteFilter(filePath, task.deletes(), counter(), false); + new SparkDeleteFilter(filePath, task.deletes(), counter(), true); + + Map idToConstant = constantsMap(task, deleteFilter.requiredSchema()); return newBatchIterable( inputFile, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java index 044ea3d93c0b..ccd783915c62 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java @@ -21,8 +21,6 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; import java.io.File; import java.io.IOException; @@ -37,12 +35,15 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.deletes.DeleteCounter; import org.apache.iceberg.deletes.PositionDeleteIndex; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.InputFile; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.parquet.ParquetSchemaUtil; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -51,6 +52,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; +import org.apache.iceberg.spark.source.BatchReaderUtil; import org.apache.iceberg.types.Types; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.hadoop.ParquetFileReader; @@ -181,22 +183,52 @@ public void testReadRowNumbersWithDelete() throws IOException { Parquet.ReadBuilder builder = Parquet.read(Files.localInput(testFile)).project(PROJECTION_SCHEMA); - DeleteFilter deleteFilter = mock(DeleteFilter.class); - when(deleteFilter.hasPosDeletes()).thenReturn(true); - PositionDeleteIndex deletedRowPos = new CustomizedPositionDeleteIndex(); - deletedRowPos.delete(98, 103); - when(deleteFilter.deletedRowPositions()).thenReturn(deletedRowPos); + DeleteFilter deleteFilter = new TestDeleteFilter(true); builder.createBatchedReaderFunc( fileSchema -> VectorizedSparkParquetReaders.buildReader( - PROJECTION_SCHEMA, fileSchema, Maps.newHashMap(), deleteFilter)); + PROJECTION_SCHEMA, fileSchema, Maps.newHashMap())); builder.recordsPerBatch(RECORDS_PER_BATCH); - validate(expectedRowsAfterDelete, builder); + validate(expectedRowsAfterDelete, builder, deleteFilter); } - private class CustomizedPositionDeleteIndex implements PositionDeleteIndex { + private static class TestDeleteFilter extends DeleteFilter { + private final boolean hasDeletes; + + protected TestDeleteFilter(boolean hasDeletes) { + super("", List.of(), DATA_SCHEMA, PROJECTION_SCHEMA, new DeleteCounter(), true); + this.hasDeletes = hasDeletes; + } + + @Override + protected StructLike asStructLike(InternalRow record) { + return null; + } + + @Override + protected InputFile getInputFile(String location) { + return null; + } + + @Override + public boolean hasPosDeletes() { + return hasDeletes; + } + + @Override + public PositionDeleteIndex deletedRowPositions() { + PositionDeleteIndex deletedRowPos = new CustomizedPositionDeleteIndex(); + if (hasDeletes) { + deletedRowPos.delete(98, 103); + } + + return deletedRowPos; + } + } + + private static class CustomizedPositionDeleteIndex implements PositionDeleteIndex { private final Set deleteIndex; private CustomizedPositionDeleteIndex() { @@ -266,7 +298,7 @@ private void readAndValidate( builder.createBatchedReaderFunc( fileSchema -> VectorizedSparkParquetReaders.buildReader( - PROJECTION_SCHEMA, fileSchema, Maps.newHashMap(), null)); + PROJECTION_SCHEMA, fileSchema, Maps.newHashMap())); builder.recordsPerBatch(RECORDS_PER_BATCH); } else { builder = @@ -282,13 +314,13 @@ private void readAndValidate( builder = builder.split(splitStart, splitLength); } - validate(expected, builder); + validate(expected, builder, new TestDeleteFilter(false)); } - private void validate(List expected, Parquet.ReadBuilder builder) + private void validate( + List expected, Parquet.ReadBuilder builder, DeleteFilter filter) throws IOException { - try (CloseableIterable reader = - vectorized ? batchesToRows(builder.build()) : builder.build()) { + try (CloseableIterable reader = reader(builder, filter)) { final Iterator actualRows = reader.iterator(); for (InternalRow internalRow : expected) { @@ -300,6 +332,15 @@ private void validate(List expected, Parquet.ReadBuilder builder) } } + private CloseableIterable reader( + Parquet.ReadBuilder builder, DeleteFilter filter) { + if (!vectorized) { + return builder.build(); + } else { + return batchesToRows(BatchReaderUtil.applyDeleteFilter(builder.build(), filter)); + } + } + private CloseableIterable batchesToRows(CloseableIterable batches) { return CloseableIterable.combine( Iterables.concat(Iterables.transform(batches, b -> (Iterable) b::rowIterator)), diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java index 973a17c9a388..284fa0b0552f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java @@ -179,7 +179,7 @@ public void testDecimalNotAllPagesDictionaryEncoded() throws Exception { .createBatchedReaderFunc( type -> VectorizedSparkParquetReaders.buildReader( - schema, type, ImmutableMap.of(), null, allocator)); + schema, type, ImmutableMap.of(), allocator)); try (CloseableIterable batchReader = readBuilder.build()) { Iterator expectedIter = expected.iterator(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java index e664db554bc0..6e823a8bfc05 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java @@ -277,7 +277,7 @@ void assertRecordsMatch( .createBatchedReaderFunc( type -> VectorizedSparkParquetReaders.buildReader( - schema, type, idToConstant, null, allocator)); + schema, type, idToConstant, allocator)); if (reuseContainers) { readBuilder.reuseContainers(); } @@ -308,8 +308,7 @@ public void testNestedStruct() { new Schema(required(1, "struct", SUPPORTED_PRIMITIVES))), new MessageType( "struct", new GroupType(Type.Repetition.OPTIONAL, "struct").withId(1)), - Maps.newHashMap(), - null)) + Maps.newHashMap())) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Vectorized reads are not supported yet for struct fields"); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java new file mode 100644 index 000000000000..e5d03a4efba5 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java @@ -0,0 +1,34 @@ +/* + * 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.data.DeleteFilter; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +public class BatchReaderUtil { + private BatchReaderUtil() {} + + public static CloseableIterable applyDeleteFilter( + CloseableIterable batches, DeleteFilter filter) { + return CloseableIterable.transform( + batches, new BaseBatchReader.BatchDeleteFilter(filter)::filterBatch); + } +} From 667bc595476c6e991fba2c1544e72926dd82fc42 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Thu, 4 Dec 2025 10:26:02 +0100 Subject: [PATCH 038/201] Flink: Dynamic Sink: Document writeParallelism and fail on invalid configuration (#14191) --- .../flink/sink/dynamic/DynamicRecord.java | 14 +++++++++ .../flink/sink/dynamic/HashKeyGenerator.java | 22 +++++++------- .../sink/dynamic/TestHashKeyGenerator.java | 29 +++++++++++++++++++ 3 files changed, 55 insertions(+), 10 deletions(-) diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java index 600a4d8b950c..9f445766083e 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java @@ -39,6 +39,20 @@ public class DynamicRecord { private boolean upsertMode; @Nullable private Set equalityFields; + /** + * Constructs a new DynamicRecord. + * + * @param tableIdentifier The target table identifier. + * @param branch The target table branch. + * @param schema The target table schema. + * @param rowData The data matching the provided schema. + * @param partitionSpec The target table {@link PartitionSpec}. + * @param distributionMode The {@link DistributionMode}. + * @param writeParallelism The number of parallel writers. Can be set to any value {@literal > 0}, + * but will always be automatically capped by the maximum write parallelism, which is the + * parallelism of the sink. Set to Integer.MAX_VALUE for always using the maximum available + * write parallelism. + */ public DynamicRecord( TableIdentifier tableIdentifier, String branch, diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java index 91aa4a91710c..1c611c46b90c 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -99,7 +99,7 @@ int generateKey( dynamicRecord.distributionMode(), DistributionMode.NONE), MoreObjects.firstNonNull( dynamicRecord.equalityFields(), Collections.emptySet()), - dynamicRecord.writeParallelism())); + Math.min(dynamicRecord.writeParallelism(), maxWriteParallelism))); try { return keySelector.getKey( overrideRowData != null ? overrideRowData : dynamicRecord.rowData()); @@ -242,15 +242,17 @@ private static class TargetLimitedKeySelector implements KeySelector maxWriteParallelism) { - LOG.warn( - "{}: writeParallelism {} is greater than maxWriteParallelism {}. Capping writeParallelism at {}", - tableName, - writeParallelism, - maxWriteParallelism, - maxWriteParallelism); - writeParallelism = maxWriteParallelism; - } + Preconditions.checkArgument( + writeParallelism > 0, + "%s: writeParallelism must be > 0 (is: %s)", + tableName, + writeParallelism); + Preconditions.checkArgument( + writeParallelism <= maxWriteParallelism, + "%s: writeParallelism (%s) must be <= maxWriteParallelism (%s)", + tableName, + writeParallelism, + maxWriteParallelism); this.wrapped = wrapped; this.writeParallelism = writeParallelism; this.distinctKeys = new int[writeParallelism]; diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java index 8d559e920620..04246bf03996 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.dynamic; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Collections; import java.util.Map; @@ -157,6 +158,34 @@ void testEqualityKeys() throws Exception { assertThat(getSubTaskId(writeKey3, writeParallelism, maxWriteParallelism)).isEqualTo(0); } + @Test + void testFailOnNonPositiveWriteParallelism() { + final int maxWriteParallelism = 5; + HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); + + assertThatThrownBy( + () -> { + getWriteKey( + generator, + PartitionSpec.unpartitioned(), + DistributionMode.NONE, + -1, // writeParallelism + Collections.emptySet(), + GenericRowData.of()); + }); + + assertThatThrownBy( + () -> { + getWriteKey( + generator, + PartitionSpec.unpartitioned(), + DistributionMode.NONE, + 0, // writeParallelism + Collections.emptySet(), + GenericRowData.of()); + }); + } + @Test void testCapAtMaxWriteParallelism() throws Exception { int writeParallelism = 10; From 86e53a7e83d1604b25535cf8ce34c39e8fd4fa1f Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Thu, 4 Dec 2025 11:25:41 +0100 Subject: [PATCH 039/201] Flink: Backport: Dynamic Sink: Document writeParallelism and fail on invalid configuration (#14758) backports #14191 --- .../flink/sink/dynamic/DynamicRecord.java | 14 +++++++++ .../flink/sink/dynamic/HashKeyGenerator.java | 22 +++++++------- .../sink/dynamic/TestHashKeyGenerator.java | 29 +++++++++++++++++++ .../flink/sink/dynamic/DynamicRecord.java | 14 +++++++++ .../flink/sink/dynamic/HashKeyGenerator.java | 22 +++++++------- .../sink/dynamic/TestHashKeyGenerator.java | 29 +++++++++++++++++++ 6 files changed, 110 insertions(+), 20 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java index 600a4d8b950c..9f445766083e 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java @@ -39,6 +39,20 @@ public class DynamicRecord { private boolean upsertMode; @Nullable private Set equalityFields; + /** + * Constructs a new DynamicRecord. + * + * @param tableIdentifier The target table identifier. + * @param branch The target table branch. + * @param schema The target table schema. + * @param rowData The data matching the provided schema. + * @param partitionSpec The target table {@link PartitionSpec}. + * @param distributionMode The {@link DistributionMode}. + * @param writeParallelism The number of parallel writers. Can be set to any value {@literal > 0}, + * but will always be automatically capped by the maximum write parallelism, which is the + * parallelism of the sink. Set to Integer.MAX_VALUE for always using the maximum available + * write parallelism. + */ public DynamicRecord( TableIdentifier tableIdentifier, String branch, diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java index 91aa4a91710c..1c611c46b90c 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -99,7 +99,7 @@ int generateKey( dynamicRecord.distributionMode(), DistributionMode.NONE), MoreObjects.firstNonNull( dynamicRecord.equalityFields(), Collections.emptySet()), - dynamicRecord.writeParallelism())); + Math.min(dynamicRecord.writeParallelism(), maxWriteParallelism))); try { return keySelector.getKey( overrideRowData != null ? overrideRowData : dynamicRecord.rowData()); @@ -242,15 +242,17 @@ private static class TargetLimitedKeySelector implements KeySelector maxWriteParallelism) { - LOG.warn( - "{}: writeParallelism {} is greater than maxWriteParallelism {}. Capping writeParallelism at {}", - tableName, - writeParallelism, - maxWriteParallelism, - maxWriteParallelism); - writeParallelism = maxWriteParallelism; - } + Preconditions.checkArgument( + writeParallelism > 0, + "%s: writeParallelism must be > 0 (is: %s)", + tableName, + writeParallelism); + Preconditions.checkArgument( + writeParallelism <= maxWriteParallelism, + "%s: writeParallelism (%s) must be <= maxWriteParallelism (%s)", + tableName, + writeParallelism, + maxWriteParallelism); this.wrapped = wrapped; this.writeParallelism = writeParallelism; this.distinctKeys = new int[writeParallelism]; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java index 8d559e920620..04246bf03996 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.dynamic; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Collections; import java.util.Map; @@ -157,6 +158,34 @@ void testEqualityKeys() throws Exception { assertThat(getSubTaskId(writeKey3, writeParallelism, maxWriteParallelism)).isEqualTo(0); } + @Test + void testFailOnNonPositiveWriteParallelism() { + final int maxWriteParallelism = 5; + HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); + + assertThatThrownBy( + () -> { + getWriteKey( + generator, + PartitionSpec.unpartitioned(), + DistributionMode.NONE, + -1, // writeParallelism + Collections.emptySet(), + GenericRowData.of()); + }); + + assertThatThrownBy( + () -> { + getWriteKey( + generator, + PartitionSpec.unpartitioned(), + DistributionMode.NONE, + 0, // writeParallelism + Collections.emptySet(), + GenericRowData.of()); + }); + } + @Test void testCapAtMaxWriteParallelism() throws Exception { int writeParallelism = 10; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java index 600a4d8b950c..9f445766083e 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecord.java @@ -39,6 +39,20 @@ public class DynamicRecord { private boolean upsertMode; @Nullable private Set equalityFields; + /** + * Constructs a new DynamicRecord. + * + * @param tableIdentifier The target table identifier. + * @param branch The target table branch. + * @param schema The target table schema. + * @param rowData The data matching the provided schema. + * @param partitionSpec The target table {@link PartitionSpec}. + * @param distributionMode The {@link DistributionMode}. + * @param writeParallelism The number of parallel writers. Can be set to any value {@literal > 0}, + * but will always be automatically capped by the maximum write parallelism, which is the + * parallelism of the sink. Set to Integer.MAX_VALUE for always using the maximum available + * write parallelism. + */ public DynamicRecord( TableIdentifier tableIdentifier, String branch, diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java index 91aa4a91710c..1c611c46b90c 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/HashKeyGenerator.java @@ -99,7 +99,7 @@ int generateKey( dynamicRecord.distributionMode(), DistributionMode.NONE), MoreObjects.firstNonNull( dynamicRecord.equalityFields(), Collections.emptySet()), - dynamicRecord.writeParallelism())); + Math.min(dynamicRecord.writeParallelism(), maxWriteParallelism))); try { return keySelector.getKey( overrideRowData != null ? overrideRowData : dynamicRecord.rowData()); @@ -242,15 +242,17 @@ private static class TargetLimitedKeySelector implements KeySelector maxWriteParallelism) { - LOG.warn( - "{}: writeParallelism {} is greater than maxWriteParallelism {}. Capping writeParallelism at {}", - tableName, - writeParallelism, - maxWriteParallelism, - maxWriteParallelism); - writeParallelism = maxWriteParallelism; - } + Preconditions.checkArgument( + writeParallelism > 0, + "%s: writeParallelism must be > 0 (is: %s)", + tableName, + writeParallelism); + Preconditions.checkArgument( + writeParallelism <= maxWriteParallelism, + "%s: writeParallelism (%s) must be <= maxWriteParallelism (%s)", + tableName, + writeParallelism, + maxWriteParallelism); this.wrapped = wrapped; this.writeParallelism = writeParallelism; this.distinctKeys = new int[writeParallelism]; diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java index 8d559e920620..04246bf03996 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.dynamic; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Collections; import java.util.Map; @@ -157,6 +158,34 @@ void testEqualityKeys() throws Exception { assertThat(getSubTaskId(writeKey3, writeParallelism, maxWriteParallelism)).isEqualTo(0); } + @Test + void testFailOnNonPositiveWriteParallelism() { + final int maxWriteParallelism = 5; + HashKeyGenerator generator = new HashKeyGenerator(16, maxWriteParallelism); + + assertThatThrownBy( + () -> { + getWriteKey( + generator, + PartitionSpec.unpartitioned(), + DistributionMode.NONE, + -1, // writeParallelism + Collections.emptySet(), + GenericRowData.of()); + }); + + assertThatThrownBy( + () -> { + getWriteKey( + generator, + PartitionSpec.unpartitioned(), + DistributionMode.NONE, + 0, // writeParallelism + Collections.emptySet(), + GenericRowData.of()); + }); + } + @Test void testCapAtMaxWriteParallelism() throws Exception { int writeParallelism = 10; From 4c0ad422624eb7e2b3b196a4ea77ee8ad83a69e9 Mon Sep 17 00:00:00 2001 From: Drew Gallardo Date: Thu, 4 Dec 2025 08:37:21 -0800 Subject: [PATCH 040/201] Core: Align ContentFile enum serialization with REST Spec (#14739) --- .../org/apache/iceberg/ContentFileParser.java | 33 ++++- .../apache/iceberg/TestContentFileParser.java | 124 ++++++++++++++---- .../apache/iceberg/TestDataTaskParser.java | 12 +- .../iceberg/TestFileScanTaskParser.java | 36 ++--- ...TestFetchPlanningResultResponseParser.java | 12 +- .../TestFetchScanTasksResponseParser.java | 12 +- .../TestPlanTableScanResponseParser.java | 48 +++---- 7 files changed, 186 insertions(+), 91 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/ContentFileParser.java b/core/src/main/java/org/apache/iceberg/ContentFileParser.java index 0033fa97725a..f024a24b18ce 100644 --- a/core/src/main/java/org/apache/iceberg/ContentFileParser.java +++ b/core/src/main/java/org/apache/iceberg/ContentFileParser.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.List; +import java.util.Locale; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.types.Types; @@ -50,6 +51,9 @@ public class ContentFileParser { private static final String REFERENCED_DATA_FILE = "referenced-data-file"; private static final String CONTENT_OFFSET = "content-offset"; private static final String CONTENT_SIZE = "content-size-in-bytes"; + private static final String CONTENT_DATA = "data"; + private static final String CONTENT_POSITION_DELETES = "position-deletes"; + private static final String CONTENT_EQUALITY_DELETES = "equality-deletes"; private ContentFileParser() {} @@ -84,9 +88,13 @@ public static void toJson(ContentFile contentFile, PartitionSpec spec, JsonGe // as it isn't used and BaseFile constructor doesn't support it. generator.writeNumberField(SPEC_ID, contentFile.specId()); - generator.writeStringField(CONTENT, contentFile.content().name()); + // Since 1.11, we serialize content as lowercase kebab-case values like "equality-deletes" + generator.writeStringField( + CONTENT, contentFile.content().name().toLowerCase(Locale.ENGLISH).replace('_', '-')); generator.writeStringField(FILE_PATH, contentFile.location()); - generator.writeStringField(FILE_FORMAT, contentFile.format().name()); + // Since 1.11, we serialize format as lower-case strings like "parquet" + generator.writeStringField( + FILE_FORMAT, contentFile.format().name().toLowerCase(Locale.ENGLISH)); if (contentFile.partition() != null) { generator.writeFieldName(PARTITION); @@ -147,7 +155,7 @@ public static ContentFile fromJson(JsonNode jsonNode, Map ContentFileParser.fromJson(node, Map.of(0, PartitionSpec.unpartitioned()))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file content value: 'invalid-content'"); + } + + @Test + public void testUppercaseFileFormat() throws Exception { + String jsonStr = + "{\"spec-id\":0," + + "\"content\":\"data\"," + + "\"file-path\":\"/path/to/file.parquet\"," + + "\"file-format\":\"PARQUET\"," + + "\"partition\":{}," + + "\"file-size-in-bytes\":1," + + "\"record-count\":1}"; + + JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); + ContentFile deserializedContentFile = + ContentFileParser.fromJson(jsonNode, Map.of(0, PartitionSpec.unpartitioned())); + assertThat(deserializedContentFile.format()).isEqualTo(FileFormat.PARQUET); + } + + @ParameterizedTest + @MethodSource("enumContentTypeCases") + public void testEnumContentTypeSerialization(FileContent content, String expectedJsonContent) + throws Exception { + String jsonStr = + "{\"spec-id\":0," + + "\"content\":\"" + + content.name() + + "\"," + + "\"file-path\":\"/path/to/data.parquet\"," + + "\"file-format\":\"parquet\"," + + "\"partition\":{}," + + "\"file-size-in-bytes\":1," + + "\"record-count\":1}"; + + JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); + ContentFile deserializedContentFile = + ContentFileParser.fromJson(jsonNode, Map.of(0, PartitionSpec.unpartitioned())); + assertThat(deserializedContentFile.content()).isEqualTo(content); + + String serializedStr = + ContentFileParser.toJson(deserializedContentFile, PartitionSpec.unpartitioned()); + assertThat(serializedStr).contains("\"content\":\"" + expectedJsonContent + "\""); + } + + private static Stream enumContentTypeCases() { + return Stream.of( + Arguments.of(FileContent.DATA, "data"), + Arguments.of(FileContent.POSITION_DELETES, "position-deletes"), + Arguments.of(FileContent.EQUALITY_DELETES, "equality-deletes")); + } + private static Stream provideSpecAndDataFile() { return Stream.of( Arguments.of( @@ -271,18 +339,18 @@ private static DataFile dataFileWithOnlyNanCounts(PartitionSpec spec) { private static String dataFileJsonWithRequiredOnly(PartitionSpec spec) { if (spec.isUnpartitioned()) { - return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\",\"file-format\":\"PARQUET\"," + return "{\"spec-id\":0,\"content\":\"data\",\"file-path\":\"/path/to/data-a.parquet\",\"file-format\":\"parquet\"," + "\"partition\":[],\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}"; } else { - return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\",\"file-format\":\"PARQUET\"," + return "{\"spec-id\":0,\"content\":\"data\",\"file-path\":\"/path/to/data-a.parquet\",\"file-format\":\"parquet\"," + "\"partition\":[1],\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}"; } } private static String dataFileJsonWithAllOptional(PartitionSpec spec) { if (spec.isUnpartitioned()) { - return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-with-stats.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[],\"file-size-in-bytes\":350,\"record-count\":10," + return "{\"spec-id\":0,\"content\":\"data\",\"file-path\":\"/path/to/data-with-stats.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[],\"file-size-in-bytes\":350,\"record-count\":10," + "\"column-sizes\":{\"keys\":[3,4],\"values\":[100,200]}," + "\"value-counts\":{\"keys\":[3,4],\"values\":[90,180]}," + "\"null-value-counts\":{\"keys\":[3,4],\"values\":[10,20]}," @@ -292,8 +360,8 @@ private static String dataFileJsonWithAllOptional(PartitionSpec spec) { + "\"key-metadata\":\"00000000000000000000000000000000\"," + "\"split-offsets\":[128,256],\"sort-order-id\":1}"; } else { - return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-with-stats.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[1],\"file-size-in-bytes\":350,\"record-count\":10," + return "{\"spec-id\":0,\"content\":\"data\",\"file-path\":\"/path/to/data-with-stats.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[1],\"file-size-in-bytes\":350,\"record-count\":10," + "\"column-sizes\":{\"keys\":[3,4],\"values\":[100,200]}," + "\"value-counts\":{\"keys\":[3,4],\"values\":[90,180]}," + "\"null-value-counts\":{\"keys\":[3,4],\"values\":[10,20]}," @@ -388,8 +456,8 @@ private static DeleteFile deleteFileWithDataRef(PartitionSpec spec) { } private static String deleteFileWithDataRefJson() { - return "{\"spec-id\":0,\"content\":\"POSITION_DELETES\",\"file-path\":\"/path/to/delete.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[4],\"file-size-in-bytes\":1234," + return "{\"spec-id\":0,\"content\":\"position-deletes\",\"file-path\":\"/path/to/delete.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[4],\"file-size-in-bytes\":1234," + "\"record-count\":10,\"referenced-data-file\":\"/path/to/data/file.parquet\"}"; } @@ -414,8 +482,8 @@ private static DeleteFile dv(PartitionSpec spec) { } private static String dvJson() { - return "{\"spec-id\":0,\"content\":\"POSITION_DELETES\",\"file-path\":\"/path/to/delete.puffin\"," - + "\"file-format\":\"PUFFIN\",\"partition\":[4],\"file-size-in-bytes\":1234,\"record-count\":10," + return "{\"spec-id\":0,\"content\":\"position-deletes\",\"file-path\":\"/path/to/delete.puffin\"," + + "\"file-format\":\"puffin\",\"partition\":[4],\"file-size-in-bytes\":1234,\"record-count\":10," + "\"referenced-data-file\":\"/path/to/data/file.parquet\",\"content-offset\":4,\"content-size-in-bytes\":40}"; } @@ -487,18 +555,18 @@ private static DeleteFile deleteFileWithAllOptional(PartitionSpec spec) { private static String deleteFileJsonWithRequiredOnly(PartitionSpec spec) { if (spec.isUnpartitioned()) { - return "{\"spec-id\":0,\"content\":\"POSITION_DELETES\",\"file-path\":\"/path/to/delete-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[],\"file-size-in-bytes\":1234,\"record-count\":9}"; + return "{\"spec-id\":0,\"content\":\"position-deletes\",\"file-path\":\"/path/to/delete-a.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[],\"file-size-in-bytes\":1234,\"record-count\":9}"; } else { - return "{\"spec-id\":0,\"content\":\"POSITION_DELETES\",\"file-path\":\"/path/to/delete-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[9],\"file-size-in-bytes\":1234,\"record-count\":9}"; + return "{\"spec-id\":0,\"content\":\"position-deletes\",\"file-path\":\"/path/to/delete-a.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[9],\"file-size-in-bytes\":1234,\"record-count\":9}"; } } private static String deleteFileJsonWithAllOptional(PartitionSpec spec) { if (spec.isUnpartitioned()) { - return "{\"spec-id\":0,\"content\":\"EQUALITY_DELETES\",\"file-path\":\"/path/to/delete-with-stats.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[],\"file-size-in-bytes\":1234,\"record-count\":10," + return "{\"spec-id\":0,\"content\":\"equality-deletes\",\"file-path\":\"/path/to/delete-with-stats.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[],\"file-size-in-bytes\":1234,\"record-count\":10," + "\"column-sizes\":{\"keys\":[3,4],\"values\":[100,200]}," + "\"value-counts\":{\"keys\":[3,4],\"values\":[90,180]}," + "\"null-value-counts\":{\"keys\":[3,4],\"values\":[10,20]}," @@ -508,8 +576,8 @@ private static String deleteFileJsonWithAllOptional(PartitionSpec spec) { + "\"key-metadata\":\"00000000000000000000000000000000\"," + "\"split-offsets\":[128],\"equality-ids\":[3],\"sort-order-id\":1}"; } else { - return "{\"spec-id\":0,\"content\":\"EQUALITY_DELETES\",\"file-path\":\"/path/to/delete-with-stats.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[9],\"file-size-in-bytes\":1234,\"record-count\":10," + return "{\"spec-id\":0,\"content\":\"equality-deletes\",\"file-path\":\"/path/to/delete-with-stats.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[9],\"file-size-in-bytes\":1234,\"record-count\":10," + "\"column-sizes\":{\"keys\":[3,4],\"values\":[100,200]}," + "\"value-counts\":{\"keys\":[3,4],\"values\":[90,180]}," + "\"null-value-counts\":{\"keys\":[3,4],\"values\":[10,20]}," diff --git a/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java b/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java index 758c11db788a..03065abe8744 100644 --- a/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java +++ b/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java @@ -144,9 +144,9 @@ public void missingFields() throws Exception { + "{\"id\":6,\"name\":\"summary\",\"required\":false,\"type\":{\"type\":\"map\"," + "\"key-id\":7,\"key\":\"string\",\"value-id\":8," + "\"value\":\"string\",\"value-required\":true}}]}," - + "\"metadata-file\":{\"spec-id\":0,\"content\":\"DATA\"," + + "\"metadata-file\":{\"spec-id\":0,\"content\":\"data\"," + "\"file-path\":\"/tmp/metadata2.json\"," - + "\"file-format\":\"METADATA\",\"partition\":[]," + + "\"file-format\":\"metadata\",\"partition\":[]," + "\"file-size-in-bytes\":0,\"record-count\":2,\"sort-order-id\":0}" + "}"; JsonNode missingTableRowsNode = mapper.reader().readTree(missingTableRowsStr); @@ -172,8 +172,8 @@ public void testDataTaskParsesFieldIdPartitionMap() { + "\"fields\":[{\"id\":1,\"name\":\"committed_at\",\"required\":true,\"type\":\"timestamptz\"}]}," + "\"projection\":{\"type\":\"struct\",\"schema-id\":0," + "\"fields\":[{\"id\":1,\"name\":\"committed_at\",\"required\":true,\"type\":\"timestamptz\"}]}," - + "\"metadata-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/tmp/metadata.json\"," - + "\"file-format\":\"METADATA\",\"partition\":{},\"file-size-in-bytes\":0,\"record-count\":1,\"sort-order-id\":0}," + + "\"metadata-file\":{\"spec-id\":0,\"content\":\"data\",\"file-path\":\"/tmp/metadata.json\"," + + "\"file-format\":\"metadata\",\"partition\":{},\"file-size-in-bytes\":0,\"record-count\":1,\"sort-order-id\":0}," + "\"rows\":[{\"1\":\"2009-02-13T23:31:30+00:00\"}]}"; StaticDataTask deserializedTask = (StaticDataTask) ScanTaskParser.fromJson(jsonStr, true); @@ -263,9 +263,9 @@ private String snapshotsDataTaskJson() { + "{\"id\":6,\"name\":\"summary\",\"required\":false,\"type\":{\"type\":\"map\"," + "\"key-id\":7,\"key\":\"string\",\"value-id\":8," + "\"value\":\"string\",\"value-required\":true}}]}," - + "\"metadata-file\":{\"spec-id\":0,\"content\":\"DATA\"," + + "\"metadata-file\":{\"spec-id\":0,\"content\":\"data\"," + "\"file-path\":\"/tmp/metadata2.json\"," - + "\"file-format\":\"METADATA\",\"partition\":[]," + + "\"file-format\":\"metadata\",\"partition\":[]," + "\"file-size-in-bytes\":0,\"record-count\":2,\"sort-order-id\":0}," + "\"rows\":[{\"1\":\"2009-02-13T23:31:30+00:00\",\"2\":1,\"4\":\"append\"," + "\"5\":\"file:/tmp/manifest1.avro\"," diff --git a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java index 882c2b33496d..1eff487d86cf 100644 --- a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java +++ b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java @@ -96,15 +96,15 @@ private String fileScanTaskJsonWithoutTaskType() { + "{\"id\":4,\"name\":\"data\",\"required\":true,\"type\":\"string\"}]}," + "\"spec\":{\"spec-id\":0,\"fields\":[{\"name\":\"data_bucket\"," + "\"transform\":\"bucket[16]\",\"source-id\":4,\"field-id\":1000}]}," - + "\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[0]," + + "\"data-file\":{\"spec-id\":0,\"content\":\"data\",\"file-path\":\"/path/to/data-a.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[0]," + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + "\"start\":0,\"length\":10," - + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," - + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," + + "\"delete-files\":[{\"spec-id\":0,\"content\":\"position-deletes\"," + + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"parquet\"," + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}," - + "{\"spec-id\":0,\"content\":\"EQUALITY_DELETES\",\"file-path\":\"/path/to/data-a2-deletes.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[0],\"file-size-in-bytes\":10," + + "{\"spec-id\":0,\"content\":\"equality-deletes\",\"file-path\":\"/path/to/data-a2-deletes.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[0],\"file-size-in-bytes\":10," + "\"record-count\":1,\"equality-ids\":[1],\"sort-order-id\":0}]," + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}"; } @@ -116,15 +116,15 @@ private String fileScanTaskJson() { + "{\"id\":4,\"name\":\"data\",\"required\":true,\"type\":\"string\"}]}," + "\"spec\":{\"spec-id\":0,\"fields\":[{\"name\":\"data_bucket\"," + "\"transform\":\"bucket[16]\",\"source-id\":4,\"field-id\":1000}]}," - + "\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[0]," + + "\"data-file\":{\"spec-id\":0,\"content\":\"data\",\"file-path\":\"/path/to/data-a.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[0]," + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + "\"start\":0,\"length\":10," - + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," - + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," + + "\"delete-files\":[{\"spec-id\":0,\"content\":\"position-deletes\"," + + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"parquet\"," + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}," - + "{\"spec-id\":0,\"content\":\"EQUALITY_DELETES\",\"file-path\":\"/path/to/data-a2-deletes.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[0],\"file-size-in-bytes\":10," + + "{\"spec-id\":0,\"content\":\"equality-deletes\",\"file-path\":\"/path/to/data-a2-deletes.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[0],\"file-size-in-bytes\":10," + "\"record-count\":1,\"equality-ids\":[1],\"sort-order-id\":0}]," + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}"; } @@ -136,15 +136,15 @@ private String fileScanTaskFieldIdPartitionMapJson() { + "{\"id\":4,\"name\":\"data\",\"required\":true,\"type\":\"string\"}]}," + "\"spec\":{\"spec-id\":0,\"fields\":[{\"name\":\"data_bucket\"," + "\"transform\":\"bucket[16]\",\"source-id\":4,\"field-id\":1000}]}," - + "\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":0}," + + "\"data-file\":{\"spec-id\":0,\"content\":\"data\",\"file-path\":\"/path/to/data-a.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":{\"1000\":0}," + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + "\"start\":0,\"length\":10," - + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," - + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," + + "\"delete-files\":[{\"spec-id\":0,\"content\":\"position-deletes\"," + + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"parquet\"," + "\"partition\":{\"1000\":0},\"file-size-in-bytes\":10,\"record-count\":1}," - + "{\"spec-id\":0,\"content\":\"EQUALITY_DELETES\",\"file-path\":\"/path/to/data-a2-deletes.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":0},\"file-size-in-bytes\":10," + + "{\"spec-id\":0,\"content\":\"equality-deletes\",\"file-path\":\"/path/to/data-a2-deletes.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":{\"1000\":0},\"file-size-in-bytes\":10," + "\"record-count\":1,\"equality-ids\":[1],\"sort-order-id\":0}]," + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}"; } diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java index 8b82e9794d5d..7c1021d4afab 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java @@ -155,8 +155,8 @@ public void roundTripSerdeWithInvalidPlanStatusSubmittedWithDeleteFilesNoFileSca String invalidJson = "{\"status\":\"submitted\"," - + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," - + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," + + "\"delete-files\":[{\"spec-id\":0,\"content\":\"position-deletes\"," + + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"parquet\"," + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}]" + "}"; @@ -193,12 +193,12 @@ public void roundTripSerdeWithValidStatusAndFileScanTasks() throws JsonProcessin String expectedToJson = "{\"status\":\"completed\"," - + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," - + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," + + "\"delete-files\":[{\"spec-id\":0,\"content\":\"position-deletes\"," + + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"parquet\"," + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}]," + "\"file-scan-tasks\":[" - + "{\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[0]," + + "{\"data-file\":{\"spec-id\":0,\"content\":\"data\",\"file-path\":\"/path/to/data-a.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[0]," + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + "\"delete-file-references\":[0]," + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}]" diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchScanTasksResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchScanTasksResponseParser.java index 62de75e3818b..d7824bc6a673 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchScanTasksResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchScanTasksResponseParser.java @@ -95,8 +95,8 @@ public void roundTripSerdeWithDeleteFilesNoFileScanTasksPresent() { String invalidJson = "{\"plan-tasks\":[\"task1\",\"task2\"]," - + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," - + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," + + "\"delete-files\":[{\"spec-id\":0,\"content\":\"position-deletes\"," + + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"parquet\"," + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}]" + "}"; @@ -129,12 +129,12 @@ public void roundTripSerdeWithFileScanTasks() { String expectedToJson = "{" - + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," - + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," + + "\"delete-files\":[{\"spec-id\":0,\"content\":\"position-deletes\"," + + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"parquet\"," + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}]," + "\"file-scan-tasks\":[" - + "{\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[0]," + + "{\"data-file\":{\"spec-id\":0,\"content\":\"data\",\"file-path\":\"/path/to/data-a.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[0]," + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + "\"delete-file-references\":[0]," + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}]" diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestPlanTableScanResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestPlanTableScanResponseParser.java index a59bbbd16b45..e2c9f21dabba 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestPlanTableScanResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestPlanTableScanResponseParser.java @@ -210,8 +210,8 @@ public void roundTripSerdeWithInvalidPlanStatusSubmittedWithDeleteFilesNoFileSca String invalidJson = "{\"status\":\"submitted\"," + "\"plan-id\":\"somePlanId\"," - + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," - + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," + + "\"delete-files\":[{\"spec-id\":0,\"content\":\"position-deletes\"," + + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"parquet\"," + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}]" + "}"; @@ -244,12 +244,12 @@ public void roundTripSerdeWithValidStatusAndFileScanTasks() { String expectedToJson = "{\"status\":\"completed\"," - + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," - + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," + + "\"delete-files\":[{\"spec-id\":0,\"content\":\"position-deletes\"," + + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"parquet\"," + "\"partition\":[0],\"file-size-in-bytes\":10,\"record-count\":1}]," + "\"file-scan-tasks\":[" - + "{\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[0]," + + "{\"data-file\":{\"spec-id\":0,\"content\":\"data\",\"file-path\":\"/path/to/data-a.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[0]," + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + "\"delete-file-references\":[0]," + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}]" @@ -316,25 +316,25 @@ public void multipleTasksWithDifferentDeleteFilesDontAccumulateReferences() { + " \"status\" : \"completed\",\n" + " \"delete-files\" : [ {\n" + " \"spec-id\" : 0,\n" - + " \"content\" : \"POSITION_DELETES\",\n" + + " \"content\" : \"position-deletes\",\n" + " \"file-path\" : \"/path/to/data-a-deletes.parquet\",\n" - + " \"file-format\" : \"PARQUET\",\n" + + " \"file-format\" : \"parquet\",\n" + " \"partition\" : [ 0 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1\n" + " }, {\n" + " \"spec-id\" : 0,\n" - + " \"content\" : \"POSITION_DELETES\",\n" + + " \"content\" : \"position-deletes\",\n" + " \"file-path\" : \"/path/to/data-b-deletes.parquet\",\n" - + " \"file-format\" : \"PARQUET\",\n" + + " \"file-format\" : \"parquet\",\n" + " \"partition\" : [ 1 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1\n" + " }, {\n" + " \"spec-id\" : 0,\n" - + " \"content\" : \"EQUALITY_DELETES\",\n" + + " \"content\" : \"equality-deletes\",\n" + " \"file-path\" : \"/path/to/data-c-deletes.parquet\",\n" - + " \"file-format\" : \"PARQUET\",\n" + + " \"file-format\" : \"parquet\",\n" + " \"partition\" : [ 2 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1,\n" @@ -344,9 +344,9 @@ public void multipleTasksWithDifferentDeleteFilesDontAccumulateReferences() { + " \"file-scan-tasks\" : [ {\n" + " \"data-file\" : {\n" + " \"spec-id\" : 0,\n" - + " \"content\" : \"DATA\",\n" + + " \"content\" : \"data\",\n" + " \"file-path\" : \"/path/to/data-a.parquet\",\n" - + " \"file-format\" : \"PARQUET\",\n" + + " \"file-format\" : \"parquet\",\n" + " \"partition\" : [ 0 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1,\n" @@ -357,9 +357,9 @@ public void multipleTasksWithDifferentDeleteFilesDontAccumulateReferences() { + " }, {\n" + " \"data-file\" : {\n" + " \"spec-id\" : 0,\n" - + " \"content\" : \"DATA\",\n" + + " \"content\" : \"data\",\n" + " \"file-path\" : \"/path/to/data-b.parquet\",\n" - + " \"file-format\" : \"PARQUET\",\n" + + " \"file-format\" : \"parquet\",\n" + " \"partition\" : [ 1 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1,\n" @@ -371,9 +371,9 @@ public void multipleTasksWithDifferentDeleteFilesDontAccumulateReferences() { + " }, {\n" + " \"data-file\" : {\n" + " \"spec-id\" : 0,\n" - + " \"content\" : \"DATA\",\n" + + " \"content\" : \"data\",\n" + " \"file-path\" : \"/path/to/data-c.parquet\",\n" - + " \"file-format\" : \"PARQUET\",\n" + + " \"file-format\" : \"parquet\",\n" + " \"partition\" : [ 2 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1,\n" @@ -409,8 +409,8 @@ public void roundTripSerdeWithoutDeleteFiles() { String expectedJson = "{\"status\":\"completed\"," + "\"file-scan-tasks\":[" - + "{\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," - + "\"file-format\":\"PARQUET\",\"partition\":[0]," + + "{\"data-file\":{\"spec-id\":0,\"content\":\"data\",\"file-path\":\"/path/to/data-a.parquet\"," + + "\"file-format\":\"parquet\",\"partition\":[0]," + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}]" + "}"; @@ -608,9 +608,9 @@ public void roundTripSerdeWithValidStatusAndFileScanTasksAndCredentials() { + " } ],\n" + " \"delete-files\" : [ {\n" + " \"spec-id\" : 0,\n" - + " \"content\" : \"POSITION_DELETES\",\n" + + " \"content\" : \"position-deletes\",\n" + " \"file-path\" : \"/path/to/data-a-deletes.parquet\",\n" - + " \"file-format\" : \"PARQUET\",\n" + + " \"file-format\" : \"parquet\",\n" + " \"partition\" : [ 0 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1\n" @@ -618,9 +618,9 @@ public void roundTripSerdeWithValidStatusAndFileScanTasksAndCredentials() { + " \"file-scan-tasks\" : [ {\n" + " \"data-file\" : {\n" + " \"spec-id\" : 0,\n" - + " \"content\" : \"DATA\",\n" + + " \"content\" : \"data\",\n" + " \"file-path\" : \"/path/to/data-a.parquet\",\n" - + " \"file-format\" : \"PARQUET\",\n" + + " \"file-format\" : \"parquet\",\n" + " \"partition\" : [ 0 ],\n" + " \"file-size-in-bytes\" : 10,\n" + " \"record-count\" : 1,\n" From d7f8950ab4a74ece5b02a5a76741a01e61df3b7d Mon Sep 17 00:00:00 2001 From: ggershinsky Date: Fri, 5 Dec 2025 02:30:14 +0200 Subject: [PATCH 041/201] Exception on encryption key altering (#14723) * initial commit * illegal args exception --- .../org/apache/iceberg/hive/HiveTableOperations.java | 9 ++++++++- .../apache/iceberg/spark/sql/TestTableEncryption.java | 7 +++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java index a4338ec07a5f..4d382f8d388e 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java @@ -321,7 +321,14 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { } if (removedProps.contains(TableProperties.ENCRYPTION_TABLE_KEY)) { - throw new RuntimeException("Cannot remove key in encrypted table"); + throw new IllegalArgumentException("Cannot remove key in encrypted table"); + } + + if (base != null + && !Objects.equals( + base.properties().get(TableProperties.ENCRYPTION_TABLE_KEY), + metadata.properties().get(TableProperties.ENCRYPTION_TABLE_KEY))) { + throw new IllegalArgumentException("Cannot modify key in encrypted table"); } HMSTablePropertyHelper.updateHmsTableForIcebergTable( diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java index c71bd28706c9..8f0552a37877 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java @@ -169,6 +169,13 @@ public void testKeyDelete() { .hasMessageContaining("Cannot remove key in encrypted table"); } + @TestTemplate + public void testKeyAlter() { + assertThatThrownBy( + () -> sql("ALTER TABLE %s SET TBLPROPERTIES ('encryption.key-id'='abcd')", tableName)) + .hasMessageContaining("Cannot modify key in encrypted table"); + } + @TestTemplate public void testDirectDataFileRead() { List dataFileTable = From ba983470e7caf03290933402cdc6f15d883e1073 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 4 Dec 2025 17:19:34 -0800 Subject: [PATCH 042/201] Docs: fix rendering issues in encryption doc (#14756) --- docs/docs/encryption.md | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/docs/docs/encryption.md b/docs/docs/encryption.md index 0b694c44005b..5da51486f76b 100644 --- a/docs/docs/encryption.md +++ b/docs/docs/encryption.md @@ -26,7 +26,8 @@ The `metadata.json` file does not contain data or stats, and is therefore not en Currently, encryption is supported in the Hive and REST catalogs for tables with Parquet and Avro data formats. -Two parameters are required to activate encryption of a table +Two parameters are required to activate encryption of a table: + 1. Catalog property `encryption.kms-impl`, that specifies the class path for a client of a KMS ("key management service"). 2. Table property `encryption.key-id`, that specifies the ID of a master key used to encrypt and decrypt the table. Master keys are stored and managed in the KMS. @@ -46,7 +47,7 @@ spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-{{ sparkVersionMaj ```sql CREATE TABLE local.db.table (id bigint, data string) USING iceberg -TBLPROPERTIES ('encryption.key-id'='{{ master key id }}'); +TBLPROPERTIES ('encryption.key-id'=''); ``` Inserted data will be automatically encrypted, @@ -58,7 +59,7 @@ INSERT INTO local.db.table VALUES (1, 'a'), (2, 'b'), (3, 'c'); To verify encryption, the contents of data, manifest and manifest list files can be dumped in the command line with ```sh -hexdump -C {{ /path/to/file }} | more +hexdump -C | more ``` The Parquet files must start with the "PARE" magic string (PARquet Encrypted footer mode), and manifest/list files must start with "AGS1" magic string (Aes Gcm Stream version 1). @@ -73,13 +74,12 @@ SELECT * FROM local.db.table; 1. Catalogs must ensure the `encryption.key-id` property is not modified or removed during table lifetime. -2. To function properly, Iceberg table encryption requires the catalog implementations not to retrieve the metadata -directly from metadata.json files, if these files are kept unprotected in a storage vulnerable to tampering. +2. To function properly, Iceberg table encryption requires the catalog implementations not to retrieve the metadata directly from metadata.json files, if these files are kept unprotected in a storage vulnerable to tampering: -* Catalogs may keep the metadata in a trusted independent object store. -* Catalogs may work with metadata.json files in a tamper-proof storage. -* Catalogs may use checksum techniques to verify integrity of metadata.json files in a storage vulnerable to tampering -(the checksums must be kept in a separate trusted storage). + - Catalogs may keep the metadata in a trusted independent object store. + - Catalogs may work with metadata.json files in a tamper-proof storage. + - Catalogs may use checksum techniques to verify integrity of metadata.json files in a storage vulnerable to tampering + (the checksums must be kept in a separate trusted storage). ## Key Management Clients @@ -124,17 +124,17 @@ The standard Iceberg encryption manager generates an encryption key and a unique for each data and delete file. The generation is performed in the worker nodes, by using a secure random number generator. For Parquet data files, these parameters are passed to the native Parquet Modular Encryption [mechanism](https://parquet.apache.org/docs/file-format/data-pages/encryption). For Avro data files, -these parameters are passed to the AES GCM Stream encryption [mechanism](../../format/gcm-stream-spec.md). +these parameters are passed to the AES GCM Stream encryption [mechanism](../../gcm-stream-spec.md). The parent manifest file stores the encryption key and AAD prefix for each data and delete file in the -`key_metadata` [field](../../format/spec.md#data-file-fields). For Avro data tables, the data file length +`key_metadata` [field](../../spec.md#data-file-fields). For Avro data tables, the data file length is also added to the `key_metadata`. The manifest file is encrypted by the AES GCM Stream encryption mechanism, using an encryption key and an AAD prefix generated by the standard encryption manager. The generation is performed in the driver nodes, by using a secure random number generator. The parent manifest list file stores the encryption key, AAD prefix and file length for each manifest file -in the `key_metadata` [field](../../format/spec.md#manifest-lists). The manifest list file is encrypted by +in the `key_metadata` [field](../../spec.md#manifest-lists). The manifest list file is encrypted by the AES GCM Stream encryption mechanism, using an encryption key and an AAD prefix generated by the standard encryption manager. @@ -142,9 +142,9 @@ The manifest list encryption key, AAD prefix and file length are packed in a key is serialized and encrypted with a "key encryption key" (KEK), using the KEK creation timestamp as the AES GCM AAD. A KEK and its unique KEK_ID are generated by using a secure random number generator. For each snapshot, the KEK_ID of the encryption key that encrypts the manifest list key metadata is kept in the -`key-id` field in the table metadata snapshot [structure](../../format/spec.md#snapshots). The encrypted +`key-id` field in the table metadata snapshot [structure](../../spec.md#snapshots). The encrypted manifest list key metadata is kept in the `encryption-keys` list in the table metadata -[structure](../../format/spec.md#table-metadata-fields). +[structure](../../spec.md#table-metadata-fields). The KEK is encrypted by the table master key via the KMS client. The result is kept in the `encryption-keys` list in the table metadata structure. The KEK is re-used for a period allowed by the NIST SP 800-57 From 8db3d2115d948ab8b150cd0ca7ae3fe1a17aa1d3 Mon Sep 17 00:00:00 2001 From: aiborodin Date: Fri, 5 Dec 2025 19:07:23 +1100 Subject: [PATCH 043/201] Flink: Fix cache refreshing in dynamic sink (#14406) --- .../sink/dynamic/TableMetadataCache.java | 25 +++++++++++--- .../sink/dynamic/TestTableMetadataCache.java | 33 +++++++++++++++++++ 2 files changed, 54 insertions(+), 4 deletions(-) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java index 85a5a4abf29c..2c08a3486e7c 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.dynamic; +import java.time.Clock; import java.util.Map; import java.util.Set; import org.apache.flink.annotation.Internal; @@ -50,13 +51,25 @@ class TableMetadataCache { private final Catalog catalog; private final long refreshMs; + private final Clock cacheRefreshClock; private final int inputSchemasPerTableCacheMaximumSize; private final Map tableCache; TableMetadataCache( Catalog catalog, int maximumSize, long refreshMs, int inputSchemasPerTableCacheMaximumSize) { + this(catalog, maximumSize, refreshMs, inputSchemasPerTableCacheMaximumSize, Clock.systemUTC()); + } + + @VisibleForTesting + TableMetadataCache( + Catalog catalog, + int maximumSize, + long refreshMs, + int inputSchemasPerTableCacheMaximumSize, + Clock cacheRefreshClock) { this.catalog = catalog; this.refreshMs = refreshMs; + this.cacheRefreshClock = cacheRefreshClock; this.inputSchemasPerTableCacheMaximumSize = inputSchemasPerTableCacheMaximumSize; this.tableCache = new LRUCache<>(maximumSize); } @@ -88,6 +101,7 @@ void update(TableIdentifier identifier, Table table) { tableCache.put( identifier, new CacheItem( + cacheRefreshClock.millis(), true, table.refs().keySet(), table.schemas(), @@ -186,14 +200,16 @@ private Tuple2 refreshTable(TableIdentifier identifier) { return EXISTS; } catch (NoSuchTableException e) { LOG.debug("Table doesn't exist {}", identifier, e); - tableCache.put(identifier, new CacheItem(false, null, null, null, 1)); + tableCache.put( + identifier, new CacheItem(cacheRefreshClock.millis(), false, null, null, null, 1)); return Tuple2.of(false, e); } } private boolean needsRefresh(CacheItem cacheItem, boolean allowRefresh) { return allowRefresh - && (cacheItem == null || cacheItem.created + refreshMs > System.currentTimeMillis()); + && (cacheItem == null + || cacheRefreshClock.millis() - cacheItem.createdTimestampMillis > refreshMs); } public void invalidate(TableIdentifier identifier) { @@ -202,8 +218,7 @@ public void invalidate(TableIdentifier identifier) { /** Handles timeout for missing items only. Caffeine performance causes noticeable delays. */ static class CacheItem { - private final long created = System.currentTimeMillis(); - + private final long createdTimestampMillis; private final boolean tableExists; private final Set branches; private final Map tableSchemas; @@ -211,11 +226,13 @@ static class CacheItem { private final Map inputSchemas; private CacheItem( + long createdTimestampMillis, boolean tableExists, Set branches, Map tableSchemas, Map specs, int inputSchemaCacheMaximumSize) { + this.createdTimestampMillis = createdTimestampMillis; this.tableExists = tableExists; this.branches = branches; this.tableSchemas = tableSchemas; diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java index 42c93c13c264..bf5b9f562f9a 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java @@ -20,9 +20,13 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.time.Clock; +import java.time.Instant; +import java.time.ZoneId; import org.apache.commons.lang3.SerializationUtils; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; @@ -92,4 +96,33 @@ void testCachingDisabled() { assertThat(cache.getInternalCache()).isEmpty(); } + + @Test + void testNoCacheRefreshingBeforeRefreshIntervalElapses() { + // Create table + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); + Table table = catalog.createTable(tableIdentifier, SCHEMA2); + + // Init cache + TableMetadataCache cache = + new TableMetadataCache( + catalog, 10, 100L, 10, Clock.fixed(Instant.now(), ZoneId.systemDefault())); + cache.update(tableIdentifier, table); + + // Cache schema + Schema schema = cache.schema(tableIdentifier, SCHEMA2).resolvedTableSchema(); + assertThat(schema.sameSchema(SCHEMA2)).isTrue(); + + // Cache schema with fewer fields + TableMetadataCache.ResolvedSchemaInfo schemaInfo = cache.schema(tableIdentifier, SCHEMA); + assertThat(schemaInfo.resolvedTableSchema().sameSchema(SCHEMA2)).isTrue(); + assertThat(schemaInfo.compareResult()) + .isEqualTo(CompareSchemasVisitor.Result.DATA_CONVERSION_NEEDED); + + // Assert both schemas are in cache + TableMetadataCache.CacheItem cacheItem = cache.getInternalCache().get(tableIdentifier); + assertThat(cacheItem).isNotNull(); + assertThat(cacheItem.inputSchemas()).containsKeys(SCHEMA, SCHEMA2); + } } From c4ba60d27b02d8618621ad701e52d51b9a98d0d5 Mon Sep 17 00:00:00 2001 From: aiborodin Date: Fri, 5 Dec 2025 19:08:30 +1100 Subject: [PATCH 044/201] Flink: Backport fix cache refreshing in dynamic sink (#14765) Backport #14406 --- .../sink/dynamic/TableMetadataCache.java | 25 +++++++++++--- .../sink/dynamic/TestTableMetadataCache.java | 33 +++++++++++++++++++ .../sink/dynamic/TableMetadataCache.java | 25 +++++++++++--- .../sink/dynamic/TestTableMetadataCache.java | 33 +++++++++++++++++++ 4 files changed, 108 insertions(+), 8 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java index 85a5a4abf29c..2c08a3486e7c 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.dynamic; +import java.time.Clock; import java.util.Map; import java.util.Set; import org.apache.flink.annotation.Internal; @@ -50,13 +51,25 @@ class TableMetadataCache { private final Catalog catalog; private final long refreshMs; + private final Clock cacheRefreshClock; private final int inputSchemasPerTableCacheMaximumSize; private final Map tableCache; TableMetadataCache( Catalog catalog, int maximumSize, long refreshMs, int inputSchemasPerTableCacheMaximumSize) { + this(catalog, maximumSize, refreshMs, inputSchemasPerTableCacheMaximumSize, Clock.systemUTC()); + } + + @VisibleForTesting + TableMetadataCache( + Catalog catalog, + int maximumSize, + long refreshMs, + int inputSchemasPerTableCacheMaximumSize, + Clock cacheRefreshClock) { this.catalog = catalog; this.refreshMs = refreshMs; + this.cacheRefreshClock = cacheRefreshClock; this.inputSchemasPerTableCacheMaximumSize = inputSchemasPerTableCacheMaximumSize; this.tableCache = new LRUCache<>(maximumSize); } @@ -88,6 +101,7 @@ void update(TableIdentifier identifier, Table table) { tableCache.put( identifier, new CacheItem( + cacheRefreshClock.millis(), true, table.refs().keySet(), table.schemas(), @@ -186,14 +200,16 @@ private Tuple2 refreshTable(TableIdentifier identifier) { return EXISTS; } catch (NoSuchTableException e) { LOG.debug("Table doesn't exist {}", identifier, e); - tableCache.put(identifier, new CacheItem(false, null, null, null, 1)); + tableCache.put( + identifier, new CacheItem(cacheRefreshClock.millis(), false, null, null, null, 1)); return Tuple2.of(false, e); } } private boolean needsRefresh(CacheItem cacheItem, boolean allowRefresh) { return allowRefresh - && (cacheItem == null || cacheItem.created + refreshMs > System.currentTimeMillis()); + && (cacheItem == null + || cacheRefreshClock.millis() - cacheItem.createdTimestampMillis > refreshMs); } public void invalidate(TableIdentifier identifier) { @@ -202,8 +218,7 @@ public void invalidate(TableIdentifier identifier) { /** Handles timeout for missing items only. Caffeine performance causes noticeable delays. */ static class CacheItem { - private final long created = System.currentTimeMillis(); - + private final long createdTimestampMillis; private final boolean tableExists; private final Set branches; private final Map tableSchemas; @@ -211,11 +226,13 @@ static class CacheItem { private final Map inputSchemas; private CacheItem( + long createdTimestampMillis, boolean tableExists, Set branches, Map tableSchemas, Map specs, int inputSchemaCacheMaximumSize) { + this.createdTimestampMillis = createdTimestampMillis; this.tableExists = tableExists; this.branches = branches; this.tableSchemas = tableSchemas; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java index 42c93c13c264..bf5b9f562f9a 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java @@ -20,9 +20,13 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.time.Clock; +import java.time.Instant; +import java.time.ZoneId; import org.apache.commons.lang3.SerializationUtils; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; @@ -92,4 +96,33 @@ void testCachingDisabled() { assertThat(cache.getInternalCache()).isEmpty(); } + + @Test + void testNoCacheRefreshingBeforeRefreshIntervalElapses() { + // Create table + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); + Table table = catalog.createTable(tableIdentifier, SCHEMA2); + + // Init cache + TableMetadataCache cache = + new TableMetadataCache( + catalog, 10, 100L, 10, Clock.fixed(Instant.now(), ZoneId.systemDefault())); + cache.update(tableIdentifier, table); + + // Cache schema + Schema schema = cache.schema(tableIdentifier, SCHEMA2).resolvedTableSchema(); + assertThat(schema.sameSchema(SCHEMA2)).isTrue(); + + // Cache schema with fewer fields + TableMetadataCache.ResolvedSchemaInfo schemaInfo = cache.schema(tableIdentifier, SCHEMA); + assertThat(schemaInfo.resolvedTableSchema().sameSchema(SCHEMA2)).isTrue(); + assertThat(schemaInfo.compareResult()) + .isEqualTo(CompareSchemasVisitor.Result.DATA_CONVERSION_NEEDED); + + // Assert both schemas are in cache + TableMetadataCache.CacheItem cacheItem = cache.getInternalCache().get(tableIdentifier); + assertThat(cacheItem).isNotNull(); + assertThat(cacheItem.inputSchemas()).containsKeys(SCHEMA, SCHEMA2); + } } diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java index 85a5a4abf29c..2c08a3486e7c 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.dynamic; +import java.time.Clock; import java.util.Map; import java.util.Set; import org.apache.flink.annotation.Internal; @@ -50,13 +51,25 @@ class TableMetadataCache { private final Catalog catalog; private final long refreshMs; + private final Clock cacheRefreshClock; private final int inputSchemasPerTableCacheMaximumSize; private final Map tableCache; TableMetadataCache( Catalog catalog, int maximumSize, long refreshMs, int inputSchemasPerTableCacheMaximumSize) { + this(catalog, maximumSize, refreshMs, inputSchemasPerTableCacheMaximumSize, Clock.systemUTC()); + } + + @VisibleForTesting + TableMetadataCache( + Catalog catalog, + int maximumSize, + long refreshMs, + int inputSchemasPerTableCacheMaximumSize, + Clock cacheRefreshClock) { this.catalog = catalog; this.refreshMs = refreshMs; + this.cacheRefreshClock = cacheRefreshClock; this.inputSchemasPerTableCacheMaximumSize = inputSchemasPerTableCacheMaximumSize; this.tableCache = new LRUCache<>(maximumSize); } @@ -88,6 +101,7 @@ void update(TableIdentifier identifier, Table table) { tableCache.put( identifier, new CacheItem( + cacheRefreshClock.millis(), true, table.refs().keySet(), table.schemas(), @@ -186,14 +200,16 @@ private Tuple2 refreshTable(TableIdentifier identifier) { return EXISTS; } catch (NoSuchTableException e) { LOG.debug("Table doesn't exist {}", identifier, e); - tableCache.put(identifier, new CacheItem(false, null, null, null, 1)); + tableCache.put( + identifier, new CacheItem(cacheRefreshClock.millis(), false, null, null, null, 1)); return Tuple2.of(false, e); } } private boolean needsRefresh(CacheItem cacheItem, boolean allowRefresh) { return allowRefresh - && (cacheItem == null || cacheItem.created + refreshMs > System.currentTimeMillis()); + && (cacheItem == null + || cacheRefreshClock.millis() - cacheItem.createdTimestampMillis > refreshMs); } public void invalidate(TableIdentifier identifier) { @@ -202,8 +218,7 @@ public void invalidate(TableIdentifier identifier) { /** Handles timeout for missing items only. Caffeine performance causes noticeable delays. */ static class CacheItem { - private final long created = System.currentTimeMillis(); - + private final long createdTimestampMillis; private final boolean tableExists; private final Set branches; private final Map tableSchemas; @@ -211,11 +226,13 @@ static class CacheItem { private final Map inputSchemas; private CacheItem( + long createdTimestampMillis, boolean tableExists, Set branches, Map tableSchemas, Map specs, int inputSchemaCacheMaximumSize) { + this.createdTimestampMillis = createdTimestampMillis; this.tableExists = tableExists; this.branches = branches; this.tableSchemas = tableSchemas; diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java index 42c93c13c264..bf5b9f562f9a 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java @@ -20,9 +20,13 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.time.Clock; +import java.time.Instant; +import java.time.ZoneId; import org.apache.commons.lang3.SerializationUtils; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; @@ -92,4 +96,33 @@ void testCachingDisabled() { assertThat(cache.getInternalCache()).isEmpty(); } + + @Test + void testNoCacheRefreshingBeforeRefreshIntervalElapses() { + // Create table + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); + Table table = catalog.createTable(tableIdentifier, SCHEMA2); + + // Init cache + TableMetadataCache cache = + new TableMetadataCache( + catalog, 10, 100L, 10, Clock.fixed(Instant.now(), ZoneId.systemDefault())); + cache.update(tableIdentifier, table); + + // Cache schema + Schema schema = cache.schema(tableIdentifier, SCHEMA2).resolvedTableSchema(); + assertThat(schema.sameSchema(SCHEMA2)).isTrue(); + + // Cache schema with fewer fields + TableMetadataCache.ResolvedSchemaInfo schemaInfo = cache.schema(tableIdentifier, SCHEMA); + assertThat(schemaInfo.resolvedTableSchema().sameSchema(SCHEMA2)).isTrue(); + assertThat(schemaInfo.compareResult()) + .isEqualTo(CompareSchemasVisitor.Result.DATA_CONVERSION_NEEDED); + + // Assert both schemas are in cache + TableMetadataCache.CacheItem cacheItem = cache.getInternalCache().get(tableIdentifier); + assertThat(cacheItem).isNotNull(); + assertThat(cacheItem.inputSchemas()).containsKeys(SCHEMA, SCHEMA2); + } } From 55bfc7e82d03b5038bc5d0da852bd16615486926 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Fri, 5 Dec 2025 08:39:28 -0800 Subject: [PATCH 045/201] OpenAPI: use yaml linter (#14686) --- .../main/resources/s3-signer-open-api.yaml | 6 +- open-api/.yamllint | 29 + open-api/Makefile | 8 +- open-api/requirements.txt | 1 + open-api/rest-catalog-open-api.yaml | 540 +++++++++--------- 5 files changed, 309 insertions(+), 275 deletions(-) create mode 100644 open-api/.yamllint diff --git a/aws/src/main/resources/s3-signer-open-api.yaml b/aws/src/main/resources/s3-signer-open-api.yaml index 1e4dafa6c1b7..3d719c515b2a 100644 --- a/aws/src/main/resources/s3-signer-open-api.yaml +++ b/aws/src/main/resources/s3-signer-open-api.yaml @@ -88,9 +88,9 @@ paths: 5XX: $ref: '../../../../open-api/rest-catalog-open-api.yaml#/components/responses/ServerErrorResponse' - ############################## - # Application Schema Objects # - ############################## +############################## +# Application Schema Objects # +############################## components: schemas: diff --git a/open-api/.yamllint b/open-api/.yamllint new file mode 100644 index 000000000000..1d9808457e34 --- /dev/null +++ b/open-api/.yamllint @@ -0,0 +1,29 @@ +# 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. + +--- +extends: default + +rules: + line-length: + max: 350 + braces: + min-spaces-inside: 0 + max-spaces-inside: -1 + brackets: + min-spaces-inside: 0 + max-spaces-inside: -1 diff --git a/open-api/Makefile b/open-api/Makefile index c9795ac22adb..3c2c07936e41 100644 --- a/open-api/Makefile +++ b/open-api/Makefile @@ -19,10 +19,16 @@ install: uv venv --python 3.12 --allow-existing # Match --target-python-version in the `generate` target uv pip install -r requirements.txt -lint: +validate-spec: uv run openapi-spec-validator --errors all rest-catalog-open-api.yaml uv run openapi-spec-validator --errors all ../aws/src/main/resources/s3-signer-open-api.yaml +lint-spec: + uv run yamllint --strict rest-catalog-open-api.yaml + uv run yamllint --strict ../aws/src/main/resources/s3-signer-open-api.yaml + +lint: validate-spec lint-spec + generate: uv run datamodel-codegen \ --enum-field-as-literal all \ diff --git a/open-api/requirements.txt b/open-api/requirements.txt index c76cb3485572..fcc9285b35bd 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -17,3 +17,4 @@ openapi-spec-validator==0.7.2 datamodel-code-generator==0.36.0 +yamllint==1.37.1 diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 788c6bd5812c..c0b1c0f0fcea 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -141,21 +141,21 @@ paths: schema: $ref: '#/components/schemas/CatalogConfig' example: { - "overrides": { - "warehouse": "s3://bucket/warehouse/" - }, - "defaults": { - "clients": "4" - }, - "idempotency-key-lifetime": "PT30M", - "endpoints": [ - "GET /v1/{prefix}/namespaces/{namespace}", - "GET /v1/{prefix}/namespaces", - "POST /v1/{prefix}/namespaces", - "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", - "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" - ] - } + "overrides": { + "warehouse": "s3://bucket/warehouse/" + }, + "defaults": { + "clients": "4" + }, + "idempotency-key-lifetime": "PT30M", + "endpoints": [ + "GET /v1/{prefix}/namespaces/{namespace}", + "GET /v1/{prefix}/namespaces", + "POST /v1/{prefix}/namespaces", + "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", + "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" + ] + } 400: $ref: '#/components/responses/BadRequestErrorResponse' 401: @@ -632,7 +632,7 @@ paths: point-in-time scan of the latest snapshot in the table's main branch. - Responses must include a valid status listed below. A "cancelled" status is considered invalid for this endpoint. + Responses must include a valid status listed below. A "cancelled" status is considered invalid for this endpoint. - When "completed" the planning operation has produced plan tasks and file scan tasks that must be returned in the response (not fetched @@ -863,8 +863,6 @@ paths: 5XX: $ref: '#/components/responses/ServerErrorResponse' - - /v1/{prefix}/namespaces/{namespace}/register: parameters: - $ref: '#/components/parameters/prefix' @@ -964,7 +962,7 @@ paths: The snapshots to return in the body of the metadata. Setting the value to `all` would return the full set of snapshots currently valid for the table. Setting the value to `refs` would load all snapshots referenced by branches or tags. - + Default if no param is provided is `all`. required: false schema: @@ -1069,12 +1067,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Internal Server Error", - "type": "CommitStateUnknownException", - "code": 500 - } + "error": { + "message": "Internal Server Error", + "type": "CommitStateUnknownException", + "code": 500 } + } 503: $ref: '#/components/responses/ServiceUnavailableResponse' 502: @@ -1085,12 +1083,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Invalid response from the upstream server", - "type": "CommitStateUnknownException", - "code": 502 - } + "error": { + "message": "Invalid response from the upstream server", + "type": "CommitStateUnknownException", + "code": 502 } + } 504: description: A server-side gateway timeout occurred; the commit state is unknown. @@ -1099,12 +1097,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Gateway timed out during commit", - "type": "CommitStateUnknownException", - "code": 504 - } + "error": { + "message": "Gateway timed out during commit", + "type": "CommitStateUnknownException", + "code": 504 } + } 5XX: description: A server-side problem that might not be addressable on the client. @@ -1113,12 +1111,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Bad Gateway", - "type": "InternalServerError", - "code": 502 - } + "error": { + "message": "Bad Gateway", + "type": "InternalServerError", + "code": 502 } + } delete: tags: @@ -1413,12 +1411,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Internal Server Error", - "type": "CommitStateUnknownException", - "code": 500 - } + "error": { + "message": "Internal Server Error", + "type": "CommitStateUnknownException", + "code": 500 } + } 503: $ref: '#/components/responses/ServiceUnavailableResponse' 502: @@ -1429,12 +1427,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Invalid response from the upstream server", - "type": "CommitStateUnknownException", - "code": 502 - } + "error": { + "message": "Invalid response from the upstream server", + "type": "CommitStateUnknownException", + "code": 502 } + } 504: description: A server-side gateway timeout occurred; the commit state is unknown. @@ -1443,12 +1441,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Gateway timed out during commit", - "type": "CommitStateUnknownException", - "code": 504 - } + "error": { + "message": "Gateway timed out during commit", + "type": "CommitStateUnknownException", + "code": 504 } + } 5XX: description: A server-side problem that might not be addressable on the client. @@ -1457,12 +1455,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Bad Gateway", - "type": "InternalServerError", - "code": 502 - } + "error": { + "message": "Bad Gateway", + "type": "InternalServerError", + "code": 502 } + } /v1/{prefix}/namespaces/{namespace}/views: parameters: @@ -1653,12 +1651,12 @@ paths: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "Internal Server Error", - "type": "CommitStateUnknownException", - "code": 500 - } + "error": { + "message": "Internal Server Error", + "type": "CommitStateUnknownException", + "code": 500 } + } 503: $ref: '#/components/responses/ServiceUnavailableResponse' 502: @@ -1669,12 +1667,12 @@ paths: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "Invalid response from the upstream server", - "type": "CommitStateUnknownException", - "code": 502 - } + "error": { + "message": "Invalid response from the upstream server", + "type": "CommitStateUnknownException", + "code": 502 } + } 504: description: A server-side gateway timeout occurred; the commit state is unknown. @@ -1683,12 +1681,12 @@ paths: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "Gateway timed out during commit", - "type": "CommitStateUnknownException", - "code": 504 - } + "error": { + "message": "Gateway timed out during commit", + "type": "CommitStateUnknownException", + "code": 504 } + } 5XX: description: A server-side problem that might not be addressable on the client. @@ -1697,12 +1695,12 @@ paths: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "Bad Gateway", - "type": "InternalServerError", - "code": 502 - } + "error": { + "message": "Bad Gateway", + "type": "InternalServerError", + "code": 502 } + } delete: tags: @@ -1887,13 +1885,13 @@ components: Optional signal to the server that the client supports delegated access via a comma-separated list of access mechanisms. The server may choose to supply access via any or none of the requested mechanisms. - - + + Specific properties and handling for `vended-credentials` is documented in the `LoadTableResult` schema section of this spec document. - - - The protocol and specification for `remote-signing` is documented in + + + The protocol and specification for `remote-signing` is documented in the `s3-signer-open-api.yaml` OpenApi spec in the `aws` module. required: false @@ -1917,7 +1915,7 @@ components: name: pageSize in: query description: - For servers that support pagination, this signals an upper bound of the number of results that a client will receive. + For servers that support pagination, this signals an upper bound of the number of results that a client will receive. For servers that do not support pagination, clients may receive results larger than the indicated `pageSize`. required: false schema: @@ -2019,12 +2017,12 @@ components: description: A list of endpoints that the server supports. The format of each endpoint must be " ". The HTTP verb and the resource path must be separated by a space character. example: [ - "GET /v1/{prefix}/namespaces/{namespace}", - "GET /v1/{prefix}/namespaces", - "POST /v1/{prefix}/namespaces", - "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", - "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" - ] + "GET /v1/{prefix}/namespaces/{namespace}", + "GET /v1/{prefix}/namespaces", + "POST /v1/{prefix}/namespaces", + "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", + "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" + ] idempotency-key-lifetime: type: string format: duration @@ -2091,20 +2089,20 @@ components: An opaque token that allows clients to make use of pagination for list APIs (e.g. ListTables). Clients may initiate the first paginated request by sending an empty query parameter `pageToken` to the server. - + Servers that support pagination should identify the `pageToken` parameter and return a `next-page-token` in the response if there are more results available. After the initial request, the value of `next-page-token` from each response must be used as the `pageToken` parameter value for the next request. The server must return `null` value for the `next-page-token` in the last response. - + Servers that support pagination must return all results in a single response with the value of `next-page-token` set to `null` if the query parameter `pageToken` is not set in the request. - + Servers that do not support pagination should ignore the `pageToken` parameter and return all results in a single response. The `next-page-token` must be omitted from the response. - + Clients must interpret either `null` or missing response value of `next-page-token` as the end of the listing results. @@ -2469,7 +2467,7 @@ components: type: string encrypted-key-metadata: type: string - format: byte # for compatibility + format: byte # for compatibility contentEncoding: base64 encrypted-by-id: type: string @@ -3357,21 +3355,21 @@ components: The `config` map returns table-specific configuration for the table's resources, including its HTTP client and FileIO. For example, config may contain a specific FileIO implementation class for the table depending on its underlying storage. - - + + The following configurations should be respected by clients: - + ## General Configurations - - - `token`: Authorization bearer token to use for table requests if OAuth2 security is enabled - + + - `token`: Authorization bearer token to use for table requests if OAuth2 security is enabled + ## AWS Configurations - + The following configurations should be respected when working with tables stored in AWS S3 - `client.region`: region to configure client for making requests to AWS - `s3.access-key-id`: id for credentials that provide access to the data in S3 - - `s3.secret-access-key`: secret for credentials that provide access to data in S3 - - `s3.session-token`: if present, this value should be used for as the session token + - `s3.secret-access-key`: secret for credentials that provide access to data in S3 + - `s3.session-token`: if present, this value should be used for as the session token - `s3.remote-signing-enabled`: if `true` remote signing should be performed as described in the `s3-signer-open-api.yaml` specification - `s3.cross-region-access-enabled`: if `true`, S3 Cross-Region bucket access is enabled @@ -3448,7 +3446,7 @@ components: type: array description: Storage credentials for accessing the files returned in the scan result. - + If the server returns storage credentials as part of the completed scan planning response, the expectation is for the client to use these credentials to read the files returned in the FileScanTasks as part of the scan result. @@ -3697,7 +3695,7 @@ components: OAuthClientCredentialsRequest: deprecated: true - description: + description: The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this spec, see description of the endpoint. @@ -3818,46 +3816,46 @@ components: additionalProperties: $ref: '#/components/schemas/MetricResult' example: { - "metrics": { - "total-planning-duration": { - "count": 1, - "time-unit": "nanoseconds", - "total-duration": 2644235116 - }, - "result-data-files": { - "unit": "count", - "value": 1, - }, - "result-delete-files": { - "unit": "count", - "value": 0, - }, - "total-data-manifests": { - "unit": "count", - "value": 1, - }, - "total-delete-manifests": { - "unit": "count", - "value": 0, - }, - "scanned-data-manifests": { - "unit": "count", - "value": 1, - }, - "skipped-data-manifests": { - "unit": "count", - "value": 0, - }, - "total-file-size-bytes": { - "unit": "bytes", - "value": 10, - }, - "total-delete-file-size-bytes": { - "unit": "bytes", - "value": 0, - } + "metrics": { + "total-planning-duration": { + "count": 1, + "time-unit": "nanoseconds", + "total-duration": 2644235116 + }, + "result-data-files": { + "unit": "count", + "value": 1, + }, + "result-delete-files": { + "unit": "count", + "value": 0, + }, + "total-data-manifests": { + "unit": "count", + "value": 1, + }, + "total-delete-manifests": { + "unit": "count", + "value": 0, + }, + "scanned-data-manifests": { + "unit": "count", + "value": 1, + }, + "skipped-data-manifests": { + "unit": "count", + "value": 0, + }, + "total-file-size-bytes": { + "unit": "bytes", + "value": 10, + }, + "total-delete-file-size-bytes": { + "unit": "bytes", + "value": 0, } } + } ReportMetricsRequest: anyOf: @@ -4196,9 +4194,9 @@ components: DecimalTypeValue: type: string description: - "Decimal type values are serialized as strings. Decimals with a positive scale serialize as numeric plain - text, while decimals with a negative scale use scientific notation and the exponent will be equal to the - negated scale. For instance, a decimal with a positive scale is '123.4500', with zero scale is '2', + "Decimal type values are serialized as strings. Decimals with a positive scale serialize as numeric plain + text, while decimals with a negative scale use scientific notation and the exponent will be equal to the + negated scale. For instance, a decimal with a positive scale is '123.4500', with zero scale is '2', and with a negative scale is '2E+20'" example: "123.4500" @@ -4213,7 +4211,7 @@ components: maxLength: 36 minLength: 36 description: - "UUID type values are serialized as a 36-character lowercase string in standard UUID format as specified + "UUID type values are serialized as a 36-character lowercase string in standard UUID format as specified by RFC-4122" example: "eb26bdb1-a1d8-4aa6-990e-da940875492c" @@ -4239,7 +4237,7 @@ components: TimestampTzTypeValue: type: string description: - "TimestampTz type values follow the 'YYYY-MM-DDTHH:MM:SS.ssssss+00:00' ISO-8601 format with microsecond precision, + "TimestampTz type values follow the 'YYYY-MM-DDTHH:MM:SS.ssssss+00:00' ISO-8601 format with microsecond precision, and a timezone offset (+00:00 for UTC)" example: "2007-12-03T10:15:30.123456+00:00" @@ -4252,14 +4250,14 @@ components: TimestampTzNanoTypeValue: type: string description: - "Timestamp_ns type values follow the 'YYYY-MM-DDTHH:MM:SS.sssssssss+00:00' ISO-8601 format with nanosecond + "Timestamp_ns type values follow the 'YYYY-MM-DDTHH:MM:SS.sssssssss+00:00' ISO-8601 format with nanosecond precision, and a timezone offset (+00:00 for UTC)" example: "2007-12-03T10:15:30.123456789+00:00" FixedTypeValue: type: string description: - "Fixed length type values are stored and serialized as an uppercase hexadecimal string + "Fixed length type values are stored and serialized as an uppercase hexadecimal string preserving the fixed length" example: "78797A" @@ -4285,7 +4283,7 @@ components: example: { "keys": [ 1, 2 ], - "values": [ 100,200 ] + "values": [ 100, 200 ] } ValueMap: @@ -4303,8 +4301,8 @@ components: description: "List of primitive type values, matched to 'keys' by index" example: { - "keys": [ 1, 2 ], - "values": [ 100, "test" ] + "keys": [ 1, 2 ], + "values": [ 100, "test" ] } PrimitiveTypeValue: @@ -4489,9 +4487,9 @@ components: $ref: '#/components/schemas/Expression' min-rows-requested: description: - The minimum number of rows requested for the scan. This is used as a hint - to the server to not have to return more rows than necessary. It is not required - for the server to return that many rows since the scan may not produce that + The minimum number of rows requested for the scan. This is used as a hint + to the server to not have to return more rows than necessary. It is not required + for the server to return that many rows since the scan may not produce that many rows. The server can also return more rows than requested. type: integer format: int64 @@ -4553,7 +4551,7 @@ components: PlanTask: description: An opaque string provided by the REST server that represents a - unit of work to produce file scan tasks for scan planning. This allows + unit of work to produce file scan tasks for scan planning. This allows clients to fetch tasks across multiple requests to accommodate large result sets. type: string @@ -4608,12 +4606,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Malformed request", - "type": "BadRequestException", - "code": 400 - } + "error": { + "message": "Malformed request", + "type": "BadRequestException", + "code": 400 } + } # Note that this is a representative example response for use as a shorthand in the spec. # The fields `message` and `type` as indicated here are not presently prescriptive. @@ -4627,12 +4625,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Not authorized to make this request", - "type": "NotAuthorizedException", - "code": 401 - } + "error": { + "message": "Not authorized to make this request", + "type": "NotAuthorizedException", + "code": 401 } + } # Note that this is a representative example response for use as a shorthand in the spec. # The fields `message` and `type` as indicated here are not presently prescriptive. @@ -4643,12 +4641,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Not authorized to make this request", - "type": "NotAuthorizedException", - "code": 403 - } + "error": { + "message": "Not authorized to make this request", + "type": "NotAuthorizedException", + "code": 403 } + } # Note that this is a representative example response for use as a shorthand in the spec. # The fields `message` and `type` as indicated here are not presently prescriptive. @@ -4659,12 +4657,12 @@ components: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "The server does not support this operation", - "type": "UnsupportedOperationException", - "code": 406 - } + "error": { + "message": "The server does not support this operation", + "type": "UnsupportedOperationException", + "code": 406 } + } IcebergErrorResponse: description: JSON wrapper for all error responses (non-2xx) @@ -4690,9 +4688,9 @@ components: schema: $ref: '#/components/schemas/CreateNamespaceResponse' example: { - "namespace": ["accounting", "tax"], - "properties": { "owner": "Ralph", "created_at": "1452120468" } - } + "namespace": ["accounting", "tax"], + "properties": { "owner": "Ralph", "created_at": "1452120468" } + } GetNamespaceResponse: description: @@ -4737,12 +4735,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Credentials have timed out", - "type": "AuthenticationTimeoutException", - "code": 419 - } + "error": { + "message": "Credentials have timed out", + "type": "AuthenticationTimeoutException", + "code": 419 } + } ServiceUnavailableResponse: description: @@ -4755,12 +4753,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Slow down", - "type": "SlowDownException", - "code": 503 - } + "error": { + "message": "Slow down", + "type": "SlowDownException", + "code": 503 } + } ServerErrorResponse: description: @@ -4772,12 +4770,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Internal Server Error", - "type": "InternalServerError", - "code": 500 - } + "error": { + "message": "Internal Server Error", + "type": "InternalServerError", + "code": 500 } + } UpdateNamespacePropertiesResponse: description: JSON data response for a synchronous update properties request. @@ -4786,10 +4784,10 @@ components: schema: $ref: '#/components/schemas/UpdateNamespacePropertiesResponse' example: { - "updated": [ "owner" ], - "removed": [ "foo" ], - "missing": [ "bar" ] - } + "updated": [ "owner" ], + "removed": [ "foo" ], + "missing": [ "bar" ] + } CreateTableResponse: description: Table metadata result after creating a table @@ -4864,32 +4862,32 @@ components: ListTablesEmptyExample: summary: An empty list for a namespace with no tables value: { - "identifiers": [ ] - } + "identifiers": [ ] + } ListNamespacesEmptyExample: summary: An empty list of namespaces value: { - "namespaces": [ ] - } + "namespaces": [ ] + } ListNamespacesNonEmptyExample: summary: A non-empty list of namespaces value: { - "namespaces": [ - ["accounting", "tax"], - ["accounting", "credits"] - ] - } + "namespaces": [ + ["accounting", "tax"], + ["accounting", "credits"] + ] + } ListTablesNonEmptyExample: summary: A non-empty list of table identifiers value: { - "identifiers": [ - { "namespace": ["accounting", "tax"], "name": "paid" }, - { "namespace": ["accounting", "tax"], "name": "owed" } - ] - } + "identifiers": [ + { "namespace": ["accounting", "tax"], "name": "paid" }, + { "namespace": ["accounting", "tax"], "name": "owed" } + ] + } MultipartNamespaceAsPathVariable: summary: A multi-part namespace, as represented in a path parameter @@ -4902,62 +4900,62 @@ components: NamespaceAlreadyExistsError: summary: The requested namespace already exists value: { - "error": { - "message": "The given namespace already exists", - "type": "AlreadyExistsException", - "code": 409 - } + "error": { + "message": "The given namespace already exists", + "type": "AlreadyExistsException", + "code": 409 } + } NoSuchPlanIdError: summary: The plan id does not exist value: { - "error": { - "message": "The plan id does not exist", - "type": "NoSuchPlanIdException", - "code": 404 - } + "error": { + "message": "The plan id does not exist", + "type": "NoSuchPlanIdException", + "code": 404 } + } NoSuchPlanTaskError: summary: The plan task does not exist value: { - "error": { - "message": "The plan task does not exist", - "type": "NoSuchPlanTaskException", - "code": 404 - } + "error": { + "message": "The plan task does not exist", + "type": "NoSuchPlanTaskException", + "code": 404 } + } NoSuchTableError: summary: The requested table does not exist value: { - "error": { - "message": "The given table does not exist", - "type": "NoSuchTableException", - "code": 404 - } + "error": { + "message": "The given table does not exist", + "type": "NoSuchTableException", + "code": 404 } + } NoSuchViewError: summary: The requested view does not exist value: { - "error": { - "message": "The given view does not exist", - "type": "NoSuchViewException", - "code": 404 - } + "error": { + "message": "The given view does not exist", + "type": "NoSuchViewException", + "code": 404 } + } NoSuchNamespaceError: summary: The requested namespace does not exist value: { - "error": { - "message": "The given namespace does not exist", - "type": "NoSuchNamespaceException", - "code": 404 - } + "error": { + "message": "The given namespace does not exist", + "type": "NoSuchNamespaceException", + "code": 404 } + } NamespaceNotEmptyError: summary: The requested namespace is not empty @@ -4972,36 +4970,36 @@ components: RenameTableSameNamespace: summary: Rename a table in the same namespace value: { - "source": { "namespace": ["accounting", "tax"], "name": "paid" }, - "destination": { "namespace": ["accounting", "tax"], "name": "owed" } - } + "source": { "namespace": ["accounting", "tax"], "name": "paid" }, + "destination": { "namespace": ["accounting", "tax"], "name": "owed" } + } RenameViewSameNamespace: summary: Rename a view in the same namespace value: { - "source": { "namespace": [ "accounting", "tax" ], "name": "paid-view" }, - "destination": { "namespace": [ "accounting", "tax" ], "name": "owed-view" } - } + "source": { "namespace": [ "accounting", "tax" ], "name": "paid-view" }, + "destination": { "namespace": [ "accounting", "tax" ], "name": "owed-view" } + } TableAlreadyExistsError: summary: The requested table identifier already exists value: { - "error": { - "message": "The given table already exists", - "type": "AlreadyExistsException", - "code": 409 - } + "error": { + "message": "The given table already exists", + "type": "AlreadyExistsException", + "code": 409 } + } ViewAlreadyExistsError: summary: The requested view identifier already exists value: { - "error": { - "message": "The given view already exists", - "type": "AlreadyExistsException", - "code": 409 - } + "error": { + "message": "The given view already exists", + "type": "AlreadyExistsException", + "code": 409 } + } # This is an example response and is not meant to be prescriptive regarding the message or type. UnprocessableEntityDuplicateKey: @@ -5009,19 +5007,19 @@ components: The request body either has the same key multiple times in what should be a map with unique keys or the request body has keys in two or more fields which should be disjoint sets. value: { - "error": { - "message": "The request cannot be processed as there is a key present multiple times", - "type": "UnprocessableEntityException", - "code": 422 - } + "error": { + "message": "The request cannot be processed as there is a key present multiple times", + "type": "UnprocessableEntityException", + "code": 422 } + } UpdateAndRemoveNamespacePropertiesRequest: summary: An update namespace properties request with both properties to remove and properties to upsert. value: { - "removals": [ "foo", "bar" ], - "updates": { "owner": "Raoul" } - } + "removals": [ "foo", "bar" ], + "updates": { "owner": "Raoul" } + } securitySchemes: OAuth2: @@ -5034,8 +5032,8 @@ components: 403 response. Implementations must not return altered success (200) responses when a request is unauthenticated or unauthorized. - If a separate authorization server is used, substitute the tokenUrl with - the full token path of the external authorization server, and use the + If a separate authorization server is used, substitute the tokenUrl with + the full token path of the external authorization server, and use the resulting token to access the resources defined in the spec. flows: clientCredentials: From 73f8ab814dd37ac4590c20ded26e96dc739ad767 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Fri, 5 Dec 2025 17:41:53 +0100 Subject: [PATCH 046/201] Core: Reference IRC to return 204 (#14724) There are some endpoints where the spec requires 204 status code to indicate success, however, the reference IRC implementation returns 200 instead. This PR is to eliminate this deviation from spec. --- .../apache/iceberg/rest/RESTCatalogServlet.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java index 6996db4805c2..beb1401c2249 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java @@ -121,6 +121,8 @@ protected void execute(ServletRequestContext context, HttpServletResponse respon Route route = routeAndVars.first(); if (route == Route.LOAD_TABLE) { response.setStatus(HttpServletResponse.SC_NOT_MODIFIED); + } else if (shouldReturn204(route)) { + response.setStatus(HttpServletResponse.SC_NO_CONTENT); } } } @@ -133,6 +135,20 @@ protected void execute(ServletRequestContext context, HttpServletResponse respon } } + private boolean shouldReturn204(Route route) { + return route == Route.NAMESPACE_EXISTS + || route == Route.TABLE_EXISTS + || route == Route.VIEW_EXISTS + || route == Route.DROP_NAMESPACE + || route == Route.DROP_TABLE + || route == Route.DROP_VIEW + || route == Route.RENAME_TABLE + || route == Route.RENAME_VIEW + || route == Route.CANCEL_PLAN_TABLE_SCAN + || route == Route.REPORT_METRICS + || route == Route.COMMIT_TRANSACTION; + } + protected Consumer handle(HttpServletResponse response) { return (errorResponse) -> { response.setStatus(errorResponse.code()); From 7bac8650f65279c470d7d2c005c40a858933134a Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 5 Dec 2025 09:23:37 -0800 Subject: [PATCH 047/201] Core: Send Idempotency-Key on mutation requests when advertised (#14740) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Core: REST client – send Idempotency-Key on mutation requests when advertised * rebase * address comments * address comments --- .../iceberg/rest/RESTSessionCatalog.java | 85 ++++++++++++----- .../iceberg/rest/RESTTableOperations.java | 54 ++++++++++- .../org/apache/iceberg/rest/RESTUtil.java | 11 +++ .../iceberg/rest/RESTViewOperations.java | 24 ++++- .../apache/iceberg/rest/TestRESTCatalog.java | 92 ++++++++++++++++++- .../iceberg/rest/TestRESTViewCatalog.java | 6 +- 6 files changed, 235 insertions(+), 37 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 85b04f3868cd..fe0d13217b62 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -160,6 +160,7 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private Integer pageSize = null; private CloseableGroup closeables = null; private Set endpoints; + private Supplier> mutationHeaders = Map::of; public RESTSessionCatalog() { this( @@ -203,6 +204,11 @@ public void initialize(String name, Map unresolved) { // build the final configuration and set up the catalog's auth Map mergedProps = config.merge(props); + // Enable Idempotency-Key header for mutation endpoints if the server advertises support + if (config.idempotencyKeyLifetime() != null) { + this.mutationHeaders = RESTUtil::idempotencyHeaders; + } + if (config.endpoints().isEmpty()) { this.endpoints = PropertyUtil.propertyAsBoolean( @@ -307,7 +313,8 @@ public boolean dropTable(SessionContext context, TableIdentifier identifier) { AuthSession contextualSession = authManager.contextualSession(context, catalogAuth); client .withAuthSession(contextualSession) - .delete(paths.table(identifier), null, Map.of(), ErrorHandlers.tableErrorHandler()); + .delete( + paths.table(identifier), null, mutationHeaders, ErrorHandlers.tableErrorHandler()); return true; } catch (NoSuchTableException e) { return false; @@ -327,7 +334,7 @@ public boolean purgeTable(SessionContext context, TableIdentifier identifier) { paths.table(identifier), ImmutableMap.of("purgeRequested", "true"), null, - Map.of(), + mutationHeaders, ErrorHandlers.tableErrorHandler()); return true; } catch (NoSuchTableException e) { @@ -348,7 +355,7 @@ public void renameTable(SessionContext context, TableIdentifier from, TableIdent AuthSession contextualSession = authManager.contextualSession(context, catalogAuth); client .withAuthSession(contextualSession) - .post(paths.rename(), request, null, Map.of(), ErrorHandlers.tableErrorHandler()); + .post(paths.rename(), request, null, mutationHeaders, ErrorHandlers.tableErrorHandler()); } @Override @@ -455,6 +462,7 @@ public Table loadTable(SessionContext context, TableIdentifier identifier) { tableClient, paths.table(finalIdentifier), Map::of, + mutationHeaders, tableFileIO(context, tableConf, response.credentials()), tableMetadata, endpoints); @@ -523,7 +531,7 @@ public Table registerTable( paths.register(ident.namespace()), request, LoadTableResponse.class, - Map.of(), + mutationHeaders, ErrorHandlers.tableErrorHandler()); Map tableConf = response.config(); @@ -534,6 +542,7 @@ public Table registerTable( tableClient, paths.table(ident), Map::of, + mutationHeaders, tableFileIO(context, tableConf, response.credentials()), response.tableMetadata(), endpoints); @@ -559,7 +568,7 @@ public void createNamespace( paths.namespaces(), request, CreateNamespaceResponse.class, - Map.of(), + mutationHeaders, ErrorHandlers.namespaceErrorHandler()); } @@ -645,7 +654,11 @@ public boolean dropNamespace(SessionContext context, Namespace ns) { AuthSession contextualSession = authManager.contextualSession(context, catalogAuth); client .withAuthSession(contextualSession) - .delete(paths.namespace(ns), null, Map.of(), ErrorHandlers.dropNamespaceErrorHandler()); + .delete( + paths.namespace(ns), + null, + mutationHeaders, + ErrorHandlers.dropNamespaceErrorHandler()); return true; } catch (NoSuchNamespaceException e) { return false; @@ -669,7 +682,7 @@ public boolean updateNamespaceMetadata( paths.namespaceProperties(ns), request, UpdateNamespacePropertiesResponse.class, - Map.of(), + mutationHeaders, ErrorHandlers.namespaceErrorHandler()); return !response.updated().isEmpty(); @@ -782,7 +795,7 @@ public Table create() { paths.tables(ident.namespace()), request, LoadTableResponse.class, - Map.of(), + mutationHeaders, ErrorHandlers.tableErrorHandler()); Map tableConf = response.config(); @@ -793,6 +806,7 @@ public Table create() { tableClient, paths.table(ident), Map::of, + mutationHeaders, tableFileIO(context, tableConf, response.credentials()), response.tableMetadata(), endpoints); @@ -820,6 +834,7 @@ public Transaction createTransaction() { tableClient, paths.table(ident), Map::of, + mutationHeaders, tableFileIO(context, tableConf, response.credentials()), RESTTableOperations.UpdateType.CREATE, createChanges(meta), @@ -883,6 +898,7 @@ public Transaction replaceTransaction() { tableClient, paths.table(ident), Map::of, + mutationHeaders, tableFileIO(context, tableConf, response.credentials()), RESTTableOperations.UpdateType.REPLACE, changes.build(), @@ -932,7 +948,7 @@ private LoadTableResponse stageCreate() { paths.tables(ident.namespace()), request, LoadTableResponse.class, - Map.of(), + mutationHeaders, ErrorHandlers.tableErrorHandler()); } } @@ -1019,7 +1035,10 @@ private FileIO tableFileIO( * * @param restClient the REST client to use for communicating with the catalog server * @param path the REST path for the table - * @param headers a supplier for additional HTTP headers to include in requests + * @param readHeaders a supplier for additional HTTP headers to include in read requests + * (GET/HEAD) + * @param mutationHeaderSupplier a supplier for additional HTTP headers to include in mutation + * requests (POST/DELETE) * @param fileIO the FileIO implementation for reading and writing table metadata and data files * @param current the current table metadata * @param supportedEndpoints the set of supported REST endpoints @@ -1028,11 +1047,13 @@ private FileIO tableFileIO( protected RESTTableOperations newTableOps( RESTClient restClient, String path, - Supplier> headers, + Supplier> readHeaders, + Supplier> mutationHeaderSupplier, FileIO fileIO, TableMetadata current, Set supportedEndpoints) { - return new RESTTableOperations(restClient, path, headers, fileIO, current, supportedEndpoints); + return new RESTTableOperations( + restClient, path, readHeaders, mutationHeaderSupplier, fileIO, current, supportedEndpoints); } /** @@ -1044,7 +1065,10 @@ protected RESTTableOperations newTableOps( * * @param restClient the REST client to use for communicating with the catalog server * @param path the REST path for the table - * @param headers a supplier for additional HTTP headers to include in requests + * @param readHeaders a supplier for additional HTTP headers to include in read requests + * (GET/HEAD) + * @param mutationHeaderSupplier a supplier for additional HTTP headers to include in mutation + * requests (POST/DELETE) * @param fileIO the FileIO implementation for reading and writing table metadata and data files * @param updateType the {@link RESTTableOperations.UpdateType} being performed * @param createChanges the list of metadata updates to apply during table creation or replacement @@ -1055,14 +1079,23 @@ protected RESTTableOperations newTableOps( protected RESTTableOperations newTableOps( RESTClient restClient, String path, - Supplier> headers, + Supplier> readHeaders, + Supplier> mutationHeaderSupplier, FileIO fileIO, RESTTableOperations.UpdateType updateType, List createChanges, TableMetadata current, Set supportedEndpoints) { return new RESTTableOperations( - restClient, path, headers, fileIO, updateType, createChanges, current, supportedEndpoints); + restClient, + path, + readHeaders, + mutationHeaderSupplier, + fileIO, + updateType, + createChanges, + current, + supportedEndpoints); } /** @@ -1073,7 +1106,10 @@ protected RESTTableOperations newTableOps( * * @param restClient the REST client to use for communicating with the catalog server * @param path the REST path for the view - * @param headers a supplier for additional HTTP headers to include in requests + * @param readHeaders a supplier for additional HTTP headers to include in read requests + * (GET/HEAD) + * @param mutationHeaderSupplier a supplier for additional HTTP headers to include in mutation + * requests (POST/DELETE) * @param current the current view metadata * @param supportedEndpoints the set of supported REST endpoints * @return a new RESTViewOperations instance @@ -1081,10 +1117,12 @@ protected RESTTableOperations newTableOps( protected RESTViewOperations newViewOps( RESTClient restClient, String path, - Supplier> headers, + Supplier> readHeaders, + Supplier> mutationHeaderSupplier, ViewMetadata current, Set supportedEndpoints) { - return new RESTViewOperations(restClient, path, headers, current, supportedEndpoints); + return new RESTViewOperations( + restClient, path, readHeaders, mutationHeaderSupplier, current, supportedEndpoints); } private static ConfigResponse fetchConfig( @@ -1148,7 +1186,7 @@ public void commitTransaction(SessionContext context, List commits) paths.commitTransaction(), new CommitTransactionRequest(tableChanges), null, - Map.of(), + mutationHeaders, ErrorHandlers.tableCommitHandler()); } @@ -1235,6 +1273,7 @@ public View loadView(SessionContext context, TableIdentifier identifier) { client.withAuthSession(tableSession), paths.view(identifier), Map::of, + mutationHeaders, metadata, endpoints); @@ -1255,7 +1294,7 @@ public boolean dropView(SessionContext context, TableIdentifier identifier) { AuthSession contextualSession = authManager.contextualSession(context, catalogAuth); client .withAuthSession(contextualSession) - .delete(paths.view(identifier), null, Map.of(), ErrorHandlers.viewErrorHandler()); + .delete(paths.view(identifier), null, mutationHeaders, ErrorHandlers.viewErrorHandler()); return true; } catch (NoSuchViewException e) { return false; @@ -1274,7 +1313,7 @@ public void renameView(SessionContext context, TableIdentifier from, TableIdenti AuthSession contextualSession = authManager.contextualSession(context, catalogAuth); client .withAuthSession(contextualSession) - .post(paths.renameView(), request, null, Map.of(), ErrorHandlers.viewErrorHandler()); + .post(paths.renameView(), request, null, mutationHeaders, ErrorHandlers.viewErrorHandler()); } private class RESTViewBuilder implements ViewBuilder { @@ -1404,7 +1443,7 @@ public View create() { paths.views(identifier.namespace()), request, LoadViewResponse.class, - Map.of(), + mutationHeaders, ErrorHandlers.viewErrorHandler()); Map tableConf = response.config(); @@ -1414,6 +1453,7 @@ public View create() { client.withAuthSession(tableSession), paths.view(identifier), Map::of, + mutationHeaders, response.metadata(), endpoints); @@ -1505,6 +1545,7 @@ private View replace(LoadViewResponse response) { client.withAuthSession(tableSession), paths.view(identifier), Map::of, + mutationHeaders, metadata, endpoints); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java b/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java index 38dabc8ae568..d2a6ab618ca8 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java @@ -55,7 +55,8 @@ enum UpdateType { private final RESTClient client; private final String path; - private final Supplier> headers; + private final Supplier> readHeaders; + private final Supplier> mutationHeaders; private final FileIO io; private final List createChanges; private final TableMetadata replaceBase; @@ -70,7 +71,16 @@ enum UpdateType { FileIO io, TableMetadata current, Set endpoints) { - this(client, path, headers, io, UpdateType.SIMPLE, Lists.newArrayList(), current, endpoints); + this( + client, + path, + headers, + headers, + io, + UpdateType.SIMPLE, + Lists.newArrayList(), + current, + endpoints); } RESTTableOperations( @@ -82,9 +92,43 @@ enum UpdateType { List createChanges, TableMetadata current, Set endpoints) { + this(client, path, headers, headers, io, updateType, createChanges, current, endpoints); + } + + RESTTableOperations( + RESTClient client, + String path, + Supplier> readHeaders, + Supplier> mutationHeaders, + FileIO io, + TableMetadata current, + Set endpoints) { + this( + client, + path, + readHeaders, + mutationHeaders, + io, + UpdateType.SIMPLE, + Lists.newArrayList(), + current, + endpoints); + } + + RESTTableOperations( + RESTClient client, + String path, + Supplier> readHeaders, + Supplier> mutationHeaders, + FileIO io, + UpdateType updateType, + List createChanges, + TableMetadata current, + Set endpoints) { this.client = client; this.path = path; - this.headers = headers; + this.readHeaders = readHeaders; + this.mutationHeaders = mutationHeaders; this.io = io; this.updateType = updateType; this.createChanges = createChanges; @@ -106,7 +150,7 @@ public TableMetadata current() { public TableMetadata refresh() { Endpoint.check(endpoints, Endpoint.V1_LOAD_TABLE); return updateCurrentMetadata( - client.get(path, LoadTableResponse.class, headers, ErrorHandlers.tableErrorHandler())); + client.get(path, LoadTableResponse.class, readHeaders, ErrorHandlers.tableErrorHandler())); } @Override @@ -159,7 +203,7 @@ public void commit(TableMetadata base, TableMetadata metadata) { // TODO: ensure that the HTTP client lib passes HTTP client errors to the error handler LoadTableResponse response; try { - response = client.post(path, request, LoadTableResponse.class, headers, errorHandler); + response = client.post(path, request, LoadTableResponse.class, mutationHeaders, errorHandler); } catch (CommitStateUnknownException e) { // Lightweight reconciliation for snapshot-add-only updates on transient unknown commit state if (updateType == UpdateType.SIMPLE && reconcileOnSimpleUpdate(updates, e)) { diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java b/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java index 6ca04e74d9f5..ec02a9dc8459 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java @@ -29,6 +29,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.UUIDUtil; public class RESTUtil { private static final char NAMESPACE_SEPARATOR = '\u001f'; @@ -49,6 +50,8 @@ public class RESTUtil { */ @Deprecated public static final Splitter NAMESPACE_SPLITTER = Splitter.on(NAMESPACE_SEPARATOR); + public static final String IDEMPOTENCY_KEY_HEADER = "Idempotency-Key"; + private RESTUtil() {} public static String stripTrailingSlash(String path) { @@ -271,4 +274,12 @@ public static String resolveEndpoint(String catalogUri, String endpointPath) { public static Map configHeaders(Map properties) { return RESTUtil.extractPrefixMap(properties, "header."); } + + /** + * Returns a single-use headers map containing a freshly generated idempotency key. The key is a + * UUIDv7 string suitable for use in the Idempotency-Key header. + */ + public static Map idempotencyHeaders() { + return ImmutableMap.of(IDEMPOTENCY_KEY_HEADER, UUIDUtil.generateUuidV7().toString()); + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java b/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java index 466a8e66899b..0018e2f91903 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java @@ -32,7 +32,8 @@ class RESTViewOperations implements ViewOperations { private final RESTClient client; private final String path; - private final Supplier> headers; + private final Supplier> readHeaders; + private final Supplier> mutationHeaders; private final Set endpoints; private ViewMetadata current; @@ -42,10 +43,21 @@ class RESTViewOperations implements ViewOperations { Supplier> headers, ViewMetadata current, Set endpoints) { + this(client, path, headers, headers, current, endpoints); + } + + RESTViewOperations( + RESTClient client, + String path, + Supplier> readHeaders, + Supplier> mutationHeaders, + ViewMetadata current, + Set endpoints) { Preconditions.checkArgument(null != current, "Invalid view metadata: null"); this.client = client; this.path = path; - this.headers = headers; + this.readHeaders = readHeaders; + this.mutationHeaders = mutationHeaders; this.current = current; this.endpoints = endpoints; } @@ -59,7 +71,7 @@ public ViewMetadata current() { public ViewMetadata refresh() { Endpoint.check(endpoints, Endpoint.V1_LOAD_VIEW); return updateCurrentMetadata( - client.get(path, LoadViewResponse.class, headers, ErrorHandlers.viewErrorHandler())); + client.get(path, LoadViewResponse.class, readHeaders, ErrorHandlers.viewErrorHandler())); } @Override @@ -74,7 +86,11 @@ public void commit(ViewMetadata base, ViewMetadata metadata) { LoadViewResponse response = client.post( - path, request, LoadViewResponse.class, headers, ErrorHandlers.viewCommitHandler()); + path, + request, + LoadViewResponse.class, + mutationHeaders, + ErrorHandlers.viewCommitHandler()); updateCurrentMetadata(response); } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index efe76e2bf060..385893ea7130 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -37,6 +37,7 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.nio.file.Path; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -50,6 +51,7 @@ import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.http.HttpHeaders; import org.apache.iceberg.BaseTable; @@ -167,6 +169,7 @@ public T execute( assertThat(request.headers().entries()).containsAll(contextHeaders.entries()); } } + Object body = roundTripSerialize(request.body(), "request"); HTTPRequest req = ImmutableHTTPRequest.builder().from(request).body(body).build(); T response = super.execute(req, responseType, errorHandler, responseHeaders); @@ -3170,13 +3173,14 @@ class CustomRESTSessionCatalog extends RESTSessionCatalog { protected RESTTableOperations newTableOps( RESTClient restClient, String path, - Supplier> headers, + Supplier> readHeaders, + Supplier> mutationHeaders, FileIO fileIO, TableMetadata current, Set supportedEndpoints) { RESTTableOperations ops = new CustomRESTTableOperations( - restClient, path, headers, fileIO, current, supportedEndpoints); + restClient, path, mutationHeaders, fileIO, current, supportedEndpoints); RESTTableOperations spy = Mockito.spy(ops); capturedOps.set(spy); return spy; @@ -3186,7 +3190,8 @@ protected RESTTableOperations newTableOps( protected RESTTableOperations newTableOps( RESTClient restClient, String path, - Supplier> headers, + Supplier> readHeaders, + Supplier> mutationHeaders, FileIO fileIO, RESTTableOperations.UpdateType updateType, List createChanges, @@ -3196,7 +3201,7 @@ protected RESTTableOperations newTableOps( new CustomRESTTableOperations( restClient, path, - headers, + mutationHeaders, fileIO, updateType, createChanges, @@ -3261,6 +3266,85 @@ protected RESTTableOperations newTableOps( } } + @Test + public void testClientAutoSendsIdempotencyWhenServerAdvertises() { + ConfigResponse cfgWithIdem = + ConfigResponse.builder() + .withIdempotencyKeyLifetime("PT30M") + .withEndpoints( + Arrays.stream(Route.values()) + .map(r -> Endpoint.create(r.method().name(), r.resourcePath())) + .collect(Collectors.toList())) + .build(); + + RESTCatalog local = createCatalogWithIdempAdapter(cfgWithIdem, true); + + Namespace ns = Namespace.of("ns_cfg_idem"); + TableIdentifier ident = TableIdentifier.of(ns, "t_cfg_idem"); + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + local.createNamespace(ns, ImmutableMap.of()); + local.createTable(ident, schema); + assertThat(local.tableExists(ident)).isTrue(); + local.dropTable(ident); + } + + @Test + public void testClientDoesNotSendIdempotencyWhenServerNotAdvertising() { + ConfigResponse cfgNoIdem = + ConfigResponse.builder() + .withEndpoints( + Arrays.stream(Route.values()) + .map(r -> Endpoint.create(r.method().name(), r.resourcePath())) + .collect(Collectors.toList())) + .build(); + + RESTCatalog local = createCatalogWithIdempAdapter(cfgNoIdem, false); + + Namespace ns = Namespace.of("ns_cfg_no_idem"); + TableIdentifier ident = TableIdentifier.of(ns, "t_cfg_no_idem"); + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + local.createNamespace(ns, ImmutableMap.of()); + local.createTable(ident, schema); + assertThat(local.tableExists(ident)).isTrue(); + local.dropTable(ident); + } + + private RESTCatalog createCatalogWithIdempAdapter(ConfigResponse cfg, boolean expectOnMutations) { + RESTCatalogAdapter adapter = + Mockito.spy( + new RESTCatalogAdapter(backendCatalog) { + @Override + public T execute( + HTTPRequest request, + Class responseType, + Consumer errorHandler, + Consumer> responseHeaders) { + if (ResourcePaths.config().equals(request.path())) { + return castResponse(responseType, cfg); + } + + boolean isMutation = + request.method() == HTTPMethod.POST || request.method() == HTTPMethod.DELETE; + boolean hasIdemp = request.headers().contains(RESTUtil.IDEMPOTENCY_KEY_HEADER); + + if (isMutation) { + assertThat(hasIdemp) + .as("Idempotency-Key presence on mutations did not match expectation") + .isEqualTo(expectOnMutations); + } else { + assertThat(hasIdemp).as("Idempotency-Key must NOT be sent on reads").isFalse(); + } + + return super.execute(request, responseType, errorHandler, responseHeaders); + } + }); + + RESTCatalog local = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + local.initialize("test", ImmutableMap.of()); + return local; + } + private RESTCatalog catalog(RESTCatalogAdapter adapter) { RESTCatalog catalog = new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java index 6b39907098f9..8bfe26b18cda 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java @@ -350,11 +350,13 @@ class CustomRESTSessionCatalog extends RESTSessionCatalog { protected RESTViewOperations newViewOps( RESTClient restClient, String path, - Supplier> headers, + Supplier> readHeaders, + Supplier> mutationHeaders, ViewMetadata current, Set supportedEndpoints) { RESTViewOperations ops = - new CustomRESTViewOperations(restClient, path, headers, current, supportedEndpoints); + new CustomRESTViewOperations( + restClient, path, mutationHeaders, current, supportedEndpoints); RESTViewOperations spy = Mockito.spy(ops); capturedOps.set(spy); return spy; From 4c908314ec3a81a04544784d8a1b69f76fc395e5 Mon Sep 17 00:00:00 2001 From: pvary Date: Fri, 5 Dec 2025 19:33:15 +0100 Subject: [PATCH 048/201] Spark: ORC vectorized reader to use the delete filter (#14746) --- .../vectorized/VectorizedSparkOrcReaders.java | 4 +- .../iceberg/spark/data/TestHelpers.java | 71 +++++++++++++++++ .../data/TestSparkOrcReadMetadataColumns.java | 54 ++++++++++--- .../TestSparkParquetReadMetadataColumns.java | 78 ++----------------- 4 files changed, 125 insertions(+), 82 deletions(-) diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java index 5f68c233f60d..4f324239881e 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java @@ -462,7 +462,9 @@ public ColumnVector convert( } else if (field.equals(MetadataColumns.ROW_POSITION)) { fieldVectors.add(new RowPositionColumnVector(batchOffsetInFile)); } else if (field.equals(MetadataColumns.IS_DELETED)) { - fieldVectors.add(new ConstantColumnVector(field.type(), batchSize, false)); + DeletedColumnVector deletedVector = new DeletedColumnVector(field.type()); + deletedVector.setValue(new boolean[batchSize]); + fieldVectors.add(deletedVector); } else if (field.type().equals(Types.UnknownType.get())) { fieldVectors.add(new ConstantColumnVector(field.type(), batchSize, null)); } else { diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 6bacf6470173..91606d16a0a0 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -52,10 +52,16 @@ import org.apache.iceberg.ManifestFile; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableScan; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.deletes.DeleteCounter; +import org.apache.iceberg.deletes.PositionDeleteIndex; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Type; @@ -883,4 +889,69 @@ public static Dataset selectNonDerived(Dataset metadataTable) { public static Types.StructType nonDerivedSchema(Dataset metadataTable) { return SparkSchemaUtil.convert(TestHelpers.selectNonDerived(metadataTable).schema()).asStruct(); } + + public static class CustomizedDeleteFilter extends DeleteFilter { + private final boolean hasDeletes; + + protected CustomizedDeleteFilter( + boolean hasDeletes, Schema tableSchema, Schema projectedSchema) { + super("", List.of(), tableSchema, projectedSchema, new DeleteCounter(), true); + this.hasDeletes = hasDeletes; + } + + @Override + protected StructLike asStructLike(InternalRow record) { + return null; + } + + @Override + protected InputFile getInputFile(String location) { + return null; + } + + @Override + public boolean hasPosDeletes() { + return hasDeletes; + } + + @Override + public PositionDeleteIndex deletedRowPositions() { + PositionDeleteIndex deletedRowPos = new CustomizedPositionDeleteIndex(); + if (hasDeletes) { + deletedRowPos.delete(98, 103); + } + + return deletedRowPos; + } + } + + public static class CustomizedPositionDeleteIndex implements PositionDeleteIndex { + private final Set deleteIndex; + + private CustomizedPositionDeleteIndex() { + deleteIndex = Sets.newHashSet(); + } + + @Override + public void delete(long position) { + deleteIndex.add(position); + } + + @Override + public void delete(long posStart, long posEnd) { + for (long l = posStart; l < posEnd; l++) { + delete(l); + } + } + + @Override + public boolean isDeleted(long position) { + return deleteIndex.contains(position); + } + + @Override + public boolean isEmpty() { + return deleteIndex.isEmpty(); + } + } } diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java index 9d725250d3d2..13acaa1e3a7b 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -36,6 +37,7 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -46,6 +48,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; +import org.apache.iceberg.spark.source.BatchReaderUtil; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.orc.OrcConf; @@ -74,7 +77,11 @@ public class TestSparkOrcReadMetadataColumns { MetadataColumns.ROW_POSITION, MetadataColumns.IS_DELETED); + private static final DeleteFilter NO_DELETES_FILTER = + new TestHelpers.CustomizedDeleteFilter(false, DATA_SCHEMA, PROJECTION_SCHEMA); + private static final int NUM_ROWS = 1000; + private static final int RECORDS_PER_BATCH = 10; private static final List DATA_ROWS; private static final List EXPECTED_ROWS; @@ -128,13 +135,35 @@ public void writeFile() throws IOException { @TestTemplate public void testReadRowNumbers() throws IOException { - readAndValidate(null, null, null, EXPECTED_ROWS); + readAndValidate(null, null, null, EXPECTED_ROWS, NO_DELETES_FILTER); + } + + @TestTemplate + public void testReadRowNumbersWithDelete() throws IOException { + assumeThat(vectorized).isTrue(); + + List expectedRowsAfterDelete = Lists.newArrayList(); + EXPECTED_ROWS.forEach(row -> expectedRowsAfterDelete.add(row.copy())); + // remove row at position 98, 99, 100, 101, 102, this crosses two row groups [0, 100) and [100, + // 200) + for (int i = 98; i <= 102; i++) { + expectedRowsAfterDelete.get(i).update(3, true); + } + + DeleteFilter deleteFilter = + new TestHelpers.CustomizedDeleteFilter(true, DATA_SCHEMA, PROJECTION_SCHEMA); + + readAndValidate(null, null, null, expectedRowsAfterDelete, deleteFilter); } @TestTemplate public void testReadRowNumbersWithFilter() throws IOException { readAndValidate( - Expressions.greaterThanOrEqual("id", 500), null, null, EXPECTED_ROWS.subList(500, 1000)); + Expressions.greaterThanOrEqual("id", 500), + null, + null, + EXPECTED_ROWS.subList(500, 1000), + NO_DELETES_FILTER); } @TestTemplate @@ -157,12 +186,17 @@ public void testReadRowNumbersWithSplits() throws IOException { null, splitOffsets.get(i), splitLengths.get(i), - EXPECTED_ROWS.subList(i * 100, (i + 1) * 100)); + EXPECTED_ROWS.subList(i * 100, (i + 1) * 100), + NO_DELETES_FILTER); } } private void readAndValidate( - Expression filter, Long splitStart, Long splitLength, List expected) + Expression filter, + Long splitStart, + Long splitLength, + List expected, + DeleteFilter deleteFilter) throws IOException { Schema projectionWithoutMetadataFields = TypeUtil.selectNot(PROJECTION_SCHEMA, MetadataColumns.metadataFieldIds()); @@ -173,10 +207,12 @@ private void readAndValidate( if (vectorized) { builder = - builder.createBatchedReaderFunc( - readOrcSchema -> - VectorizedSparkOrcReaders.buildReader( - PROJECTION_SCHEMA, readOrcSchema, ImmutableMap.of())); + builder + .recordsPerBatch(RECORDS_PER_BATCH) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + PROJECTION_SCHEMA, readOrcSchema, ImmutableMap.of())); } else { builder = builder.createReaderFunc( @@ -192,7 +228,7 @@ private void readAndValidate( } if (vectorized) { - reader = batchesToRows(builder.build()); + reader = batchesToRows(BatchReaderUtil.applyDeleteFilter(builder.build(), deleteFilter)); } else { reader = builder.build(); } diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java index ccd783915c62..e2e5a98ccb8b 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java @@ -26,7 +26,6 @@ import java.io.IOException; import java.util.Iterator; import java.util.List; -import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.Files; @@ -35,21 +34,16 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; import org.apache.iceberg.data.DeleteFilter; -import org.apache.iceberg.deletes.DeleteCounter; -import org.apache.iceberg.deletes.PositionDeleteIndex; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.InputFile; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.parquet.ParquetSchemaUtil; 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.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.spark.source.BatchReaderUtil; @@ -183,7 +177,8 @@ public void testReadRowNumbersWithDelete() throws IOException { Parquet.ReadBuilder builder = Parquet.read(Files.localInput(testFile)).project(PROJECTION_SCHEMA); - DeleteFilter deleteFilter = new TestDeleteFilter(true); + DeleteFilter deleteFilter = + new TestHelpers.CustomizedDeleteFilter(true, DATA_SCHEMA, PROJECTION_SCHEMA); builder.createBatchedReaderFunc( fileSchema -> @@ -194,70 +189,6 @@ public void testReadRowNumbersWithDelete() throws IOException { validate(expectedRowsAfterDelete, builder, deleteFilter); } - private static class TestDeleteFilter extends DeleteFilter { - private final boolean hasDeletes; - - protected TestDeleteFilter(boolean hasDeletes) { - super("", List.of(), DATA_SCHEMA, PROJECTION_SCHEMA, new DeleteCounter(), true); - this.hasDeletes = hasDeletes; - } - - @Override - protected StructLike asStructLike(InternalRow record) { - return null; - } - - @Override - protected InputFile getInputFile(String location) { - return null; - } - - @Override - public boolean hasPosDeletes() { - return hasDeletes; - } - - @Override - public PositionDeleteIndex deletedRowPositions() { - PositionDeleteIndex deletedRowPos = new CustomizedPositionDeleteIndex(); - if (hasDeletes) { - deletedRowPos.delete(98, 103); - } - - return deletedRowPos; - } - } - - private static class CustomizedPositionDeleteIndex implements PositionDeleteIndex { - private final Set deleteIndex; - - private CustomizedPositionDeleteIndex() { - deleteIndex = Sets.newHashSet(); - } - - @Override - public void delete(long position) { - deleteIndex.add(position); - } - - @Override - public void delete(long posStart, long posEnd) { - for (long l = posStart; l < posEnd; l++) { - delete(l); - } - } - - @Override - public boolean isDeleted(long position) { - return deleteIndex.contains(position); - } - - @Override - public boolean isEmpty() { - return deleteIndex.isEmpty(); - } - } - @TestTemplate public void testReadRowNumbersWithFilter() throws IOException { // current iceberg supports row group filter. @@ -314,7 +245,10 @@ private void readAndValidate( builder = builder.split(splitStart, splitLength); } - validate(expected, builder, new TestDeleteFilter(false)); + validate( + expected, + builder, + new TestHelpers.CustomizedDeleteFilter(false, DATA_SCHEMA, PROJECTION_SCHEMA)); } private void validate( From 9632a2f76061643a6cc51a29acb5d1f4c08cf2d3 Mon Sep 17 00:00:00 2001 From: Anurag Mantripragada Date: Fri, 5 Dec 2025 12:42:44 -0800 Subject: [PATCH 049/201] AWS: Configure builder for reuse of http connection pool in SDKv2 (#14161) AWS: Configure builder for reuse of http clients in SDKv2 --- .../aws/ApacheHttpClientConfigurations.java | 39 ++- .../aws/BaseHttpClientConfigurations.java | 75 +++++ .../apache/iceberg/aws/HttpClientCache.java | 203 +++++++++++++ ...UrlConnectionHttpClientConfigurations.java | 32 +- .../iceberg/aws/TestHttpClientCache.java | 276 ++++++++++++++++++ .../iceberg/aws/TestHttpClientProperties.java | 67 ++++- 6 files changed, 668 insertions(+), 24 deletions(-) create mode 100644 aws/src/main/java/org/apache/iceberg/aws/BaseHttpClientConfigurations.java create mode 100644 aws/src/main/java/org/apache/iceberg/aws/HttpClientCache.java create mode 100644 aws/src/test/java/org/apache/iceberg/aws/TestHttpClientCache.java diff --git a/aws/src/main/java/org/apache/iceberg/aws/ApacheHttpClientConfigurations.java b/aws/src/main/java/org/apache/iceberg/aws/ApacheHttpClientConfigurations.java index 95fe34b742ff..3445928d1551 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/ApacheHttpClientConfigurations.java +++ b/aws/src/main/java/org/apache/iceberg/aws/ApacheHttpClientConfigurations.java @@ -21,13 +21,16 @@ import java.net.URI; import java.time.Duration; import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.PropertyUtil; -import software.amazon.awssdk.awscore.client.builder.AwsSyncClientBuilder; +import software.amazon.awssdk.http.SdkHttpClient; import software.amazon.awssdk.http.apache.ApacheHttpClient; import software.amazon.awssdk.http.apache.ProxyConfiguration; -class ApacheHttpClientConfigurations { +class ApacheHttpClientConfigurations extends BaseHttpClientConfigurations { private Long connectionTimeoutMs; private Long socketTimeoutMs; private Long acquisitionTimeoutMs; @@ -41,10 +44,11 @@ class ApacheHttpClientConfigurations { private ApacheHttpClientConfigurations() {} - public void configureHttpClientBuilder(T awsClientBuilder) { - ApacheHttpClient.Builder apacheHttpClientBuilder = ApacheHttpClient.builder(); + @Override + protected SdkHttpClient buildHttpClient() { + final ApacheHttpClient.Builder apacheHttpClientBuilder = ApacheHttpClient.builder(); configureApacheHttpClientBuilder(apacheHttpClientBuilder); - awsClientBuilder.httpClientBuilder(apacheHttpClientBuilder); + return apacheHttpClientBuilder.build(); } private void initialize(Map httpClientProperties) { @@ -115,6 +119,31 @@ void configureApacheHttpClientBuilder(ApacheHttpClient.Builder apacheHttpClientB } } + /** + * Generate a cache key based on HTTP client configuration. This ensures clients with identical + * configurations share the same HTTP client instance. + */ + @Override + protected String generateHttpClientCacheKey() { + Map keyComponents = Maps.newTreeMap(); + + keyComponents.put("type", "apache"); + keyComponents.put("connectionTimeoutMs", connectionTimeoutMs); + keyComponents.put("socketTimeoutMs", socketTimeoutMs); + keyComponents.put("acquisitionTimeoutMs", acquisitionTimeoutMs); + keyComponents.put("connectionMaxIdleTimeMs", connectionMaxIdleTimeMs); + keyComponents.put("connectionTimeToLiveMs", connectionTimeToLiveMs); + keyComponents.put("expectContinueEnabled", expectContinueEnabled); + keyComponents.put("maxConnections", maxConnections); + keyComponents.put("tcpKeepAliveEnabled", tcpKeepAliveEnabled); + keyComponents.put("useIdleConnectionReaperEnabled", useIdleConnectionReaperEnabled); + keyComponents.put("proxyEndpoint", proxyEndpoint); + + return keyComponents.entrySet().stream() + .map(entry -> entry.getKey() + "=" + Objects.toString(entry.getValue(), "null")) + .collect(Collectors.joining(",", "apache[", "]")); + } + public static ApacheHttpClientConfigurations create(Map properties) { ApacheHttpClientConfigurations configurations = new ApacheHttpClientConfigurations(); configurations.initialize(properties); diff --git a/aws/src/main/java/org/apache/iceberg/aws/BaseHttpClientConfigurations.java b/aws/src/main/java/org/apache/iceberg/aws/BaseHttpClientConfigurations.java new file mode 100644 index 000000000000..d4301f04487e --- /dev/null +++ b/aws/src/main/java/org/apache/iceberg/aws/BaseHttpClientConfigurations.java @@ -0,0 +1,75 @@ +/* + * 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.aws; + +import software.amazon.awssdk.awscore.client.builder.AwsSyncClientBuilder; +import software.amazon.awssdk.http.SdkHttpClient; + +/** + * Base class for HTTP client configurations that provides managed HTTP client lifecycle with + * reference counting. + * + *

This class encapsulates the interaction with {@link HttpClientCache} to ensure HTTP clients + * are properly shared and their lifecycle managed via reference counting. Subclasses are + * responsible for providing configuration-specific cache keys and building the appropriate HTTP + * client type (Apache, UrlConnection, etc.). + */ +abstract class BaseHttpClientConfigurations { + + private static final HttpClientCache CACHE = HttpClientCache.instance(); + + /** + * Generate a unique cache key based on the HTTP client configuration. The cache key is used to + * determine whether HTTP clients can be shared across different factory instances. + * + *

Implementations should include all configuration parameters that affect HTTP client behavior + * (timeouts, connection settings, proxy configuration, etc.) to ensure clients are only shared + * when they have identical configurations. + * + * @return a unique string representing this HTTP client configuration + */ + protected abstract String generateHttpClientCacheKey(); + + /** + * Build the actual HTTP client instance based on the configuration. This method is called only + * when a new HTTP client needs to be created (i.e., when no cached client exists for the given + * cache key). + * + * @return a configured {@link SdkHttpClient} instance + */ + protected abstract SdkHttpClient buildHttpClient(); + + /** + * Configure the AWS client builder with a managed HTTP client. + * + *

This method obtains a managed HTTP client from the cache using the configuration-specific + * cache key. If a client with the same configuration already exists in the cache, it will be + * reused with an incremented reference count. Otherwise, a new client will be built and cached. + * + * @param awsClientBuilder the AWS client builder to configure + * @param the type of AWS client builder + */ + public void configureHttpClientBuilder(T awsClientBuilder) { + String cacheKey = generateHttpClientCacheKey(); + + SdkHttpClient managedHttpClient = CACHE.getOrCreateClient(cacheKey, this::buildHttpClient); + + awsClientBuilder.httpClient(managedHttpClient); + } +} diff --git a/aws/src/main/java/org/apache/iceberg/aws/HttpClientCache.java b/aws/src/main/java/org/apache/iceberg/aws/HttpClientCache.java new file mode 100644 index 000000000000..79444a62aee1 --- /dev/null +++ b/aws/src/main/java/org/apache/iceberg/aws/HttpClientCache.java @@ -0,0 +1,203 @@ +/* + * 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.aws; + +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.ConcurrentMap; +import java.util.function.Supplier; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.http.ExecutableHttpRequest; +import software.amazon.awssdk.http.HttpExecuteRequest; +import software.amazon.awssdk.http.SdkHttpClient; + +/** + * A cache that manages the lifecycle of shared HTTP clients for AWS SDK v2 using reference + * counting. Package-private - only accessed via {@link BaseHttpClientConfigurations}. + */ +final class HttpClientCache { + private static final Logger LOG = LoggerFactory.getLogger(HttpClientCache.class); + + private final ConcurrentMap clients = Maps.newConcurrentMap(); + private static volatile HttpClientCache instance; + + static HttpClientCache instance() { + if (instance == null) { + synchronized (HttpClientCache.class) { + if (instance == null) { + instance = new HttpClientCache(); + } + } + } + return instance; + } + + /** + * Get or create a managed HTTP client for the given configuration. Each call increments the + * reference count for the client and returns a ref-counted wrapper. + * + * @param clientKey unique key identifying the client configuration + * @param clientFactory factory to create the HTTP client if not cached + * @return a ref-counted HTTP client wrapper + */ + SdkHttpClient getOrCreateClient(String clientKey, Supplier clientFactory) { + ManagedHttpClient managedClient = + clients.computeIfAbsent( + clientKey, + key -> { + LOG.debug("Creating new managed HTTP client for key: {}", key); + SdkHttpClient httpClient = clientFactory.get(); + return new ManagedHttpClient(httpClient, key); + }); + // Return the cached ref-counted wrapper + return managedClient.acquire(); + } + + /** + * Release a reference to the HTTP client. When the reference count reaches zero, the client is + * closed and removed from the cache. + * + * @param clientKey the key identifying the client to release + */ + void releaseClient(String clientKey) { + ManagedHttpClient managedClient = clients.get(clientKey); + if (null != managedClient && managedClient.release()) { + clients.remove(clientKey, managedClient); + } + } + + @VisibleForTesting + Map clients() { + return Collections.unmodifiableMap(clients); + } + + @VisibleForTesting + void clear() { + clients.values().forEach(ManagedHttpClient::close); + clients.clear(); + } + + /** + * Managed HTTP client wrapper that provides reference counting for lifecycle management. The HTTP + * client is closed when the reference count reaches zero. + */ + static class ManagedHttpClient implements SdkHttpClient { + private final SdkHttpClient httpClient; + private final String clientKey; + private volatile int refCount = 0; + private boolean closed = false; + + ManagedHttpClient(SdkHttpClient httpClient, String clientKey) { + this.httpClient = httpClient; + this.clientKey = clientKey; + LOG.debug("Created managed HTTP client: key={}", clientKey); + } + + /** + * Acquire a reference to the HTTP client, incrementing the reference count. + * + * @return the ref-counted wrapper client + * @throws IllegalStateException if the client has already been closed + */ + synchronized ManagedHttpClient acquire() { + if (closed) { + throw new IllegalStateException("Cannot acquire closed HTTP client: " + clientKey); + } + refCount++; + LOG.debug("Acquired HTTP client: key={}, refCount={}", clientKey, refCount); + return this; + } + + /** + * Release a reference to the HTTP client, decrementing the reference count. If the count + * reaches zero, the client is closed. + * + * @return true if the client was closed, false otherwise + */ + synchronized boolean release() { + if (closed) { + LOG.warn("Attempted to release already closed HTTP client: key={}", clientKey); + return false; + } + + refCount--; + LOG.debug("Released HTTP client: key={}, refCount={}", clientKey, refCount); + if (refCount == 0) { + return closeHttpClient(); + } else if (refCount < 0) { + LOG.warn( + "HTTP client reference count went negative key={}, refCount={}", clientKey, refCount); + refCount = 0; + } + return false; + } + + @VisibleForTesting + SdkHttpClient httpClient() { + return httpClient; + } + + /** + * Close the HTTP client if not already closed. + * + * @return true if the client was closed by this call, false if already closed + */ + private boolean closeHttpClient() { + if (!closed) { + closed = true; + LOG.debug("Closing HTTP client: key={}", clientKey); + try { + httpClient.close(); + } catch (Exception e) { + LOG.error("Failed to close HTTP client: key={}", clientKey, e); + } + return true; + } + return false; + } + + @VisibleForTesting + int refCount() { + return refCount; + } + + @VisibleForTesting + boolean isClosed() { + return closed; + } + + @Override + public ExecutableHttpRequest prepareRequest(HttpExecuteRequest request) { + return httpClient.prepareRequest(request); + } + + @Override + public String clientName() { + return httpClient.clientName(); + } + + @Override + public void close() { + HttpClientCache.instance().releaseClient(clientKey); + } + } +} diff --git a/aws/src/main/java/org/apache/iceberg/aws/UrlConnectionHttpClientConfigurations.java b/aws/src/main/java/org/apache/iceberg/aws/UrlConnectionHttpClientConfigurations.java index ff8dafcf0645..273baa674804 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/UrlConnectionHttpClientConfigurations.java +++ b/aws/src/main/java/org/apache/iceberg/aws/UrlConnectionHttpClientConfigurations.java @@ -21,13 +21,16 @@ import java.net.URI; import java.time.Duration; import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.PropertyUtil; -import software.amazon.awssdk.awscore.client.builder.AwsSyncClientBuilder; +import software.amazon.awssdk.http.SdkHttpClient; import software.amazon.awssdk.http.urlconnection.ProxyConfiguration; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; -class UrlConnectionHttpClientConfigurations { +class UrlConnectionHttpClientConfigurations extends BaseHttpClientConfigurations { private Long httpClientUrlConnectionConnectionTimeoutMs; private Long httpClientUrlConnectionSocketTimeoutMs; @@ -35,11 +38,12 @@ class UrlConnectionHttpClientConfigurations { private UrlConnectionHttpClientConfigurations() {} - public void configureHttpClientBuilder(T awsClientBuilder) { - UrlConnectionHttpClient.Builder urlConnectionHttpClientBuilder = + @Override + protected SdkHttpClient buildHttpClient() { + final UrlConnectionHttpClient.Builder urlConnectionHttpClientBuilder = UrlConnectionHttpClient.builder(); configureUrlConnectionHttpClientBuilder(urlConnectionHttpClientBuilder); - awsClientBuilder.httpClientBuilder(urlConnectionHttpClientBuilder); + return urlConnectionHttpClientBuilder.build(); } private void initialize(Map httpClientProperties) { @@ -71,6 +75,24 @@ void configureUrlConnectionHttpClientBuilder( } } + /** + * Generate a cache key based on HTTP client configuration. This ensures clients with identical + * configurations share the same HTTP client instance. + */ + @Override + protected String generateHttpClientCacheKey() { + Map keyComponents = Maps.newTreeMap(); // TreeMap for consistent ordering + + keyComponents.put("type", "urlconnection"); + keyComponents.put("connectionTimeoutMs", httpClientUrlConnectionConnectionTimeoutMs); + keyComponents.put("socketTimeoutMs", httpClientUrlConnectionSocketTimeoutMs); + keyComponents.put("proxyEndpoint", proxyEndpoint); + + return keyComponents.entrySet().stream() + .map(entry -> entry.getKey() + "=" + Objects.toString(entry.getValue(), "null")) + .collect(Collectors.joining(",", "urlconnection[", "]")); + } + public static UrlConnectionHttpClientConfigurations create( Map httpClientProperties) { UrlConnectionHttpClientConfigurations configurations = diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientCache.java b/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientCache.java new file mode 100644 index 000000000000..9febf37cbd39 --- /dev/null +++ b/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientCache.java @@ -0,0 +1,276 @@ +/* + * 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.aws; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.util.Map; +import java.util.function.Supplier; +import org.apache.iceberg.aws.HttpClientCache.ManagedHttpClient; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import software.amazon.awssdk.http.SdkHttpClient; + +public class TestHttpClientCache { + + @Mock private SdkHttpClient httpClient1; + @Mock private SdkHttpClient httpClient2; + @Mock private Supplier httpClientFactory1; + @Mock private Supplier httpClientFactory2; + + private HttpClientCache cache; + + @BeforeEach + public void before() { + MockitoAnnotations.openMocks(this); + cache = HttpClientCache.instance(); + // Clean up any existing clients from previous tests + cache.clear(); + + when(httpClientFactory1.get()).thenReturn(httpClient1); + when(httpClientFactory2.get()).thenReturn(httpClient2); + } + + @Test + public void singletonPattern() { + HttpClientCache instance1 = HttpClientCache.instance(); + HttpClientCache instance2 = HttpClientCache.instance(); + + assertThat(instance1).isSameAs(instance2); + } + + @Test + public void clientCaching() { + final String cacheKey = "test-key"; + + // First call should create client and increment ref count + SdkHttpClient client1 = cache.getOrCreateClient(cacheKey, httpClientFactory1); + verify(httpClientFactory1, times(1)).get(); + + // Second call with same key should return cached client and increment ref count again + SdkHttpClient client2 = cache.getOrCreateClient(cacheKey, httpClientFactory1); + verify(httpClientFactory1, times(1)).get(); // Factory should not be called again + + assertThat(client1).isSameAs(client2); + + // Verify reference count is 2 + ManagedHttpClient managedClient = cache.clients().get(cacheKey); + assertThat(managedClient.refCount()).isEqualTo(2); + } + + @Test + public void differentKeysCreateDifferentClients() { + SdkHttpClient client1 = cache.getOrCreateClient("test-key-1", httpClientFactory1); + SdkHttpClient client2 = cache.getOrCreateClient("test-key-2", httpClientFactory2); + + verify(httpClientFactory1, times(1)).get(); + verify(httpClientFactory2, times(1)).get(); + + assertThat(client1).isNotSameAs(client2); + } + + @Test + public void referenceCountingAndCleanup() throws Exception { + SdkHttpClient mockClient = mock(SdkHttpClient.class); + final String cacheKey = "test-key"; + + ManagedHttpClient managedClient = new ManagedHttpClient(mockClient, cacheKey); + + // Acquire twice + ManagedHttpClient client1 = managedClient.acquire(); + ManagedHttpClient client2 = managedClient.acquire(); + + assertThat(client1).isSameAs(client2); + assertThat(managedClient.refCount()).isEqualTo(2); + + // First release should not close + managedClient.release(); + assertThat(managedClient.refCount()).isEqualTo(1); + assertThat(managedClient.isClosed()).isFalse(); + verify(mockClient, times(0)).close(); + + // Second release should close + managedClient.release(); + assertThat(managedClient.refCount()).isEqualTo(0); + assertThat(managedClient.isClosed()).isTrue(); + verify(mockClient, times(1)).close(); + } + + @Test + public void acquireAfterCloseThrows() { + SdkHttpClient mockClient = mock(SdkHttpClient.class); + final String cacheKey = "test-key"; + + ManagedHttpClient managedClient = new ManagedHttpClient(mockClient, cacheKey); + + // Acquire and release to close + managedClient.acquire(); + managedClient.release(); + + assertThat(managedClient.isClosed()).isTrue(); + + // Trying to acquire a closed client should throw + assertThatThrownBy(managedClient::acquire) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Cannot acquire closed HTTP client"); + } + + @Test + public void releaseRemovesFromRegistry() { + final String cacheKey = "test-key"; + + // Create client (refCount = 1) + SdkHttpClient client1 = cache.getOrCreateClient(cacheKey, httpClientFactory1); + assertThat(client1).isNotNull(); + + Map clients = cache.clients(); + assertThat(clients).containsKey(cacheKey); + + // Verify ref count is 1 + assertThat(clients.get(cacheKey).refCount()).isEqualTo(1); + + // Release (refCount = 0, should close and remove) + cache.releaseClient(cacheKey); + + // Client should be removed from map after close + assertThat(clients).doesNotContainKey(cacheKey); + verify(httpClient1, times(1)).close(); + } + + @Test + public void concurrentAccess() throws InterruptedException { + final String cacheKey = "concurrent-test-key"; + int threadCount = 10; + Thread[] threads = new Thread[threadCount]; + SdkHttpClient[] results = new SdkHttpClient[threadCount]; + + // Create multiple threads that access the same cache key + for (int i = 0; i < threadCount; i++) { + final int index = i; + threads[i] = + new Thread(() -> results[index] = cache.getOrCreateClient(cacheKey, httpClientFactory1)); + } + + // Start all threads + for (Thread thread : threads) { + thread.start(); + } + + // Wait for all threads to complete + for (Thread thread : threads) { + thread.join(); + } + + // Verify factory was called only once (proper caching under concurrency) + verify(httpClientFactory1, times(1)).get(); + + // Verify all threads got the same client instance + SdkHttpClient expectedClient = results[0]; + for (int i = 1; i < threadCount; i++) { + assertThat(results[i]).isSameAs(expectedClient); + } + + // Verify reference count equals number of threads + ManagedHttpClient managedClient = cache.clients().get(cacheKey); + assertThat(managedClient.refCount()).isEqualTo(threadCount); + } + + @Test + public void registryClear() { + Map clients = cache.clients(); + + // Create some clients + cache.getOrCreateClient("key1", httpClientFactory1); + cache.getOrCreateClient("key2", httpClientFactory2); + + // Verify clients were stored + assertThat(clients).hasSize(2); + + // Shutdown should clean up the map + cache.clear(); + + // Map should be empty after shutdown + assertThat(clients).isEmpty(); + + // Both clients should be closed + verify(httpClient1, times(1)).close(); + verify(httpClient2, times(1)).close(); + } + + @Test + public void doubleReleaseDoesNotCauseNegativeRefCount() throws Exception { + SdkHttpClient mockClient = mock(SdkHttpClient.class); + final String cacheKey = "test-key"; + + ManagedHttpClient managedClient = new ManagedHttpClient(mockClient, cacheKey); + + // Acquire once + ManagedHttpClient client = managedClient.acquire(); + assertThat(managedClient.refCount()).isEqualTo(1); + + // First release should close the client (refCount goes to 0) + boolean closed = managedClient.release(); + assertThat(closed).isTrue(); + assertThat(managedClient.refCount()).isEqualTo(0); + assertThat(managedClient.isClosed()).isTrue(); + verify(mockClient, times(1)).close(); + + // Second release on already closed client should be a no-op + // The closed flag prevents decrement, so refCount stays at 0 + boolean closedAgain = managedClient.release(); + assertThat(closedAgain).isFalse(); + assertThat(managedClient.refCount()).isEqualTo(0); // Should still be 0, not negative + assertThat(managedClient.isClosed()).isTrue(); + verify(mockClient, times(1)).close(); // Close should not be called again + } + + @Test + public void multipleReleasesAfterClose() throws Exception { + SdkHttpClient mockClient = mock(SdkHttpClient.class); + final String cacheKey = "test-key"; + + ManagedHttpClient managedClient = new ManagedHttpClient(mockClient, cacheKey); + + // Acquire once + managedClient.acquire(); + assertThat(managedClient.refCount()).isEqualTo(1); + + // Release to close + managedClient.release(); + assertThat(managedClient.isClosed()).isTrue(); + assertThat(managedClient.refCount()).isEqualTo(0); + + // Try releasing multiple more times (simulating a bug in caller code) + for (int i = 0; i < 5; i++) { + boolean result = managedClient.release(); + assertThat(result).isFalse(); // Should return false, not try to close again + assertThat(managedClient.refCount()).isEqualTo(0); // RefCount should never go negative + } + + // Close should only have been called once + verify(mockClient, times(1)).close(); + } +} diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientProperties.java b/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientProperties.java index b0602a074992..378e5e6ca94a 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientProperties.java @@ -20,8 +20,11 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; import java.util.Map; +import org.apache.iceberg.aws.HttpClientCache.ManagedHttpClient; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; @@ -40,16 +43,21 @@ public void testUrlHttpClientConfiguration() { properties.put(HttpClientProperties.CLIENT_TYPE, "urlconnection"); HttpClientProperties httpProperties = new HttpClientProperties(properties); S3ClientBuilder mockS3ClientBuilder = Mockito.mock(S3ClientBuilder.class); - ArgumentCaptor httpClientBuilderCaptor = - ArgumentCaptor.forClass(SdkHttpClient.Builder.class); + ArgumentCaptor httpClientCaptor = ArgumentCaptor.forClass(SdkHttpClient.class); httpProperties.applyHttpClientConfigurations(mockS3ClientBuilder); - Mockito.verify(mockS3ClientBuilder).httpClientBuilder(httpClientBuilderCaptor.capture()); - SdkHttpClient.Builder capturedHttpClientBuilder = httpClientBuilderCaptor.getValue(); + Mockito.verify(mockS3ClientBuilder).httpClient(httpClientCaptor.capture()); + SdkHttpClient capturedHttpClient = httpClientCaptor.getValue(); - assertThat(capturedHttpClientBuilder) - .as("Should use url connection http client") - .isInstanceOf(UrlConnectionHttpClient.Builder.class); + assertThat(capturedHttpClient) + .as("Should use managed SDK http client") + .isInstanceOf(ManagedHttpClient.class); + + // Verify the underlying delegate is UrlConnectionHttpClient + ManagedHttpClient managedClient = (ManagedHttpClient) capturedHttpClient; + assertThat(managedClient.httpClient()) + .as("Underlying client should be UrlConnectionHttpClient") + .isInstanceOf(UrlConnectionHttpClient.class); } @Test @@ -58,15 +66,21 @@ public void testApacheHttpClientConfiguration() { properties.put(HttpClientProperties.CLIENT_TYPE, "apache"); HttpClientProperties httpProperties = new HttpClientProperties(properties); S3ClientBuilder mockS3ClientBuilder = Mockito.mock(S3ClientBuilder.class); - ArgumentCaptor httpClientBuilderCaptor = - ArgumentCaptor.forClass(SdkHttpClient.Builder.class); + ArgumentCaptor httpClientCaptor = ArgumentCaptor.forClass(SdkHttpClient.class); httpProperties.applyHttpClientConfigurations(mockS3ClientBuilder); - Mockito.verify(mockS3ClientBuilder).httpClientBuilder(httpClientBuilderCaptor.capture()); - SdkHttpClient.Builder capturedHttpClientBuilder = httpClientBuilderCaptor.getValue(); - assertThat(capturedHttpClientBuilder) - .as("Should use apache http client") - .isInstanceOf(ApacheHttpClient.Builder.class); + Mockito.verify(mockS3ClientBuilder).httpClient(httpClientCaptor.capture()); + SdkHttpClient capturedHttpClient = httpClientCaptor.getValue(); + + assertThat(capturedHttpClient) + .as("Should use managed SDK http client") + .isInstanceOf(ManagedHttpClient.class); + + // Verify the underlying delegate is ApacheHttpClient + ManagedHttpClient managedClient = (ManagedHttpClient) capturedHttpClient; + assertThat(managedClient.httpClient()) + .as("Underlying client should be ApacheHttpClient") + .isInstanceOf(ApacheHttpClient.class); } @Test @@ -80,4 +94,29 @@ public void testInvalidHttpClientType() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Unrecognized HTTP client type test"); } + + @Test + public void testApacheHttpClientConfiguredAsSharedResource() { + Map properties = Maps.newHashMap(); + ApacheHttpClientConfigurations apacheConfig = ApacheHttpClientConfigurations.create(properties); + S3ClientBuilder mockS3ClientBuilder = Mockito.mock(S3ClientBuilder.class); + + apacheConfig.configureHttpClientBuilder(mockS3ClientBuilder); + + // Verify that httpClient() is called with a managed client (as a shared resource) + verify(mockS3ClientBuilder).httpClient(any(ManagedHttpClient.class)); + } + + @Test + public void testUrlConnectionHttpClientConfiguredAsSharedResource() { + Map properties = Maps.newHashMap(); + UrlConnectionHttpClientConfigurations urlConfig = + UrlConnectionHttpClientConfigurations.create(properties); + S3ClientBuilder mockS3ClientBuilder = Mockito.mock(S3ClientBuilder.class); + + urlConfig.configureHttpClientBuilder(mockS3ClientBuilder); + + // Verify that httpClient() is called with a managed client (as a shared resource) + verify(mockS3ClientBuilder).httpClient(any(ManagedHttpClient.class)); + } } From 885bcbb9fc98d8f4da3350261deb3bec11e1a181 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 5 Dec 2025 21:48:45 +0100 Subject: [PATCH 050/201] site: Update Slack link (#14772) --- .github/ISSUE_TEMPLATE/iceberg_question.yml | 2 +- site/docs/community.md | 2 +- site/mkdocs.yml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/iceberg_question.yml b/.github/ISSUE_TEMPLATE/iceberg_question.yml index a170e9acceb5..b5966dccc7ac 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_question.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_question.yml @@ -25,7 +25,7 @@ body: - type: markdown attributes: value: | - Feel free to ask your question on [Slack](https://join.slack.com/t/apache-iceberg/shared_invite/zt-2561tq9qr-UtISlHgsdY3Virs3Z2_btQ) as well. + Feel free to ask your question on [Slack](https://join.slack.com/t/apache-iceberg/shared_invite/zt-3kclosz6r-3heAW3d~_PHefmN2A_~cAg) as well. Do **NOT** share any sensitive information like passwords, security tokens, private URLs etc. - type: textarea diff --git a/site/docs/community.md b/site/docs/community.md index 06702c18f906..d8611fc6b231 100644 --- a/site/docs/community.md +++ b/site/docs/community.md @@ -47,7 +47,7 @@ Iceberg has four mailing lists: ### Slack -We use the [Apache Iceberg workspace](https://apache-iceberg.slack.com/) on Slack. To be invited, follow [this invite link](https://join.slack.com/t/apache-iceberg/shared_invite/zt-287g3akar-K9Oe_En5j1UL7Y_Ikpai3A). +We use the [Apache Iceberg workspace](https://apache-iceberg.slack.com/) on Slack. To be invited, follow [this invite link](https://join.slack.com/t/apache-iceberg/shared_invite/zt-3kclosz6r-3heAW3d~_PHefmN2A_~cAg). Please note that this link may occasionally break when Slack does an upgrade. If you encounter problems using it, please let us know by sending an email to . diff --git a/site/mkdocs.yml b/site/mkdocs.yml index ebf6006b6723..6ea9ceb3636b 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -100,7 +100,7 @@ extra: link: 'https://www.youtube.com/@ApacheIceberg' title: youtube - icon: fontawesome/brands/slack - link: 'https://join.slack.com/t/apache-iceberg/shared_invite/zt-287g3akar-K9Oe_En5j1UL7Y_Ikpai3A' + link: 'https://join.slack.com/t/apache-iceberg/shared_invite/zt-3kclosz6r-3heAW3d~_PHefmN2A_~cAg' title: slack exclude_docs: | From 1c024d7c4e66a0654118d30d04a0d9c0c5ab4de3 Mon Sep 17 00:00:00 2001 From: Kurtis Wright Date: Fri, 5 Dec 2025 13:04:03 -0800 Subject: [PATCH 051/201] Update configuration.md (#14771) Fixes spec.md link in format-version table. --- docs/docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/docs/configuration.md b/docs/docs/configuration.md index 3f730a4f4c13..71498f4f57d5 100644 --- a/docs/docs/configuration.md +++ b/docs/docs/configuration.md @@ -125,7 +125,7 @@ The value of these properties are not persisted as a part of the table metadata. | Property | Default | Description | | -------------- | -------- |--------------------------------------------------------------------------------------------------------------------------------------| -| format-version | 2 | Table's format version as defined in the [Spec](../../spec.md#format-versioning). Defaults to 2 since version 1.4.0. | +| format-version | 2 | Table's format version as defined in the [Spec](../../format/spec.md#format-versioning). Defaults to 2 since version 1.4.0. | ### Compatibility flags From fc434997fbc63a3f1f47481c0878073b1ccf6359 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Sat, 6 Dec 2025 21:17:17 +0800 Subject: [PATCH 052/201] Revert "Update configuration.md (#14771)" (#14780) This reverts commit 1c024d7c4e66a0654118d30d04a0d9c0c5ab4de3. --- docs/docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/docs/configuration.md b/docs/docs/configuration.md index 71498f4f57d5..3f730a4f4c13 100644 --- a/docs/docs/configuration.md +++ b/docs/docs/configuration.md @@ -125,7 +125,7 @@ The value of these properties are not persisted as a part of the table metadata. | Property | Default | Description | | -------------- | -------- |--------------------------------------------------------------------------------------------------------------------------------------| -| format-version | 2 | Table's format version as defined in the [Spec](../../format/spec.md#format-versioning). Defaults to 2 since version 1.4.0. | +| format-version | 2 | Table's format version as defined in the [Spec](../../spec.md#format-versioning). Defaults to 2 since version 1.4.0. | ### Compatibility flags From 3b1ec481a564ae8bce608a1044a6156c4f49ede1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 6 Dec 2025 23:32:08 -0800 Subject: [PATCH 053/201] Build: Bump actions/stale from 10.1.0 to 10.1.1 (#14784) Bumps [actions/stale](https://github.com/actions/stale) from 10.1.0 to 10.1.1. - [Release notes](https://github.com/actions/stale/releases) - [Changelog](https://github.com/actions/stale/blob/main/CHANGELOG.md) - [Commits](https://github.com/actions/stale/compare/v10.1.0...v10.1.1) --- updated-dependencies: - dependency-name: actions/stale dependency-version: 10.1.1 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/stale.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index a7118a4ffdd9..9a7134d5465c 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -32,7 +32,7 @@ jobs: if: github.repository_owner == 'apache' runs-on: ubuntu-24.04 steps: - - uses: actions/stale@v10.1.0 + - uses: actions/stale@v10.1.1 with: # stale issues stale-issue-label: 'stale' From faabc46f955924667dbcb0cc43b8a0da3ec56169 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 6 Dec 2025 23:32:41 -0800 Subject: [PATCH 054/201] Build: Bump nessie from 0.105.7 to 0.106.0 (#14785) Bumps `nessie` from 0.105.7 to 0.106.0. Updates `org.projectnessie.nessie:nessie-client` from 0.105.7 to 0.106.0 - [Release notes](https://github.com/projectnessie/nessie/releases) - [Changelog](https://github.com/projectnessie/nessie/blob/main/CHANGELOG.md) - [Commits](https://github.com/projectnessie/nessie/compare/nessie-0.105.7...nessie-0.106.0) Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.105.7 to 0.106.0 - [Release notes](https://github.com/projectnessie/nessie/releases) - [Changelog](https://github.com/projectnessie/nessie/blob/main/CHANGELOG.md) - [Commits](https://github.com/projectnessie/nessie/compare/nessie-0.105.7...nessie-0.106.0) Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.105.7 to 0.106.0 - [Release notes](https://github.com/projectnessie/nessie/releases) - [Changelog](https://github.com/projectnessie/nessie/blob/main/CHANGELOG.md) - [Commits](https://github.com/projectnessie/nessie/compare/nessie-0.105.7...nessie-0.106.0) Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.105.7 to 0.106.0 - [Release notes](https://github.com/projectnessie/nessie/releases) - [Changelog](https://github.com/projectnessie/nessie/blob/main/CHANGELOG.md) - [Commits](https://github.com/projectnessie/nessie/compare/nessie-0.105.7...nessie-0.106.0) --- updated-dependencies: - dependency-name: org.projectnessie.nessie:nessie-client dependency-version: 0.106.0 dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.projectnessie.nessie:nessie-jaxrs-testextension dependency-version: 0.106.0 dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests dependency-version: 0.106.0 dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-testextension dependency-version: 0.106.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 3ba051c8b897..aad570e406ed 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -74,7 +74,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.105.7" +nessie = "0.106.0" netty-buffer = "4.2.7.Final" object-client-bundle = "3.3.2" orc = "1.9.7" From 786d1645c78f13fbd627e1d8a9785c6960d44798 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 6 Dec 2025 23:33:39 -0800 Subject: [PATCH 055/201] Build: Bump org.xerial:sqlite-jdbc from 3.51.0.0 to 3.51.1.0 (#14786) Bumps [org.xerial:sqlite-jdbc](https://github.com/xerial/sqlite-jdbc) from 3.51.0.0 to 3.51.1.0. - [Release notes](https://github.com/xerial/sqlite-jdbc/releases) - [Changelog](https://github.com/xerial/sqlite-jdbc/blob/master/CHANGELOG) - [Commits](https://github.com/xerial/sqlite-jdbc/compare/3.51.0.0...3.51.1.0) --- updated-dependencies: - dependency-name: org.xerial:sqlite-jdbc dependency-version: 3.51.1.0 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index aad570e406ed..b1b5394ac329 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -86,7 +86,7 @@ snowflake-jdbc = "3.27.1" spark34 = "3.4.4" spark35 = "3.5.7" spark40 = "4.0.1" -sqlite-jdbc = "3.51.0.0" +sqlite-jdbc = "3.51.1.0" testcontainers = "2.0.2" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 6735edd8494257af8cbfe92f38072136a2f97715 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 6 Dec 2025 23:43:51 -0800 Subject: [PATCH 056/201] Build: Bump software.amazon.awssdk:bom from 2.39.5 to 2.40.3 (#14788) Bumps software.amazon.awssdk:bom from 2.39.5 to 2.40.3. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-version: 2.40.3 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index b1b5394ac329..a99206dba3f2 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -31,7 +31,7 @@ arrow = "15.0.2" avro = "1.12.1" assertj-core = "3.27.6" awaitility = "4.3.0" -awssdk-bom = "2.39.5" +awssdk-bom = "2.40.3" azuresdk-bom = "1.3.3" awssdk-s3accessgrants = "2.3.0" bson-ver = "4.11.5" From 19b4bd024486d9d516d0e547e273419c1bc7074e Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Mon, 8 Dec 2025 14:01:06 +0900 Subject: [PATCH 057/201] Core: Disallow encryption table properties in v1 and v2 (#14668) --- .../org/apache/iceberg/TableMetadata.java | 3 +++ .../iceberg/encryption/EncryptionUtil.java | 22 +++++++++++++++++++ .../org/apache/iceberg/TestTableMetadata.java | 19 ++++++++++++++++ .../iceberg/spark/sql/TestCTASEncryption.java | 2 +- .../spark/sql/TestTableEncryption.java | 2 +- 5 files changed, 46 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 3c2a3eb9b7a7..7dac5d401a80 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -31,6 +31,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.iceberg.encryption.EncryptedKey; +import org.apache.iceberg.encryption.EncryptionUtil; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Objects; @@ -140,6 +141,8 @@ static TableMetadata newTableMetadata( PropertyUtil.validateCommitProperties(properties); + EncryptionUtil.checkCompatibility(properties, formatVersion); + return new Builder() .setInitialFormatVersion(formatVersion) .setCurrentSchema(freshSchema, lastColumnId.get()) diff --git a/core/src/main/java/org/apache/iceberg/encryption/EncryptionUtil.java b/core/src/main/java/org/apache/iceberg/encryption/EncryptionUtil.java index fe228fae93fe..1f0461bc4d97 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/EncryptionUtil.java +++ b/core/src/main/java/org/apache/iceberg/encryption/EncryptionUtil.java @@ -22,16 +22,24 @@ import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.ManifestListFile; import org.apache.iceberg.TableProperties; import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.PropertyUtil; public class EncryptionUtil { + private static final Set ENCRYPTION_TABLE_PROPERTIES = + ImmutableSet.builder() + .add(TableProperties.ENCRYPTION_TABLE_KEY) + .add(TableProperties.ENCRYPTION_DEK_LENGTH) + .build(); private EncryptionUtil() {} @@ -182,4 +190,18 @@ static ByteBuffer encryptManifestListKeyMetadata( return ByteBuffer.wrap(encryptedKeyMetadata); } + + public static void checkCompatibility(Map tableProperties, int formatVersion) { + if (formatVersion >= 3) { + return; + } + + Set encryptionProperties = + Sets.intersection(ENCRYPTION_TABLE_PROPERTIES, tableProperties.keySet()); + Preconditions.checkArgument( + encryptionProperties.isEmpty(), + "Invalid properties for v%s: %s", + formatVersion, + encryptionProperties); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 345f506fa978..ed7b630f6043 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -72,6 +72,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; public class TestTableMetadata { private static final String TEST_LOCATION = "s3://bucket/test/location"; @@ -1094,6 +1095,24 @@ public void testVersionValidation() { .isNotNull(); } + @ParameterizedTest + @ValueSource(ints = {1, 2}) + void testEncryptionVersionValidation(int formatVersion) { + assertThatThrownBy( + () -> + TableMetadata.newTableMetadata( + TEST_SCHEMA, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + TEST_LOCATION, + ImmutableMap.of("encryption.key-id", "test", "encryption.data-key-length", "5"), + formatVersion)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Invalid properties for v%s: [encryption.key-id, encryption.data-key-length]", + formatVersion); + } + @Test public void testParserVersionValidation() throws Exception { String supportedVersion1 = readTableMetadataInputFile("TableMetadataV1Valid.json"); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java index 6094ab0ccca5..3dee6e1e1d54 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java @@ -71,7 +71,7 @@ public void createTables() { sql( "CREATE TABLE %s USING iceberg " + "TBLPROPERTIES ( " - + "'encryption.key-id'='%s')" + + "'encryption.key-id'='%s', 'format-version'='3')" + " AS SELECT * from %s", tableName, UnitestKMS.MASTER_KEY_NAME1, tableName + "1"); } diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java index 8f0552a37877..e35ebf71b5c5 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java @@ -78,7 +78,7 @@ public void createTables() { sql( "CREATE TABLE %s (id bigint, data string, float float) USING iceberg " + "TBLPROPERTIES ( " - + "'encryption.key-id'='%s')", + + "'encryption.key-id'='%s', 'format-version'='3')", tableName, UnitestKMS.MASTER_KEY_NAME1); sql("INSERT INTO %s VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', float('NaN'))", tableName); From 0c194502b802630dde71120e5e48a074a0552ee8 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 8 Dec 2025 10:15:03 +0100 Subject: [PATCH 058/201] OpenAPI: Use `PrimitiveTypeValue` rather than `object` (#14184) --- open-api/rest-catalog-open-api.py | 7 +++---- open-api/rest-catalog-open-api.yaml | 7 ++----- 2 files changed, 5 insertions(+), 9 deletions(-) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 25a1a1a5c407..6627e4516bc8 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -18,7 +18,7 @@ from __future__ import annotations from datetime import date, timedelta -from typing import Any, Dict, List, Literal, Optional, Union +from typing import Dict, List, Literal, Optional, Union from uuid import UUID from pydantic import BaseModel, Extra, Field @@ -1085,19 +1085,18 @@ class SetStatisticsUpdate(BaseUpdate): class UnaryExpression(BaseModel): type: ExpressionType term: Term - value: Dict[str, Any] class LiteralExpression(BaseModel): type: ExpressionType term: Term - value: Dict[str, Any] + value: PrimitiveTypeValue class SetExpression(BaseModel): type: ExpressionType term: Term - values: List[Dict[str, Any]] + values: List[PrimitiveTypeValue] class StructField(BaseModel): diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index c0b1c0f0fcea..c977f9b9afb8 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -2311,15 +2311,12 @@ components: required: - type - term - - value properties: type: $ref: '#/components/schemas/ExpressionType' enum: ["is-null", "not-null", "is-nan", "not-nan"] term: $ref: '#/components/schemas/Term' - value: - type: object LiteralExpression: type: object @@ -2334,7 +2331,7 @@ components: term: $ref: '#/components/schemas/Term' value: - type: object + $ref: '#/components/schemas/PrimitiveTypeValue' SetExpression: type: object @@ -2351,7 +2348,7 @@ components: values: type: array items: - type: object + $ref: '#/components/schemas/PrimitiveTypeValue' Term: oneOf: From a739cb3db89ad94a3525443cef9357ccb1897d3f Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Mon, 8 Dec 2025 12:23:24 +0100 Subject: [PATCH 059/201] Flink: Dynamic Sink: Add support for dropping columns (#14728) --- docs/docs/flink-writes.md | 17 ++- .../sink/dynamic/CompareSchemasVisitor.java | 18 ++- .../sink/dynamic/DynamicIcebergSink.java | 19 +++ .../sink/dynamic/DynamicRecordProcessor.java | 7 +- .../dynamic/DynamicTableUpdateOperator.java | 6 +- .../sink/dynamic/EvolveSchemaVisitor.java | 47 +++++-- .../sink/dynamic/TableMetadataCache.java | 10 +- .../flink/sink/dynamic/TableUpdater.java | 17 ++- .../dynamic/TestCompareSchemasVisitor.java | 62 +++++++++ .../sink/dynamic/TestDynamicIcebergSink.java | 49 +++++++ .../TestDynamicTableUpdateOperator.java | 82 ++++++++++++ .../sink/dynamic/TestEvolveSchemaVisitor.java | 124 ++++++++++++++---- .../sink/dynamic/TestTableMetadataCache.java | 25 ++-- .../flink/sink/dynamic/TestTableUpdater.java | 45 +++++-- 14 files changed, 445 insertions(+), 83 deletions(-) diff --git a/docs/docs/flink-writes.md b/docs/docs/flink-writes.md index 5d3fde5e5456..3fef3a1bf3bf 100644 --- a/docs/docs/flink-writes.md +++ b/docs/docs/flink-writes.md @@ -497,18 +497,26 @@ The dynamic sink tries to match the schema provided in `DynamicRecord` with the The dynamic sink maintains an LRU cache for both table metadata and incoming schemas, with eviction based on size and time constraints. When a DynamicRecord contains a schema that is incompatible with the current table schema, a schema update is triggered. This update can occur either immediately or via a centralized executor, depending on the `immediateTableUpdate` configuration. While centralized updates reduce load on the Catalog, they may introduce backpressure on the sink. -Supported schema updates: +#### Supported schema updates - Adding new columns - Widening existing column types (e.g., Integer → Long, Float → Double) - Making required columns optional +- Dropping columns (disabled by default) -Unsupported schema updates: +Dropping columns is disabled by default to prevent issues with late or out-of-order data, as removed fields cannot be easily restored without data loss. + +You can opt-in to allow dropping columns (see the configuration options below). Once a column has been dropped, it is +technically still possible to write data to that column because Iceberg maintains all past table schemas. However, +regular queries won't be able to reference the column. If the field was to re-appear as part of a new schema, an +entirely new column would be added, which apart from the name, has nothing in common with the old column, i.e. queries +for the new column will never return data of the old column. + +##### Unsupported schema updates -- Dropping columns - Renaming columns -Dropping columns is avoided to prevent issues with late or out-of-order data, as removed fields cannot be easily restored without data loss. Renaming is unsupported because schema comparison is name-based, and renames would require additional metadata or hints to resolve. +Renaming is unsupported because schema comparison is name-based, and renames would require additional metadata or hints to resolve. ### Caching @@ -537,6 +545,7 @@ The Dynamic Iceberg Flink Sink is configured using the Builder pattern. Here are | `set(String property, String value)` | Set any Iceberg write property (e.g., `"write.format"`, `"write.upsert.enabled"`).Check out all the options here: [write-options](flink-configuration.md#write-options) | | `setAll(Map properties)` | Set multiple properties at once | | `tableCreator(TableCreator creator)` | When DynamicIcebergSink creates new Iceberg tables, allows overriding how tables are created - setting custom table properties and location based on the table name. | +| `dropUnusedColumns(boolean enabled)` | When enabled, drops all columns from the current table schema which are not contained in the input schema (see the caveats above on dropping columns). | ### Notes diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java index 41ffa609540b..60561b0f56bf 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java @@ -43,20 +43,23 @@ public class CompareSchemasVisitor extends SchemaWithPartnerVisitor { private final Schema tableSchema; + private final boolean dropUnusedColumns; - private CompareSchemasVisitor(Schema tableSchema) { + private CompareSchemasVisitor(Schema tableSchema, boolean dropUnusedColumns) { this.tableSchema = tableSchema; + this.dropUnusedColumns = dropUnusedColumns; } public static Result visit(Schema dataSchema, Schema tableSchema) { - return visit(dataSchema, tableSchema, true); + return visit(dataSchema, tableSchema, true, false); } - public static Result visit(Schema dataSchema, Schema tableSchema, boolean caseSensitive) { + public static Result visit( + Schema dataSchema, Schema tableSchema, boolean caseSensitive, boolean dropUnusedColumns) { return visit( dataSchema, -1, - new CompareSchemasVisitor(tableSchema), + new CompareSchemasVisitor(tableSchema, dropUnusedColumns), new PartnerIdByNameAccessors(tableSchema, caseSensitive)); } @@ -70,6 +73,7 @@ public Result schema(Schema dataSchema, Integer tableSchemaId, Result downstream } @Override + @SuppressWarnings("CyclomaticComplexity") public Result struct(Types.StructType struct, Integer tableSchemaId, List fields) { if (tableSchemaId == null) { return Result.SCHEMA_UPDATE_NEEDED; @@ -88,10 +92,10 @@ public Result struct(Types.StructType struct, Integer tableSchemaId, List { private ReadableConfig readableConfig = new Configuration(); private TableCreator tableCreator = TableCreator.DEFAULT; private boolean immediateUpdate = false; + private boolean dropUnusedColumns = false; private int cacheMaximumSize = 100; private long cacheRefreshMs = 1_000; private int inputSchemasPerTableCacheMaximumSize = 10; @@ -314,6 +315,22 @@ public Builder immediateTableUpdate(boolean newImmediateUpdate) { return this; } + /** + * Dropping columns is disabled by default to prevent issues with late or out-of-order data, as + * removed fields cannot be easily restored without data loss. + * + *

You can opt-in to allow dropping columns. Once a column has been dropped, it is + * technically still possible to write data to that column because Iceberg maintains all past + * table schemas. However, regular queries won't be able to reference the column. If the field + * was to re-appear as part of a new schema, an entirely new column would be added, which apart + * from the name, has nothing in common with the old column, i.e. queries for the new column + * will never return data of the old column. + */ + public Builder dropUnusedColumns(boolean newDropUnusedColumns) { + this.dropUnusedColumns = newDropUnusedColumns; + return this; + } + /** Maximum size of the caches used in Dynamic Sink for table data and serializers. */ public Builder cacheMaxSize(int maxSize) { this.cacheMaximumSize = maxSize; @@ -382,6 +399,7 @@ public DataStreamSink append() { generator, catalogLoader, immediateUpdate, + dropUnusedColumns, cacheMaximumSize, cacheRefreshMs, inputSchemasPerTableCacheMaximumSize, @@ -400,6 +418,7 @@ public DataStreamSink append() { .map( new DynamicTableUpdateOperator( catalogLoader, + dropUnusedColumns, cacheMaximumSize, cacheRefreshMs, inputSchemasPerTableCacheMaximumSize, diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java index bc3a25468d84..427aa6ceafba 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java @@ -40,6 +40,7 @@ class DynamicRecordProcessor extends ProcessFunction generator; private final CatalogLoader catalogLoader; private final boolean immediateUpdate; + private final boolean dropUnusedColumns; private final int cacheMaximumSize; private final long cacheRefreshMs; private final int inputSchemasPerTableCacheMaximumSize; @@ -56,6 +57,7 @@ class DynamicRecordProcessor extends ProcessFunction generator, CatalogLoader catalogLoader, boolean immediateUpdate, + boolean dropUnusedColumns, int cacheMaximumSize, long cacheRefreshMs, int inputSchemasPerTableCacheMaximumSize, @@ -63,6 +65,7 @@ class DynamicRecordProcessor extends ProcessFunction( @@ -106,7 +109,7 @@ public void collect(DynamicRecord data) { TableMetadataCache.ResolvedSchemaInfo foundSchema = exists - ? tableCache.schema(data.tableIdentifier(), data.schema()) + ? tableCache.schema(data.tableIdentifier(), data.schema(), dropUnusedColumns) : TableMetadataCache.NOT_FOUND; PartitionSpec foundSpec = exists ? tableCache.spec(data.tableIdentifier(), data.spec()) : null; diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java index 586239b54bca..8f38d4f8be0c 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java @@ -38,6 +38,7 @@ class DynamicTableUpdateOperator extends RichMapFunction { private final CatalogLoader catalogLoader; + private final boolean dropUnusedColumns; private final int cacheMaximumSize; private final long cacheRefreshMs; private final int inputSchemasPerTableCacheMaximumSize; @@ -47,11 +48,13 @@ class DynamicTableUpdateOperator DynamicTableUpdateOperator( CatalogLoader catalogLoader, + boolean dropUnusedColumns, int cacheMaximumSize, long cacheRefreshMs, int inputSchemasPerTableCacheMaximumSize, TableCreator tableCreator) { this.catalogLoader = catalogLoader; + this.dropUnusedColumns = dropUnusedColumns; this.cacheMaximumSize = cacheMaximumSize; this.cacheRefreshMs = cacheRefreshMs; this.inputSchemasPerTableCacheMaximumSize = inputSchemasPerTableCacheMaximumSize; @@ -66,7 +69,8 @@ public void open(OpenContext openContext) throws Exception { new TableUpdater( new TableMetadataCache( catalog, cacheMaximumSize, cacheRefreshMs, inputSchemasPerTableCacheMaximumSize), - catalog); + catalog, + dropUnusedColumns); } @Override diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java index ee0549997178..e106cf5754b3 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java @@ -21,10 +21,13 @@ import java.util.List; import org.apache.iceberg.Schema; import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.schema.SchemaWithPartnerVisitor; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Visitor class that accumulates the set of changes needed to evolve an existing schema into the @@ -36,30 +39,39 @@ *

  • Adding new columns *
  • Widening the type of existing columsn *
  • Reordering columns + *
  • Dropping columns (when dropUnusedColumns is enabled) * * * We don't support: * *
      - *
    • Dropping columns *
    • Renaming columns *
    * - * The reason is that dropping columns would create issues with late / out of order data. Once we - * drop fields, we wouldn't be able to easily add them back later without losing the associated - * data. Renaming columns is not supported because we compare schemas by name, which doesn't allow - * for renaming without additional hints. + * By default, any columns present in the table but absent from the input schema are marked as + * optional to prevent issues caused by late or out-of-order data. If dropUnusedColumns is enabled, + * these columns are removed instead to ensure a strict one-to-one schema alignment. */ public class EvolveSchemaVisitor extends SchemaWithPartnerVisitor { + private static final Logger LOG = LoggerFactory.getLogger(EvolveSchemaVisitor.class); + private final TableIdentifier identifier; private final UpdateSchema api; private final Schema existingSchema; private final Schema targetSchema; - - private EvolveSchemaVisitor(UpdateSchema api, Schema existingSchema, Schema targetSchema) { + private final boolean dropUnusedColumns; + + private EvolveSchemaVisitor( + TableIdentifier identifier, + UpdateSchema api, + Schema existingSchema, + Schema targetSchema, + boolean dropUnusedColumns) { + this.identifier = identifier; this.api = api; this.existingSchema = existingSchema; this.targetSchema = targetSchema; + this.dropUnusedColumns = dropUnusedColumns; } /** @@ -70,12 +82,18 @@ private EvolveSchemaVisitor(UpdateSchema api, Schema existingSchema, Schema targ * @param api an UpdateSchema for adding changes * @param existingSchema an existing schema * @param targetSchema a new schema to compare with the existing + * @param dropUnusedColumns whether to drop columns not present in target schema */ - public static void visit(UpdateSchema api, Schema existingSchema, Schema targetSchema) { + public static void visit( + TableIdentifier identifier, + UpdateSchema api, + Schema existingSchema, + Schema targetSchema, + boolean dropUnusedColumns) { visit( targetSchema, -1, - new EvolveSchemaVisitor(api, existingSchema, targetSchema), + new EvolveSchemaVisitor(identifier, api, existingSchema, targetSchema, dropUnusedColumns), new CompareSchemasVisitor.PartnerIdByNameAccessors(existingSchema)); } @@ -103,11 +121,16 @@ public Boolean struct(Types.StructType struct, Integer partnerId, List after = columnName; } - // Ensure that unused fields are made optional for (Types.NestedField existingField : partnerStruct.fields()) { if (struct.field(existingField.name()) == null) { - if (existingField.isRequired()) { - this.api.makeColumnOptional(this.existingSchema.findColumnName(existingField.fieldId())); + String columnName = this.existingSchema.findColumnName(existingField.fieldId()); + if (dropUnusedColumns) { + LOG.debug("{}: Dropping column: {}", identifier.name(), columnName); + this.api.deleteColumn(columnName); + } else { + if (existingField.isRequired()) { + this.api.makeColumnOptional(columnName); + } } } } diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java index 2c08a3486e7c..8a8362a41996 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java @@ -89,8 +89,8 @@ String branch(TableIdentifier identifier, String branch) { return branch(identifier, branch, true); } - ResolvedSchemaInfo schema(TableIdentifier identifier, Schema input) { - return schema(identifier, input, true); + ResolvedSchemaInfo schema(TableIdentifier identifier, Schema input, boolean dropUnusedColumns) { + return schema(identifier, input, true, dropUnusedColumns); } PartitionSpec spec(TableIdentifier identifier, PartitionSpec spec) { @@ -124,7 +124,7 @@ private String branch(TableIdentifier identifier, String branch, boolean allowRe } private ResolvedSchemaInfo schema( - TableIdentifier identifier, Schema input, boolean allowRefresh) { + TableIdentifier identifier, Schema input, boolean allowRefresh, boolean dropUnusedColumns) { CacheItem cached = tableCache.get(identifier); Schema compatible = null; if (cached != null && cached.tableExists) { @@ -139,7 +139,7 @@ private ResolvedSchemaInfo schema( for (Map.Entry tableSchema : cached.tableSchemas.entrySet()) { CompareSchemasVisitor.Result result = - CompareSchemasVisitor.visit(input, tableSchema.getValue(), true); + CompareSchemasVisitor.visit(input, tableSchema.getValue(), true, dropUnusedColumns); if (result == CompareSchemasVisitor.Result.SAME) { ResolvedSchemaInfo newResult = new ResolvedSchemaInfo( @@ -157,7 +157,7 @@ private ResolvedSchemaInfo schema( if (needsRefresh(cached, allowRefresh)) { refreshTable(identifier); - return schema(identifier, input, false); + return schema(identifier, input, false, dropUnusedColumns); } else if (compatible != null) { ResolvedSchemaInfo newResult = new ResolvedSchemaInfo( diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java index cadfe345980c..d8809efbe541 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java @@ -43,10 +43,12 @@ class TableUpdater { private static final Logger LOG = LoggerFactory.getLogger(TableUpdater.class); private final TableMetadataCache cache; private final Catalog catalog; + private final boolean dropUnusedColumns; - TableUpdater(TableMetadataCache cache, Catalog catalog) { + TableUpdater(TableMetadataCache cache, Catalog catalog, boolean dropUnusedColumns) { this.cache = cache; this.catalog = catalog; + this.dropUnusedColumns = dropUnusedColumns; } /** @@ -118,13 +120,15 @@ private void findOrCreateBranch(TableIdentifier identifier, String branch) { private TableMetadataCache.ResolvedSchemaInfo findOrCreateSchema( TableIdentifier identifier, Schema schema) { - TableMetadataCache.ResolvedSchemaInfo fromCache = cache.schema(identifier, schema); + TableMetadataCache.ResolvedSchemaInfo fromCache = + cache.schema(identifier, schema, dropUnusedColumns); if (fromCache.compareResult() != CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED) { return fromCache; } else { Table table = catalog.loadTable(identifier); Schema tableSchema = table.schema(); - CompareSchemasVisitor.Result result = CompareSchemasVisitor.visit(schema, tableSchema, true); + CompareSchemasVisitor.Result result = + CompareSchemasVisitor.visit(schema, tableSchema, true, dropUnusedColumns); switch (result) { case SAME: cache.update(identifier, table); @@ -141,19 +145,20 @@ private TableMetadataCache.ResolvedSchemaInfo findOrCreateSchema( LOG.info( "Triggering schema update for table {} {} to {}", identifier, tableSchema, schema); UpdateSchema updateApi = table.updateSchema(); - EvolveSchemaVisitor.visit(updateApi, tableSchema, schema); + EvolveSchemaVisitor.visit(identifier, updateApi, tableSchema, schema, dropUnusedColumns); try { updateApi.commit(); cache.update(identifier, table); TableMetadataCache.ResolvedSchemaInfo comparisonAfterMigration = - cache.schema(identifier, schema); + cache.schema(identifier, schema, dropUnusedColumns); Schema newSchema = comparisonAfterMigration.resolvedTableSchema(); LOG.info("Table {} schema updated from {} to {}", identifier, tableSchema, newSchema); return comparisonAfterMigration; } catch (CommitFailedException e) { cache.invalidate(identifier); - TableMetadataCache.ResolvedSchemaInfo newSchema = cache.schema(identifier, schema); + TableMetadataCache.ResolvedSchemaInfo newSchema = + cache.schema(identifier, schema, dropUnusedColumns); if (newSchema.compareResult() != CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED) { LOG.debug("Table {} schema updated concurrently to {}", identifier, schema); return newSchema; diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java index 385a354889fb..cc8e6898d2ed 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java @@ -33,6 +33,9 @@ class TestCompareSchemasVisitor { + private static final boolean DROP_COLUMNS = true; + private static final boolean PRESERVE_COLUMNS = false; + @Test void testSchema() { assertThat( @@ -226,4 +229,63 @@ void testListChanged() { optional(2, "list1", ListType.ofOptional(3, IntegerType.get()))))) .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); } + + @Test + void testDropUnusedColumnsEnabled() { + Schema dataSchema = new Schema(optional(1, "id", IntegerType.get())); + Schema tableSchema = + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "data", StringType.get()), + optional(3, "extra", StringType.get())); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, DROP_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); + } + + @Test + void testDropUnusedColumnsWithRequiredField() { + Schema dataSchema = new Schema(optional(1, "id", IntegerType.get())); + Schema tableSchema = + new Schema(optional(1, "id", IntegerType.get()), required(2, "data", StringType.get())); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, DROP_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); + } + + @Test + void testDropUnusedColumnsWhenInputHasMoreFields() { + Schema dataSchema = + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "data", StringType.get()), + optional(3, "extra", StringType.get())); + Schema tableSchema = new Schema(optional(1, "id", IntegerType.get())); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, DROP_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); + } + + @Test + void testDropUnusedColumnsInNestedStruct() { + Schema dataSchema = + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "struct1", StructType.of(optional(3, "field1", StringType.get())))); + Schema tableSchema = + new Schema( + optional(1, "id", IntegerType.get()), + optional( + 2, + "struct1", + StructType.of( + optional(3, "field1", StringType.get()), + optional(4, "field2", IntegerType.get())))); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, DROP_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, PRESERVE_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.DATA_CONVERSION_NEEDED); + } } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java index b660d8e285d9..2711df72cab1 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java @@ -822,6 +822,55 @@ void testCommitsOnceWhenConcurrentDuplicateCommit(boolean overwriteMode) throws assertThat(totalAddedRecords).isEqualTo(records.size()); } + @Test + void testOptInDropUnusedColumns() throws Exception { + Schema schema1 = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "extra", Types.StringType.get())); + + Schema schema2 = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.of(DATABASE, "t1"); + catalog.createTable(tableIdentifier, schema1); + + List rows = + Lists.newArrayList( + // Drop columns + new DynamicIcebergDataImpl(schema2, "t1", "main", PartitionSpec.unpartitioned()), + // Re-add columns + new DynamicIcebergDataImpl(schema1, "t1", "main", PartitionSpec.unpartitioned())); + + DataStream dataStream = + env.fromData(rows, TypeInformation.of(new TypeHint<>() {})); + env.setParallelism(1); + + DynamicIcebergSink.forInput(dataStream) + .generator(new Generator()) + .catalogLoader(CATALOG_EXTENSION.catalogLoader()) + .immediateTableUpdate(true) + .dropUnusedColumns(true) + .append(); + + env.execute("Test Drop Unused Columns"); + + Table table = catalog.loadTable(tableIdentifier); + table.refresh(); + + assertThat(table.schema().columns()).hasSize(2); + assertThat(table.schema().findField("id")).isNotNull(); + assertThat(table.schema().findField("data")).isNotNull(); + assertThat(table.schema().findField("extra")).isNull(); + + List records = Lists.newArrayList(IcebergGenerics.read(table).build()); + assertThat(records).hasSize(2); + } + /** * Represents a concurrent duplicate commit during an ongoing commit operation, which can happen * in production scenarios when using REST catalog. diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java index 22655ff99f86..d68dd58c08fc 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java @@ -59,6 +59,7 @@ void testDynamicTableUpdateOperatorNewTable() throws Exception { DynamicTableUpdateOperator operator = new DynamicTableUpdateOperator( CATALOG_EXTENSION.catalogLoader(), + false, cacheMaximumSize, cacheRefreshMs, inputSchemaCacheMaximumSize, @@ -92,6 +93,7 @@ void testDynamicTableUpdateOperatorSchemaChange() throws Exception { DynamicTableUpdateOperator operator = new DynamicTableUpdateOperator( CATALOG_EXTENSION.catalogLoader(), + false, cacheMaximumSize, cacheRefreshMs, inputSchemaCacheMaximumSize, @@ -119,4 +121,84 @@ void testDynamicTableUpdateOperatorSchemaChange() throws Exception { assertThat(output2).isEqualTo(output); assertThat(catalog.loadTable(table).schema().schemaId()).isEqualTo(output.schema().schemaId()); } + + @Test + void testDynamicTableUpdateOperatorPreserveUnusedColumns() throws Exception { + int cacheMaximumSize = 10; + int cacheRefreshMs = 1000; + int inputSchemaCacheMaximumSize = 10; + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier table = TableIdentifier.of(TABLE); + + DynamicTableUpdateOperator operator = + new DynamicTableUpdateOperator( + CATALOG_EXTENSION.catalogLoader(), + false, // dropUnusedColumns = false (default) + cacheMaximumSize, + cacheRefreshMs, + inputSchemaCacheMaximumSize, + TableCreator.DEFAULT); + operator.open(null); + + catalog.createTable(table, SCHEMA2); + + DynamicRecordInternal input = + new DynamicRecordInternal( + TABLE, + "branch", + SCHEMA1, + GenericRowData.of(1), + PartitionSpec.unpartitioned(), + 42, + false, + Collections.emptySet()); + DynamicRecordInternal output = operator.map(input); + + Schema tableSchema = catalog.loadTable(table).schema(); + assertThat(tableSchema.columns()).hasSize(2); + assertThat(tableSchema.findField("id")).isNotNull(); + assertThat(tableSchema.findField("data")).isNotNull(); + assertThat(tableSchema.findField("data").isOptional()).isTrue(); + assertThat(input).isEqualTo(output); + } + + @Test + void testDynamicTableUpdateOperatorDropUnusedColumns() throws Exception { + int cacheMaximumSize = 10; + int cacheRefreshMs = 1000; + int inputSchemaCacheMaximumSize = 10; + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier table = TableIdentifier.of(TABLE); + + DynamicTableUpdateOperator operator = + new DynamicTableUpdateOperator( + CATALOG_EXTENSION.catalogLoader(), + // Drop unused columns + true, + cacheMaximumSize, + cacheRefreshMs, + inputSchemaCacheMaximumSize, + TableCreator.DEFAULT); + operator.open(null); + + catalog.createTable(table, SCHEMA2); + + DynamicRecordInternal input = + new DynamicRecordInternal( + TABLE, + "branch", + SCHEMA1, + GenericRowData.of(1), + PartitionSpec.unpartitioned(), + 42, + false, + Collections.emptySet()); + DynamicRecordInternal output = operator.map(input); + + Schema tableSchema = catalog.loadTable(table).schema(); + assertThat(tableSchema.columns()).hasSize(1); + assertThat(tableSchema.findField("id")).isNotNull(); + assertThat(tableSchema.findField("data")).isNull(); + assertThat(input).isEqualTo(output); + } } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java index d416e7ec1fc6..027adc4031bd 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java @@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.Schema; import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Type.PrimitiveType; import org.apache.iceberg.types.Types; @@ -48,6 +49,10 @@ public class TestEvolveSchemaVisitor { + private static final TableIdentifier TABLE = TableIdentifier.of("table"); + private static final boolean DROP_COLUMNS = true; + private static final boolean PRESERVE_COLUMNS = false; + private static List primitiveTypes() { return Lists.newArrayList( StringType.get(), @@ -89,7 +94,7 @@ private static Types.NestedField[] primitiveFields( public void testAddTopLevelPrimitives() { Schema targetSchema = new Schema(primitiveFields(0, primitiveTypes())); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(targetSchema.asStruct()).isEqualTo(updateApi.apply().asStruct()); } @@ -99,12 +104,59 @@ public void testMakeTopLevelPrimitivesOptional() { assertThat(existingSchema.columns().stream().allMatch(Types.NestedField::isRequired)).isTrue(); UpdateSchema updateApi = loadUpdateApi(existingSchema); - EvolveSchemaVisitor.visit(updateApi, existingSchema, new Schema()); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, new Schema(), PRESERVE_COLUMNS); Schema newSchema = updateApi.apply(); assertThat(newSchema.asStruct().fields()).hasSize(14); assertThat(newSchema.columns().stream().allMatch(Types.NestedField::isOptional)).isTrue(); } + @Test + public void testDropUnusedColumns() { + Schema existingSchema = + new Schema( + optional(1, "a", StringType.get()), + optional( + 2, + "b", + StructType.of( + optional(4, "nested1", StringType.get()), + optional(5, "nested2", StringType.get()))), + optional(3, "c", IntegerType.get())); + + Schema targetSchema = + new Schema( + optional(1, "a", StringType.get()), + optional(2, "b", StructType.of(optional(5, "nested2", StringType.get())))); + + UpdateSchema updateApi = loadUpdateApi(existingSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, DROP_COLUMNS); + + Schema newSchema = updateApi.apply(); + assertThat(newSchema.sameSchema(targetSchema)).isTrue(); + } + + @Test + public void testPreserveUnusedColumns() { + Schema existingSchema = + new Schema( + optional(1, "a", StringType.get()), + optional( + 2, + "b", + StructType.of( + optional(4, "nested1", StringType.get()), + optional(5, "nested2", StringType.get()))), + optional(3, "c", IntegerType.get())); + + Schema targetSchema = new Schema(optional(1, "a", StringType.get())); + + UpdateSchema updateApi = loadUpdateApi(existingSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, PRESERVE_COLUMNS); + + Schema newSchema = updateApi.apply(); + assertThat(newSchema.sameSchema(existingSchema)).isTrue(); + } + @Test public void testIdentifyFieldsByName() { Schema existingSchema = @@ -112,7 +164,7 @@ public void testIdentifyFieldsByName() { UpdateSchema updateApi = loadUpdateApi(existingSchema); Schema newSchema = new Schema(Arrays.asList(Types.NestedField.optional(-1, "myField", Types.LongType.get()))); - EvolveSchemaVisitor.visit(updateApi, existingSchema, newSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, newSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().sameSchema(existingSchema)).isTrue(); } @@ -125,7 +177,7 @@ public void testChangeOrderTopLevelPrimitives() { new Schema( Arrays.asList(optional(2, "b", StringType.get()), optional(1, "a", StringType.get()))); UpdateSchema updateApi = loadUpdateApi(existingSchema); - EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -134,7 +186,7 @@ public void testAddTopLevelListOfPrimitives() { for (PrimitiveType primitiveType : primitiveTypes()) { Schema targetSchema = new Schema(optional(1, "aList", ListType.ofOptional(2, primitiveType))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } } @@ -146,7 +198,7 @@ public void testMakeTopLevelListOfPrimitivesOptional() { new Schema(optional(1, "aList", ListType.ofRequired(2, primitiveType))); Schema targetSchema = new Schema(); UpdateSchema updateApi = loadUpdateApi(existingSchema); - EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, PRESERVE_COLUMNS); Schema expectedSchema = new Schema(optional(1, "aList", ListType.ofRequired(2, primitiveType))); assertThat(updateApi.apply().asStruct()).isEqualTo(expectedSchema.asStruct()); @@ -159,7 +211,7 @@ public void testAddTopLevelMapOfPrimitives() { Schema targetSchema = new Schema(optional(1, "aMap", MapType.ofOptional(2, 3, primitiveType, primitiveType))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } } @@ -171,7 +223,7 @@ public void testAddTopLevelStructOfPrimitives() { new Schema( optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), currentSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), currentSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(currentSchema.asStruct()); } } @@ -184,7 +236,7 @@ public void testAddNestedPrimitive() { new Schema( optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } } @@ -199,7 +251,7 @@ public void testMakeNestedPrimitiveOptional() { new Schema( optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } } @@ -210,7 +262,7 @@ public void testAddNestedPrimitives() { Schema targetSchema = new Schema(optional(1, "aStruct", StructType.of(primitiveFields(1, primitiveTypes())))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -240,7 +292,7 @@ public void testAddNestedLists() { ListType.ofOptional( 10, DecimalType.of(11, 20)))))))))))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -279,7 +331,7 @@ public void testAddNestedStruct() { "aString", StringType.get())))))))))))))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -314,7 +366,7 @@ public void testAddNestedMaps() { 12, 13, StringType.get(), StringType.get())))))))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -326,7 +378,11 @@ public void testDetectInvalidTopLevelList() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aList.element: string -> long") .isInstanceOf(IllegalArgumentException.class); } @@ -343,7 +399,11 @@ public void testDetectInvalidTopLevelMapValue() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aMap.value: string -> long") .isInstanceOf(IllegalArgumentException.class); } @@ -358,7 +418,11 @@ public void testDetectInvalidTopLevelMapKey() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aMap.key: string -> uuid") .isInstanceOf(IllegalArgumentException.class); } @@ -370,7 +434,7 @@ public void testTypePromoteIntegerToLong() { Schema targetSchema = new Schema(required(1, "aCol", LongType.get())); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); Schema applied = updateApi.apply(); assertThat(applied.asStruct().fields()).hasSize(1); assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(LongType.get()); @@ -383,7 +447,7 @@ public void testTypePromoteFloatToDouble() { Schema targetSchema = new Schema(required(1, "aCol", DoubleType.get())); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); Schema applied = updateApi.apply(); assertThat(applied.asStruct().fields()).hasSize(1); assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(DoubleType.get()); @@ -396,7 +460,11 @@ public void testInvalidTypePromoteDoubleToFloat() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aCol: double -> float") .isInstanceOf(IllegalArgumentException.class); } @@ -409,7 +477,7 @@ public void testTypePromoteDecimalToFixedScaleWithWiderPrecision() { Schema targetSchema = new Schema(required(1, "aCol", DecimalType.of(22, 1))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -452,7 +520,7 @@ public void testAddPrimitiveToNestedStruct() { optional(6, "time", TimeType.get()))))))))); UpdateSchema updateApi = loadUpdateApi(existingSchema); - EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -464,7 +532,11 @@ public void testReplaceListWithPrimitive() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aColumn: list -> string") .isInstanceOf(IllegalArgumentException.class); } @@ -501,7 +573,7 @@ public void addNewTopLevelStruct() { optional(7, "d1", StructType.of(optional(8, "d2", StringType.get())))))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -553,7 +625,7 @@ public void testAppendNestedStruct() { StringType.get())))))))))))))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -573,7 +645,7 @@ public void testMakeNestedStructOptional() { optional( 3, "s3", StructType.of(optional(4, "s4", StringType.get())))))))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(getNestedSchemaWithOptionalModifier(true).asStruct()) .isEqualTo(updateApi.apply().asStruct()); } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java index bf5b9f562f9a..7f91d2f8d585 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java @@ -53,17 +53,22 @@ void testCaching() { catalog.createTable(tableIdentifier, SCHEMA); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - Schema schema1 = cache.schema(tableIdentifier, SCHEMA).resolvedTableSchema(); + Schema schema1 = cache.schema(tableIdentifier, SCHEMA, false).resolvedTableSchema(); assertThat(schema1.sameSchema(SCHEMA)).isTrue(); assertThat( - cache.schema(tableIdentifier, SerializationUtils.clone(SCHEMA)).resolvedTableSchema()) + cache + .schema(tableIdentifier, SerializationUtils.clone(SCHEMA), false) + .resolvedTableSchema()) .isEqualTo(schema1); - assertThat(cache.schema(tableIdentifier, SCHEMA2)).isEqualTo(TableMetadataCache.NOT_FOUND); + assertThat(cache.schema(tableIdentifier, SCHEMA2, false)) + .isEqualTo(TableMetadataCache.NOT_FOUND); - schema1 = cache.schema(tableIdentifier, SCHEMA).resolvedTableSchema(); + schema1 = cache.schema(tableIdentifier, SCHEMA, false).resolvedTableSchema(); assertThat( - cache.schema(tableIdentifier, SerializationUtils.clone(SCHEMA)).resolvedTableSchema()) + cache + .schema(tableIdentifier, SerializationUtils.clone(SCHEMA), false) + .resolvedTableSchema()) .isEqualTo(schema1); } @@ -73,9 +78,9 @@ void testCacheInvalidationAfterSchemaChange() { TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); catalog.createTable(tableIdentifier, SCHEMA); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); - Schema schema1 = cache.schema(tableIdentifier, SCHEMA).resolvedTableSchema(); + Schema schema1 = cache.schema(tableIdentifier, SCHEMA, false).resolvedTableSchema(); assertThat(schema1.sameSchema(SCHEMA)).isTrue(); catalog.dropTable(tableIdentifier); @@ -83,7 +88,7 @@ void testCacheInvalidationAfterSchemaChange() { tableUpdater.update( tableIdentifier, "main", SCHEMA2, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); - Schema schema2 = cache.schema(tableIdentifier, SCHEMA2).resolvedTableSchema(); + Schema schema2 = cache.schema(tableIdentifier, SCHEMA2, false).resolvedTableSchema(); assertThat(schema2.sameSchema(SCHEMA2)).isTrue(); } @@ -111,11 +116,11 @@ void testNoCacheRefreshingBeforeRefreshIntervalElapses() { cache.update(tableIdentifier, table); // Cache schema - Schema schema = cache.schema(tableIdentifier, SCHEMA2).resolvedTableSchema(); + Schema schema = cache.schema(tableIdentifier, SCHEMA2, false).resolvedTableSchema(); assertThat(schema.sameSchema(SCHEMA2)).isTrue(); // Cache schema with fewer fields - TableMetadataCache.ResolvedSchemaInfo schemaInfo = cache.schema(tableIdentifier, SCHEMA); + TableMetadataCache.ResolvedSchemaInfo schemaInfo = cache.schema(tableIdentifier, SCHEMA, false); assertThat(schemaInfo.resolvedTableSchema().sameSchema(SCHEMA2)).isTrue(); assertThat(schemaInfo.compareResult()) .isEqualTo(CompareSchemasVisitor.Result.DATA_CONVERSION_NEEDED); diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java index 1d4461698746..bcc5d8064517 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java @@ -56,7 +56,7 @@ void testTableCreation(@TempDir Path tempDir) { catalog.createNamespace(Namespace.of("myNamespace")); TableIdentifier tableIdentifier = TableIdentifier.parse("myNamespace.myTable"); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); String locationOverride = tempDir.toString() + "/custom-path"; Map tableProperties = Map.of("key", "value"); @@ -69,7 +69,8 @@ void testTableCreation(@TempDir Path tempDir) { assertThat(catalog.tableExists(tableIdentifier)).isTrue(); assertThat(catalog.loadTable(tableIdentifier).properties().get("key")).isEqualTo("value"); assertThat(catalog.loadTable(tableIdentifier).location()).isEqualTo(locationOverride); - TableMetadataCache.ResolvedSchemaInfo cachedSchema = cache.schema(tableIdentifier, SCHEMA); + TableMetadataCache.ResolvedSchemaInfo cachedSchema = + cache.schema(tableIdentifier, SCHEMA, false); assertThat(cachedSchema.resolvedTableSchema().sameSchema(SCHEMA)).isTrue(); } @@ -78,7 +79,7 @@ void testTableAlreadyExists() { Catalog catalog = CATALOG_EXTENSION.catalog(); TableIdentifier tableIdentifier = TableIdentifier.parse("myTable"); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); // Make the table non-existent in cache cache.exists(tableIdentifier); @@ -98,7 +99,7 @@ void testBranchCreationAndCaching() { Catalog catalog = CATALOG_EXTENSION.catalog(); TableIdentifier tableIdentifier = TableIdentifier.parse("myTable"); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); catalog.createTable(tableIdentifier, SCHEMA); tableUpdater.update( @@ -116,7 +117,7 @@ void testSpecCreation() { Catalog catalog = CATALOG_EXTENSION.catalog(); TableIdentifier tableIdentifier = TableIdentifier.parse("myTable"); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).bucket("data", 10).build(); tableUpdater.update(tableIdentifier, "main", SCHEMA, spec, TableCreator.DEFAULT); @@ -132,8 +133,8 @@ void testInvalidateOldCacheEntryOnUpdate() { TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); catalog.createTable(tableIdentifier, SCHEMA); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - cache.schema(tableIdentifier, SCHEMA); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + cache.schema(tableIdentifier, SCHEMA, false); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); Schema updated = tableUpdater @@ -146,7 +147,8 @@ void testInvalidateOldCacheEntryOnUpdate() { .f0 .resolvedTableSchema(); assertThat(updated.sameSchema(SCHEMA2)).isTrue(); - assertThat(cache.schema(tableIdentifier, SCHEMA2).resolvedTableSchema().sameSchema(SCHEMA2)) + assertThat( + cache.schema(tableIdentifier, SCHEMA2, false).resolvedTableSchema().sameSchema(SCHEMA2)) .isTrue(); } @@ -156,7 +158,7 @@ void testLastResultInvalidation() { TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); catalog.createTable(tableIdentifier, SCHEMA); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); // Initialize cache tableUpdater.update( @@ -167,7 +169,7 @@ void testLastResultInvalidation() { catalog.createTable(tableIdentifier, SCHEMA2); // Cache still stores the old information - assertThat(cache.schema(tableIdentifier, SCHEMA2).compareResult()) + assertThat(cache.schema(tableIdentifier, SCHEMA2, false).compareResult()) .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); assertThat( @@ -186,4 +188,27 @@ void testLastResultInvalidation() { assertThat(cache.getInternalCache().get(tableIdentifier).inputSchemas()) .doesNotContainKey(SCHEMA2); } + + @Test + void testDropUnusedColumns() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("myTable"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + + final boolean dropUnusedColumns = true; + TableUpdater tableUpdater = new TableUpdater(cache, catalog, dropUnusedColumns); + + catalog.createTable(tableIdentifier, SCHEMA2); + + Tuple2 result = + tableUpdater.update( + tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + + assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); + Schema tableSchema = catalog.loadTable(tableIdentifier).schema(); + assertThat(tableSchema.columns()).hasSize(2); + assertThat(tableSchema.findField("id")).isNotNull(); + assertThat(tableSchema.findField("data")).isNotNull(); + assertThat(tableSchema.findField("extra")).isNull(); + } } From 01b29dd6403c8cc1fad084c3e11f2cb4fe88177e Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Mon, 8 Dec 2025 22:57:20 +0900 Subject: [PATCH 060/201] Build: Bump datamodel-code-generator from 0.36.0 to 0.41.0 (#14791) * Build: Bump datamodel-code-generator from 0.36.0 to 0.37.0 * Build: Bump datamodel-code-generator from 0.37.0 to 0.38.0 * Build: Bump datamodel-code-generator from 0.38.0 to 0.39.0 * Build: Bump datamodel-code-generator from 0.39.0 to 0.40.0 * Build: Bump datamodel-code-generator from 0.40.0 to 0.41.0 --- open-api/requirements.txt | 2 +- open-api/rest-catalog-open-api.py | 295 +++++++++++++++++++++++------- 2 files changed, 227 insertions(+), 70 deletions(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index fcc9285b35bd..7f8a7438115e 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,5 +16,5 @@ # under the License. openapi-spec-validator==0.7.2 -datamodel-code-generator==0.36.0 +datamodel-code-generator==0.41.0 yamllint==1.37.1 diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 6627e4516bc8..9b4fd0c6a85a 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -15,6 +15,7 @@ # specific language governing permissions and limitations # under the License. + from __future__ import annotations from datetime import date, timedelta @@ -298,22 +299,22 @@ class BaseUpdate(BaseModel): action: str -class AssignUUIDUpdate(BaseUpdate): +class AssignUUIDUpdate(BaseModel): """ Assigning a UUID to a table/view should only be done when creating the table/view. It is not safe to re-assign the UUID if a table/view already has a UUID assigned """ - action: str = Field('assign-uuid', const=True) + action: str = Field(..., const=True) uuid: str -class UpgradeFormatVersionUpdate(BaseUpdate): - action: str = Field('upgrade-format-version', const=True) +class UpgradeFormatVersionUpdate(BaseModel): + action: str = Field(..., const=True) format_version: int = Field(..., alias='format-version') -class SetCurrentSchemaUpdate(BaseUpdate): - action: str = Field('set-current-schema', const=True) +class SetCurrentSchemaUpdate(BaseModel): + action: str = Field(..., const=True) schema_id: int = Field( ..., alias='schema-id', @@ -321,13 +322,13 @@ class SetCurrentSchemaUpdate(BaseUpdate): ) -class AddPartitionSpecUpdate(BaseUpdate): - action: str = Field('add-spec', const=True) +class AddPartitionSpecUpdate(BaseModel): + action: str = Field(..., const=True) spec: PartitionSpec -class SetDefaultSpecUpdate(BaseUpdate): - action: str = Field('set-default-spec', const=True) +class SetDefaultSpecUpdate(BaseModel): + action: str = Field(..., const=True) spec_id: int = Field( ..., alias='spec-id', @@ -335,13 +336,13 @@ class SetDefaultSpecUpdate(BaseUpdate): ) -class AddSortOrderUpdate(BaseUpdate): - action: str = Field('add-sort-order', const=True) +class AddSortOrderUpdate(BaseModel): + action: str = Field(..., const=True) sort_order: SortOrder = Field(..., alias='sort-order') -class SetDefaultSortOrderUpdate(BaseUpdate): - action: str = Field('set-default-sort-order', const=True) +class SetDefaultSortOrderUpdate(BaseModel): + action: str = Field(..., const=True) sort_order_id: int = Field( ..., alias='sort-order-id', @@ -349,48 +350,53 @@ class SetDefaultSortOrderUpdate(BaseUpdate): ) -class AddSnapshotUpdate(BaseUpdate): - action: str = Field('add-snapshot', const=True) +class AddSnapshotUpdate(BaseModel): + action: str = Field(..., const=True) snapshot: Snapshot -class SetSnapshotRefUpdate(BaseUpdate, SnapshotReference): - action: str = Field('set-snapshot-ref', const=True) +class SetSnapshotRefUpdate(BaseModel): + action: str = Field(..., const=True) ref_name: str = Field(..., alias='ref-name') + type: Literal['tag', 'branch'] + snapshot_id: int = Field(..., alias='snapshot-id') + max_ref_age_ms: Optional[int] = Field(None, alias='max-ref-age-ms') + max_snapshot_age_ms: Optional[int] = Field(None, alias='max-snapshot-age-ms') + min_snapshots_to_keep: Optional[int] = Field(None, alias='min-snapshots-to-keep') -class RemoveSnapshotsUpdate(BaseUpdate): - action: str = Field('remove-snapshots', const=True) +class RemoveSnapshotsUpdate(BaseModel): + action: str = Field(..., const=True) snapshot_ids: List[int] = Field(..., alias='snapshot-ids') -class RemoveSnapshotRefUpdate(BaseUpdate): - action: str = Field('remove-snapshot-ref', const=True) +class RemoveSnapshotRefUpdate(BaseModel): + action: str = Field(..., const=True) ref_name: str = Field(..., alias='ref-name') -class SetLocationUpdate(BaseUpdate): - action: str = Field('set-location', const=True) +class SetLocationUpdate(BaseModel): + action: str = Field(..., const=True) location: str -class SetPropertiesUpdate(BaseUpdate): - action: str = Field('set-properties', const=True) +class SetPropertiesUpdate(BaseModel): + action: str = Field(..., const=True) updates: Dict[str, str] -class RemovePropertiesUpdate(BaseUpdate): - action: str = Field('remove-properties', const=True) +class RemovePropertiesUpdate(BaseModel): + action: str = Field(..., const=True) removals: List[str] -class AddViewVersionUpdate(BaseUpdate): - action: str = Field('add-view-version', const=True) +class AddViewVersionUpdate(BaseModel): + action: str = Field(..., const=True) view_version: ViewVersion = Field(..., alias='view-version') -class SetCurrentViewVersionUpdate(BaseUpdate): - action: str = Field('set-current-view-version', const=True) +class SetCurrentViewVersionUpdate(BaseModel): + action: str = Field(..., const=True) view_version_id: int = Field( ..., alias='view-version-id', @@ -398,33 +404,33 @@ class SetCurrentViewVersionUpdate(BaseUpdate): ) -class RemoveStatisticsUpdate(BaseUpdate): - action: str = Field('remove-statistics', const=True) +class RemoveStatisticsUpdate(BaseModel): + action: str = Field(..., const=True) snapshot_id: int = Field(..., alias='snapshot-id') -class RemovePartitionStatisticsUpdate(BaseUpdate): - action: str = Field('remove-partition-statistics', const=True) +class RemovePartitionStatisticsUpdate(BaseModel): + action: str = Field(..., const=True) snapshot_id: int = Field(..., alias='snapshot-id') -class RemovePartitionSpecsUpdate(BaseUpdate): - action: str = Field('remove-partition-specs', const=True) +class RemovePartitionSpecsUpdate(BaseModel): + action: str = Field(..., const=True) spec_ids: List[int] = Field(..., alias='spec-ids') -class RemoveSchemasUpdate(BaseUpdate): - action: str = Field('remove-schemas', const=True) +class RemoveSchemasUpdate(BaseModel): + action: str = Field(..., const=True) schema_ids: List[int] = Field(..., alias='schema-ids') -class AddEncryptionKeyUpdate(BaseUpdate): - action: str = Field('add-encryption-key', const=True) +class AddEncryptionKeyUpdate(BaseModel): + action: str = Field(..., const=True) encryption_key: EncryptedKey = Field(..., alias='encryption-key') -class RemoveEncryptionKeyUpdate(BaseUpdate): - action: str = Field('remove-encryption-key', const=True) +class RemoveEncryptionKeyUpdate(BaseModel): + action: str = Field(..., const=True) key_id: str = Field(..., alias='key-id') @@ -432,7 +438,7 @@ class TableRequirement(BaseModel): type: str -class AssertCreate(TableRequirement): +class AssertCreate(BaseModel): """ The table must not already exist; used for create transactions """ @@ -440,7 +446,7 @@ class AssertCreate(TableRequirement): type: str = Field(..., const=True) -class AssertTableUUID(TableRequirement): +class AssertTableUUID(BaseModel): """ The table UUID must match the requirement's `uuid` """ @@ -449,7 +455,7 @@ class AssertTableUUID(TableRequirement): uuid: str -class AssertRefSnapshotId(TableRequirement): +class AssertRefSnapshotId(BaseModel): """ The table branch or tag identified by the requirement's `ref` must reference the requirement's `snapshot-id`. The `snapshot-id` field is required in this object, but in the case of a `null` @@ -457,53 +463,53 @@ class AssertRefSnapshotId(TableRequirement): """ - type: str = Field('assert-ref-snapshot-id', const=True) + type: str = Field(..., const=True) ref: str snapshot_id: int = Field(..., alias='snapshot-id') -class AssertLastAssignedFieldId(TableRequirement): +class AssertLastAssignedFieldId(BaseModel): """ The table's last assigned column id must match the requirement's `last-assigned-field-id` """ - type: str = Field('assert-last-assigned-field-id', const=True) + type: str = Field(..., const=True) last_assigned_field_id: int = Field(..., alias='last-assigned-field-id') -class AssertCurrentSchemaId(TableRequirement): +class AssertCurrentSchemaId(BaseModel): """ The table's current schema id must match the requirement's `current-schema-id` """ - type: str = Field('assert-current-schema-id', const=True) + type: str = Field(..., const=True) current_schema_id: int = Field(..., alias='current-schema-id') -class AssertLastAssignedPartitionId(TableRequirement): +class AssertLastAssignedPartitionId(BaseModel): """ The table's last assigned partition id must match the requirement's `last-assigned-partition-id` """ - type: str = Field('assert-last-assigned-partition-id', const=True) + type: str = Field(..., const=True) last_assigned_partition_id: int = Field(..., alias='last-assigned-partition-id') -class AssertDefaultSpecId(TableRequirement): +class AssertDefaultSpecId(BaseModel): """ The table's default spec id must match the requirement's `default-spec-id` """ - type: str = Field('assert-default-spec-id', const=True) + type: str = Field(..., const=True) default_spec_id: int = Field(..., alias='default-spec-id') -class AssertDefaultSortOrderId(TableRequirement): +class AssertDefaultSortOrderId(BaseModel): """ The table's default sort order id must match the requirement's `default-sort-order-id` """ - type: str = Field('assert-default-sort-order-id', const=True) + type: str = Field(..., const=True) default_sort_order_id: int = Field(..., alias='default-sort-order-id') @@ -911,7 +917,7 @@ class ContentFile(BaseModel): sort_order_id: Optional[int] = Field(None, alias='sort-order-id') -class PositionDeleteFile(ContentFile): +class PositionDeleteFile(BaseModel): content: Literal['position-deletes'] = Field(..., const=True) content_offset: Optional[int] = Field( None, @@ -923,13 +929,55 @@ class PositionDeleteFile(ContentFile): alias='content-size-in-bytes', description='Length, in bytes, of the delete content; required if content-offset is present', ) + file_path: str = Field(..., alias='file-path') + file_format: FileFormat = Field(..., alias='file-format') + spec_id: int = Field(..., alias='spec-id') + partition: List[PrimitiveTypeValue] = Field( + ..., + description='A list of partition field values ordered based on the fields of the partition spec specified by the `spec-id`', + example=[1, 'bar'], + ) + file_size_in_bytes: int = Field( + ..., alias='file-size-in-bytes', description='Total file size in bytes' + ) + record_count: int = Field( + ..., alias='record-count', description='Number of records in the file' + ) + key_metadata: Optional[BinaryTypeValue] = Field( + None, alias='key-metadata', description='Encryption key metadata blob' + ) + split_offsets: Optional[List[int]] = Field( + None, alias='split-offsets', description='List of splittable offsets' + ) + sort_order_id: Optional[int] = Field(None, alias='sort-order-id') -class EqualityDeleteFile(ContentFile): +class EqualityDeleteFile(BaseModel): content: Literal['equality-deletes'] = Field(..., const=True) equality_ids: Optional[List[int]] = Field( None, alias='equality-ids', description='List of equality field IDs' ) + file_path: str = Field(..., alias='file-path') + file_format: FileFormat = Field(..., alias='file-format') + spec_id: int = Field(..., alias='spec-id') + partition: List[PrimitiveTypeValue] = Field( + ..., + description='A list of partition field values ordered based on the fields of the partition spec specified by the `spec-id`', + example=[1, 'bar'], + ) + file_size_in_bytes: int = Field( + ..., alias='file-size-in-bytes', description='Total file size in bytes' + ) + record_count: int = Field( + ..., alias='record-count', description='Number of records in the file' + ) + key_metadata: Optional[BinaryTypeValue] = Field( + None, alias='key-metadata', description='Encryption key metadata blob' + ) + split_offsets: Optional[List[int]] = Field( + None, alias='split-offsets', description='List of splittable offsets' + ) + sort_order_id: Optional[int] = Field(None, alias='sort-order-id') class FieldName(BaseModel): @@ -946,6 +994,27 @@ class PlanTask(BaseModel): ) +class ResidualFilter1(BaseModel): + """ + An optional filter to be applied to rows in this file scan task. + If the residual is not present, the client must produce the residual or use the original filter. + """ + + +class ResidualFilter2(TrueExpression, ResidualFilter1): + """ + An optional filter to be applied to rows in this file scan task. + If the residual is not present, the client must produce the residual or use the original filter. + """ + + +class ResidualFilter3(FalseExpression, ResidualFilter1): + """ + An optional filter to be applied to rows in this file scan task. + If the residual is not present, the client must produce the residual or use the original filter. + """ + + class CreateNamespaceRequest(BaseModel): namespace: Namespace properties: Optional[Dict[str, str]] = Field( @@ -966,8 +1035,8 @@ class TransformTerm(BaseModel): term: Reference -class SetPartitionStatisticsUpdate(BaseUpdate): - action: str = Field('set-partition-statistics', const=True) +class SetPartitionStatisticsUpdate(BaseModel): + action: str = Field(..., const=True) partition_statistics: PartitionStatisticsFile = Field( ..., alias='partition-statistics' ) @@ -1021,7 +1090,7 @@ class ValueMap(BaseModel): ) -class DataFile(ContentFile): +class DataFile(BaseModel): content: str = Field(..., const=True) first_row_id: Optional[int] = Field( None, @@ -1056,6 +1125,27 @@ class DataFile(ContentFile): alias='upper-bounds', description='Map of column id to upper bound primitive type values', ) + file_path: str = Field(..., alias='file-path') + file_format: FileFormat = Field(..., alias='file-format') + spec_id: int = Field(..., alias='spec-id') + partition: List[PrimitiveTypeValue] = Field( + ..., + description='A list of partition field values ordered based on the fields of the partition spec specified by the `spec-id`', + example=[1, 'bar'], + ) + file_size_in_bytes: int = Field( + ..., alias='file-size-in-bytes', description='Total file size in bytes' + ) + record_count: int = Field( + ..., alias='record-count', description='Number of records in the file' + ) + key_metadata: Optional[BinaryTypeValue] = Field( + None, alias='key-metadata', description='Encryption key metadata blob' + ) + split_offsets: Optional[List[int]] = Field( + None, alias='split-offsets', description='List of splittable offsets' + ) + sort_order_id: Optional[int] = Field(None, alias='sort-order-id') class DeleteFile(BaseModel): @@ -1072,8 +1162,8 @@ class Term(BaseModel): __root__: Union[Reference, TransformTerm] -class SetStatisticsUpdate(BaseUpdate): - action: str = Field('set-statistics', const=True) +class SetStatisticsUpdate(BaseModel): + action: str = Field(..., const=True) snapshot_id: Optional[int] = Field( None, alias='snapshot-id', @@ -1099,6 +1189,27 @@ class SetExpression(BaseModel): values: List[PrimitiveTypeValue] +class ResidualFilter6(SetExpression, ResidualFilter1): + """ + An optional filter to be applied to rows in this file scan task. + If the residual is not present, the client must produce the residual or use the original filter. + """ + + +class ResidualFilter7(LiteralExpression, ResidualFilter1): + """ + An optional filter to be applied to rows in this file scan task. + If the residual is not present, the client must produce the residual or use the original filter. + """ + + +class ResidualFilter8(UnaryExpression, ResidualFilter1): + """ + An optional filter to be applied to rows in this file scan task. + If the residual is not present, the client must produce the residual or use the original filter. + """ + + class StructField(BaseModel): id: int name: str @@ -1204,8 +1315,8 @@ class ViewMetadata(BaseModel): properties: Optional[Dict[str, str]] = None -class AddSchemaUpdate(BaseUpdate): - action: str = Field('add-schema', const=True) +class AddSchemaUpdate(BaseModel): + action: str = Field(..., const=True) schema_: Schema = Field(..., alias='schema') last_column_id: Optional[int] = Field( None, @@ -1351,7 +1462,18 @@ class CommitTableRequest(BaseModel): None, description='Table identifier to update; must be present for CommitTransactionRequest', ) - requirements: List[TableRequirement] + requirements: List[ + Union[ + AssertCreate, + AssertTableUUID, + AssertRefSnapshotId, + AssertLastAssignedFieldId, + AssertCurrentSchemaId, + AssertLastAssignedPartitionId, + AssertDefaultSpecId, + AssertDefaultSortOrderId, + ] + ] updates: List[TableUpdate] @@ -1476,6 +1598,26 @@ class PlanTableScanRequest(BaseModel): ) +class ResidualFilter(BaseModel): + """ + An optional filter to be applied to rows in this file scan task. + If the residual is not present, the client must produce the residual or use the original filter. + """ + + __root__: Union[ + ResidualFilter2, + ResidualFilter3, + ResidualFilter4, + ResidualFilter5, + ResidualFilter6, + ResidualFilter7, + ResidualFilter8, + ] = Field( + ..., + description='An optional filter to be applied to rows in this file scan task.\nIf the residual is not present, the client must produce the residual or use the original filter.', + ) + + class FileScanTask(BaseModel): data_file: DataFile = Field(..., alias='data-file') delete_file_references: Optional[List[int]] = Field( @@ -1483,7 +1625,7 @@ class FileScanTask(BaseModel): alias='delete-file-references', description='A list of indices in the delete files array (0-based)', ) - residual_filter: Optional[Expression] = Field( + residual_filter: Optional[ResidualFilter] = Field( None, alias='residual-filter', description='An optional filter to be applied to rows in this file scan task.\nIf the residual is not present, the client must produce the residual or use the original filter.', @@ -1497,6 +1639,20 @@ class Schema(StructType): ) +class ResidualFilter4(AndOrExpression, ResidualFilter1): + """ + An optional filter to be applied to rows in this file scan task. + If the residual is not present, the client must produce the residual or use the original filter. + """ + + +class ResidualFilter5(NotExpression, ResidualFilter1): + """ + An optional filter to be applied to rows in this file scan task. + If the residual is not present, the client must produce the residual or use the original filter. + """ + + class CompletedPlanningResult(ScanTasks): """ Completed server-side planning result @@ -1540,6 +1696,7 @@ class CompletedPlanningWithIDResult(CompletedPlanningResult): CreateTableRequest.update_forward_refs() CreateViewRequest.update_forward_refs() ReportMetricsRequest.update_forward_refs() +ResidualFilter.update_forward_refs() CompletedPlanningResult.update_forward_refs() FetchScanTasksResult.update_forward_refs() CompletedPlanningWithIDResult.update_forward_refs() From a9482bd6a51965648e3b856f130445b0490d339e Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Mon, 8 Dec 2025 16:23:12 +0100 Subject: [PATCH 061/201] Flink: Backport: Dynamic Sink: Add support for dropping columns (#14799) * Backport: Flink: Dynamic Sink: Add support for dropping columns (#14728) --- .../sink/dynamic/CompareSchemasVisitor.java | 18 ++- .../sink/dynamic/DynamicIcebergSink.java | 19 +++ .../sink/dynamic/DynamicRecordProcessor.java | 7 +- .../dynamic/DynamicTableUpdateOperator.java | 6 +- .../sink/dynamic/EvolveSchemaVisitor.java | 47 +++++-- .../sink/dynamic/TableMetadataCache.java | 10 +- .../flink/sink/dynamic/TableUpdater.java | 17 ++- .../dynamic/TestCompareSchemasVisitor.java | 62 +++++++++ .../sink/dynamic/TestDynamicIcebergSink.java | 49 +++++++ .../TestDynamicTableUpdateOperator.java | 82 ++++++++++++ .../sink/dynamic/TestEvolveSchemaVisitor.java | 124 ++++++++++++++---- .../sink/dynamic/TestTableMetadataCache.java | 25 ++-- .../flink/sink/dynamic/TestTableUpdater.java | 45 +++++-- .../sink/dynamic/CompareSchemasVisitor.java | 18 ++- .../sink/dynamic/DynamicIcebergSink.java | 19 +++ .../sink/dynamic/DynamicRecordProcessor.java | 7 +- .../dynamic/DynamicTableUpdateOperator.java | 6 +- .../sink/dynamic/EvolveSchemaVisitor.java | 47 +++++-- .../sink/dynamic/TableMetadataCache.java | 10 +- .../flink/sink/dynamic/TableUpdater.java | 17 ++- .../dynamic/TestCompareSchemasVisitor.java | 62 +++++++++ .../sink/dynamic/TestDynamicIcebergSink.java | 49 +++++++ .../TestDynamicTableUpdateOperator.java | 82 ++++++++++++ .../sink/dynamic/TestEvolveSchemaVisitor.java | 124 ++++++++++++++---- .../sink/dynamic/TestTableMetadataCache.java | 25 ++-- .../flink/sink/dynamic/TestTableUpdater.java | 45 +++++-- 26 files changed, 864 insertions(+), 158 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java index 41ffa609540b..60561b0f56bf 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java @@ -43,20 +43,23 @@ public class CompareSchemasVisitor extends SchemaWithPartnerVisitor { private final Schema tableSchema; + private final boolean dropUnusedColumns; - private CompareSchemasVisitor(Schema tableSchema) { + private CompareSchemasVisitor(Schema tableSchema, boolean dropUnusedColumns) { this.tableSchema = tableSchema; + this.dropUnusedColumns = dropUnusedColumns; } public static Result visit(Schema dataSchema, Schema tableSchema) { - return visit(dataSchema, tableSchema, true); + return visit(dataSchema, tableSchema, true, false); } - public static Result visit(Schema dataSchema, Schema tableSchema, boolean caseSensitive) { + public static Result visit( + Schema dataSchema, Schema tableSchema, boolean caseSensitive, boolean dropUnusedColumns) { return visit( dataSchema, -1, - new CompareSchemasVisitor(tableSchema), + new CompareSchemasVisitor(tableSchema, dropUnusedColumns), new PartnerIdByNameAccessors(tableSchema, caseSensitive)); } @@ -70,6 +73,7 @@ public Result schema(Schema dataSchema, Integer tableSchemaId, Result downstream } @Override + @SuppressWarnings("CyclomaticComplexity") public Result struct(Types.StructType struct, Integer tableSchemaId, List fields) { if (tableSchemaId == null) { return Result.SCHEMA_UPDATE_NEEDED; @@ -88,10 +92,10 @@ public Result struct(Types.StructType struct, Integer tableSchemaId, List { private ReadableConfig readableConfig = new Configuration(); private TableCreator tableCreator = TableCreator.DEFAULT; private boolean immediateUpdate = false; + private boolean dropUnusedColumns = false; private int cacheMaximumSize = 100; private long cacheRefreshMs = 1_000; private int inputSchemasPerTableCacheMaximumSize = 10; @@ -314,6 +315,22 @@ public Builder immediateTableUpdate(boolean newImmediateUpdate) { return this; } + /** + * Dropping columns is disabled by default to prevent issues with late or out-of-order data, as + * removed fields cannot be easily restored without data loss. + * + *

    You can opt-in to allow dropping columns. Once a column has been dropped, it is + * technically still possible to write data to that column because Iceberg maintains all past + * table schemas. However, regular queries won't be able to reference the column. If the field + * was to re-appear as part of a new schema, an entirely new column would be added, which apart + * from the name, has nothing in common with the old column, i.e. queries for the new column + * will never return data of the old column. + */ + public Builder dropUnusedColumns(boolean newDropUnusedColumns) { + this.dropUnusedColumns = newDropUnusedColumns; + return this; + } + /** Maximum size of the caches used in Dynamic Sink for table data and serializers. */ public Builder cacheMaxSize(int maxSize) { this.cacheMaximumSize = maxSize; @@ -382,6 +399,7 @@ public DataStreamSink append() { generator, catalogLoader, immediateUpdate, + dropUnusedColumns, cacheMaximumSize, cacheRefreshMs, inputSchemasPerTableCacheMaximumSize, @@ -400,6 +418,7 @@ public DataStreamSink append() { .map( new DynamicTableUpdateOperator( catalogLoader, + dropUnusedColumns, cacheMaximumSize, cacheRefreshMs, inputSchemasPerTableCacheMaximumSize, diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java index bc3a25468d84..427aa6ceafba 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java @@ -40,6 +40,7 @@ class DynamicRecordProcessor extends ProcessFunction generator; private final CatalogLoader catalogLoader; private final boolean immediateUpdate; + private final boolean dropUnusedColumns; private final int cacheMaximumSize; private final long cacheRefreshMs; private final int inputSchemasPerTableCacheMaximumSize; @@ -56,6 +57,7 @@ class DynamicRecordProcessor extends ProcessFunction generator, CatalogLoader catalogLoader, boolean immediateUpdate, + boolean dropUnusedColumns, int cacheMaximumSize, long cacheRefreshMs, int inputSchemasPerTableCacheMaximumSize, @@ -63,6 +65,7 @@ class DynamicRecordProcessor extends ProcessFunction( @@ -106,7 +109,7 @@ public void collect(DynamicRecord data) { TableMetadataCache.ResolvedSchemaInfo foundSchema = exists - ? tableCache.schema(data.tableIdentifier(), data.schema()) + ? tableCache.schema(data.tableIdentifier(), data.schema(), dropUnusedColumns) : TableMetadataCache.NOT_FOUND; PartitionSpec foundSpec = exists ? tableCache.spec(data.tableIdentifier(), data.spec()) : null; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java index 586239b54bca..8f38d4f8be0c 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java @@ -38,6 +38,7 @@ class DynamicTableUpdateOperator extends RichMapFunction { private final CatalogLoader catalogLoader; + private final boolean dropUnusedColumns; private final int cacheMaximumSize; private final long cacheRefreshMs; private final int inputSchemasPerTableCacheMaximumSize; @@ -47,11 +48,13 @@ class DynamicTableUpdateOperator DynamicTableUpdateOperator( CatalogLoader catalogLoader, + boolean dropUnusedColumns, int cacheMaximumSize, long cacheRefreshMs, int inputSchemasPerTableCacheMaximumSize, TableCreator tableCreator) { this.catalogLoader = catalogLoader; + this.dropUnusedColumns = dropUnusedColumns; this.cacheMaximumSize = cacheMaximumSize; this.cacheRefreshMs = cacheRefreshMs; this.inputSchemasPerTableCacheMaximumSize = inputSchemasPerTableCacheMaximumSize; @@ -66,7 +69,8 @@ public void open(OpenContext openContext) throws Exception { new TableUpdater( new TableMetadataCache( catalog, cacheMaximumSize, cacheRefreshMs, inputSchemasPerTableCacheMaximumSize), - catalog); + catalog, + dropUnusedColumns); } @Override diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java index ee0549997178..e106cf5754b3 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java @@ -21,10 +21,13 @@ import java.util.List; import org.apache.iceberg.Schema; import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.schema.SchemaWithPartnerVisitor; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Visitor class that accumulates the set of changes needed to evolve an existing schema into the @@ -36,30 +39,39 @@ *

  • Adding new columns *
  • Widening the type of existing columsn *
  • Reordering columns + *
  • Dropping columns (when dropUnusedColumns is enabled) * * * We don't support: * *
      - *
    • Dropping columns *
    • Renaming columns *
    * - * The reason is that dropping columns would create issues with late / out of order data. Once we - * drop fields, we wouldn't be able to easily add them back later without losing the associated - * data. Renaming columns is not supported because we compare schemas by name, which doesn't allow - * for renaming without additional hints. + * By default, any columns present in the table but absent from the input schema are marked as + * optional to prevent issues caused by late or out-of-order data. If dropUnusedColumns is enabled, + * these columns are removed instead to ensure a strict one-to-one schema alignment. */ public class EvolveSchemaVisitor extends SchemaWithPartnerVisitor { + private static final Logger LOG = LoggerFactory.getLogger(EvolveSchemaVisitor.class); + private final TableIdentifier identifier; private final UpdateSchema api; private final Schema existingSchema; private final Schema targetSchema; - - private EvolveSchemaVisitor(UpdateSchema api, Schema existingSchema, Schema targetSchema) { + private final boolean dropUnusedColumns; + + private EvolveSchemaVisitor( + TableIdentifier identifier, + UpdateSchema api, + Schema existingSchema, + Schema targetSchema, + boolean dropUnusedColumns) { + this.identifier = identifier; this.api = api; this.existingSchema = existingSchema; this.targetSchema = targetSchema; + this.dropUnusedColumns = dropUnusedColumns; } /** @@ -70,12 +82,18 @@ private EvolveSchemaVisitor(UpdateSchema api, Schema existingSchema, Schema targ * @param api an UpdateSchema for adding changes * @param existingSchema an existing schema * @param targetSchema a new schema to compare with the existing + * @param dropUnusedColumns whether to drop columns not present in target schema */ - public static void visit(UpdateSchema api, Schema existingSchema, Schema targetSchema) { + public static void visit( + TableIdentifier identifier, + UpdateSchema api, + Schema existingSchema, + Schema targetSchema, + boolean dropUnusedColumns) { visit( targetSchema, -1, - new EvolveSchemaVisitor(api, existingSchema, targetSchema), + new EvolveSchemaVisitor(identifier, api, existingSchema, targetSchema, dropUnusedColumns), new CompareSchemasVisitor.PartnerIdByNameAccessors(existingSchema)); } @@ -103,11 +121,16 @@ public Boolean struct(Types.StructType struct, Integer partnerId, List after = columnName; } - // Ensure that unused fields are made optional for (Types.NestedField existingField : partnerStruct.fields()) { if (struct.field(existingField.name()) == null) { - if (existingField.isRequired()) { - this.api.makeColumnOptional(this.existingSchema.findColumnName(existingField.fieldId())); + String columnName = this.existingSchema.findColumnName(existingField.fieldId()); + if (dropUnusedColumns) { + LOG.debug("{}: Dropping column: {}", identifier.name(), columnName); + this.api.deleteColumn(columnName); + } else { + if (existingField.isRequired()) { + this.api.makeColumnOptional(columnName); + } } } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java index 2c08a3486e7c..8a8362a41996 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java @@ -89,8 +89,8 @@ String branch(TableIdentifier identifier, String branch) { return branch(identifier, branch, true); } - ResolvedSchemaInfo schema(TableIdentifier identifier, Schema input) { - return schema(identifier, input, true); + ResolvedSchemaInfo schema(TableIdentifier identifier, Schema input, boolean dropUnusedColumns) { + return schema(identifier, input, true, dropUnusedColumns); } PartitionSpec spec(TableIdentifier identifier, PartitionSpec spec) { @@ -124,7 +124,7 @@ private String branch(TableIdentifier identifier, String branch, boolean allowRe } private ResolvedSchemaInfo schema( - TableIdentifier identifier, Schema input, boolean allowRefresh) { + TableIdentifier identifier, Schema input, boolean allowRefresh, boolean dropUnusedColumns) { CacheItem cached = tableCache.get(identifier); Schema compatible = null; if (cached != null && cached.tableExists) { @@ -139,7 +139,7 @@ private ResolvedSchemaInfo schema( for (Map.Entry tableSchema : cached.tableSchemas.entrySet()) { CompareSchemasVisitor.Result result = - CompareSchemasVisitor.visit(input, tableSchema.getValue(), true); + CompareSchemasVisitor.visit(input, tableSchema.getValue(), true, dropUnusedColumns); if (result == CompareSchemasVisitor.Result.SAME) { ResolvedSchemaInfo newResult = new ResolvedSchemaInfo( @@ -157,7 +157,7 @@ private ResolvedSchemaInfo schema( if (needsRefresh(cached, allowRefresh)) { refreshTable(identifier); - return schema(identifier, input, false); + return schema(identifier, input, false, dropUnusedColumns); } else if (compatible != null) { ResolvedSchemaInfo newResult = new ResolvedSchemaInfo( diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java index cadfe345980c..d8809efbe541 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java @@ -43,10 +43,12 @@ class TableUpdater { private static final Logger LOG = LoggerFactory.getLogger(TableUpdater.class); private final TableMetadataCache cache; private final Catalog catalog; + private final boolean dropUnusedColumns; - TableUpdater(TableMetadataCache cache, Catalog catalog) { + TableUpdater(TableMetadataCache cache, Catalog catalog, boolean dropUnusedColumns) { this.cache = cache; this.catalog = catalog; + this.dropUnusedColumns = dropUnusedColumns; } /** @@ -118,13 +120,15 @@ private void findOrCreateBranch(TableIdentifier identifier, String branch) { private TableMetadataCache.ResolvedSchemaInfo findOrCreateSchema( TableIdentifier identifier, Schema schema) { - TableMetadataCache.ResolvedSchemaInfo fromCache = cache.schema(identifier, schema); + TableMetadataCache.ResolvedSchemaInfo fromCache = + cache.schema(identifier, schema, dropUnusedColumns); if (fromCache.compareResult() != CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED) { return fromCache; } else { Table table = catalog.loadTable(identifier); Schema tableSchema = table.schema(); - CompareSchemasVisitor.Result result = CompareSchemasVisitor.visit(schema, tableSchema, true); + CompareSchemasVisitor.Result result = + CompareSchemasVisitor.visit(schema, tableSchema, true, dropUnusedColumns); switch (result) { case SAME: cache.update(identifier, table); @@ -141,19 +145,20 @@ private TableMetadataCache.ResolvedSchemaInfo findOrCreateSchema( LOG.info( "Triggering schema update for table {} {} to {}", identifier, tableSchema, schema); UpdateSchema updateApi = table.updateSchema(); - EvolveSchemaVisitor.visit(updateApi, tableSchema, schema); + EvolveSchemaVisitor.visit(identifier, updateApi, tableSchema, schema, dropUnusedColumns); try { updateApi.commit(); cache.update(identifier, table); TableMetadataCache.ResolvedSchemaInfo comparisonAfterMigration = - cache.schema(identifier, schema); + cache.schema(identifier, schema, dropUnusedColumns); Schema newSchema = comparisonAfterMigration.resolvedTableSchema(); LOG.info("Table {} schema updated from {} to {}", identifier, tableSchema, newSchema); return comparisonAfterMigration; } catch (CommitFailedException e) { cache.invalidate(identifier); - TableMetadataCache.ResolvedSchemaInfo newSchema = cache.schema(identifier, schema); + TableMetadataCache.ResolvedSchemaInfo newSchema = + cache.schema(identifier, schema, dropUnusedColumns); if (newSchema.compareResult() != CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED) { LOG.debug("Table {} schema updated concurrently to {}", identifier, schema); return newSchema; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java index 385a354889fb..cc8e6898d2ed 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java @@ -33,6 +33,9 @@ class TestCompareSchemasVisitor { + private static final boolean DROP_COLUMNS = true; + private static final boolean PRESERVE_COLUMNS = false; + @Test void testSchema() { assertThat( @@ -226,4 +229,63 @@ void testListChanged() { optional(2, "list1", ListType.ofOptional(3, IntegerType.get()))))) .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); } + + @Test + void testDropUnusedColumnsEnabled() { + Schema dataSchema = new Schema(optional(1, "id", IntegerType.get())); + Schema tableSchema = + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "data", StringType.get()), + optional(3, "extra", StringType.get())); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, DROP_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); + } + + @Test + void testDropUnusedColumnsWithRequiredField() { + Schema dataSchema = new Schema(optional(1, "id", IntegerType.get())); + Schema tableSchema = + new Schema(optional(1, "id", IntegerType.get()), required(2, "data", StringType.get())); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, DROP_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); + } + + @Test + void testDropUnusedColumnsWhenInputHasMoreFields() { + Schema dataSchema = + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "data", StringType.get()), + optional(3, "extra", StringType.get())); + Schema tableSchema = new Schema(optional(1, "id", IntegerType.get())); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, DROP_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); + } + + @Test + void testDropUnusedColumnsInNestedStruct() { + Schema dataSchema = + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "struct1", StructType.of(optional(3, "field1", StringType.get())))); + Schema tableSchema = + new Schema( + optional(1, "id", IntegerType.get()), + optional( + 2, + "struct1", + StructType.of( + optional(3, "field1", StringType.get()), + optional(4, "field2", IntegerType.get())))); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, DROP_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, PRESERVE_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.DATA_CONVERSION_NEEDED); + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java index b660d8e285d9..2711df72cab1 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java @@ -822,6 +822,55 @@ void testCommitsOnceWhenConcurrentDuplicateCommit(boolean overwriteMode) throws assertThat(totalAddedRecords).isEqualTo(records.size()); } + @Test + void testOptInDropUnusedColumns() throws Exception { + Schema schema1 = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "extra", Types.StringType.get())); + + Schema schema2 = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.of(DATABASE, "t1"); + catalog.createTable(tableIdentifier, schema1); + + List rows = + Lists.newArrayList( + // Drop columns + new DynamicIcebergDataImpl(schema2, "t1", "main", PartitionSpec.unpartitioned()), + // Re-add columns + new DynamicIcebergDataImpl(schema1, "t1", "main", PartitionSpec.unpartitioned())); + + DataStream dataStream = + env.fromData(rows, TypeInformation.of(new TypeHint<>() {})); + env.setParallelism(1); + + DynamicIcebergSink.forInput(dataStream) + .generator(new Generator()) + .catalogLoader(CATALOG_EXTENSION.catalogLoader()) + .immediateTableUpdate(true) + .dropUnusedColumns(true) + .append(); + + env.execute("Test Drop Unused Columns"); + + Table table = catalog.loadTable(tableIdentifier); + table.refresh(); + + assertThat(table.schema().columns()).hasSize(2); + assertThat(table.schema().findField("id")).isNotNull(); + assertThat(table.schema().findField("data")).isNotNull(); + assertThat(table.schema().findField("extra")).isNull(); + + List records = Lists.newArrayList(IcebergGenerics.read(table).build()); + assertThat(records).hasSize(2); + } + /** * Represents a concurrent duplicate commit during an ongoing commit operation, which can happen * in production scenarios when using REST catalog. diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java index 5ee766231b9d..5745d54c73ca 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java @@ -60,6 +60,7 @@ void testDynamicTableUpdateOperatorNewTable() throws Exception { DynamicTableUpdateOperator operator = new DynamicTableUpdateOperator( CATALOG_EXTENSION.catalogLoader(), + false, cacheMaximumSize, cacheRefreshMs, inputSchemaCacheMaximumSize, @@ -93,6 +94,7 @@ void testDynamicTableUpdateOperatorSchemaChange() throws Exception { DynamicTableUpdateOperator operator = new DynamicTableUpdateOperator( CATALOG_EXTENSION.catalogLoader(), + false, cacheMaximumSize, cacheRefreshMs, inputSchemaCacheMaximumSize, @@ -120,4 +122,84 @@ void testDynamicTableUpdateOperatorSchemaChange() throws Exception { assertThat(output2).isEqualTo(output); assertThat(catalog.loadTable(table).schema().schemaId()).isEqualTo(output.schema().schemaId()); } + + @Test + void testDynamicTableUpdateOperatorPreserveUnusedColumns() throws Exception { + int cacheMaximumSize = 10; + int cacheRefreshMs = 1000; + int inputSchemaCacheMaximumSize = 10; + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier table = TableIdentifier.of(TABLE); + + DynamicTableUpdateOperator operator = + new DynamicTableUpdateOperator( + CATALOG_EXTENSION.catalogLoader(), + false, // dropUnusedColumns = false (default) + cacheMaximumSize, + cacheRefreshMs, + inputSchemaCacheMaximumSize, + TableCreator.DEFAULT); + operator.open((OpenContext) null); + + catalog.createTable(table, SCHEMA2); + + DynamicRecordInternal input = + new DynamicRecordInternal( + TABLE, + "branch", + SCHEMA1, + GenericRowData.of(1), + PartitionSpec.unpartitioned(), + 42, + false, + Collections.emptySet()); + DynamicRecordInternal output = operator.map(input); + + Schema tableSchema = catalog.loadTable(table).schema(); + assertThat(tableSchema.columns()).hasSize(2); + assertThat(tableSchema.findField("id")).isNotNull(); + assertThat(tableSchema.findField("data")).isNotNull(); + assertThat(tableSchema.findField("data").isOptional()).isTrue(); + assertThat(input).isEqualTo(output); + } + + @Test + void testDynamicTableUpdateOperatorDropUnusedColumns() throws Exception { + int cacheMaximumSize = 10; + int cacheRefreshMs = 1000; + int inputSchemaCacheMaximumSize = 10; + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier table = TableIdentifier.of(TABLE); + + DynamicTableUpdateOperator operator = + new DynamicTableUpdateOperator( + CATALOG_EXTENSION.catalogLoader(), + // Drop unused columns + true, + cacheMaximumSize, + cacheRefreshMs, + inputSchemaCacheMaximumSize, + TableCreator.DEFAULT); + operator.open((OpenContext) null); + + catalog.createTable(table, SCHEMA2); + + DynamicRecordInternal input = + new DynamicRecordInternal( + TABLE, + "branch", + SCHEMA1, + GenericRowData.of(1), + PartitionSpec.unpartitioned(), + 42, + false, + Collections.emptySet()); + DynamicRecordInternal output = operator.map(input); + + Schema tableSchema = catalog.loadTable(table).schema(); + assertThat(tableSchema.columns()).hasSize(1); + assertThat(tableSchema.findField("id")).isNotNull(); + assertThat(tableSchema.findField("data")).isNull(); + assertThat(input).isEqualTo(output); + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java index d416e7ec1fc6..027adc4031bd 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java @@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.Schema; import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Type.PrimitiveType; import org.apache.iceberg.types.Types; @@ -48,6 +49,10 @@ public class TestEvolveSchemaVisitor { + private static final TableIdentifier TABLE = TableIdentifier.of("table"); + private static final boolean DROP_COLUMNS = true; + private static final boolean PRESERVE_COLUMNS = false; + private static List primitiveTypes() { return Lists.newArrayList( StringType.get(), @@ -89,7 +94,7 @@ private static Types.NestedField[] primitiveFields( public void testAddTopLevelPrimitives() { Schema targetSchema = new Schema(primitiveFields(0, primitiveTypes())); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(targetSchema.asStruct()).isEqualTo(updateApi.apply().asStruct()); } @@ -99,12 +104,59 @@ public void testMakeTopLevelPrimitivesOptional() { assertThat(existingSchema.columns().stream().allMatch(Types.NestedField::isRequired)).isTrue(); UpdateSchema updateApi = loadUpdateApi(existingSchema); - EvolveSchemaVisitor.visit(updateApi, existingSchema, new Schema()); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, new Schema(), PRESERVE_COLUMNS); Schema newSchema = updateApi.apply(); assertThat(newSchema.asStruct().fields()).hasSize(14); assertThat(newSchema.columns().stream().allMatch(Types.NestedField::isOptional)).isTrue(); } + @Test + public void testDropUnusedColumns() { + Schema existingSchema = + new Schema( + optional(1, "a", StringType.get()), + optional( + 2, + "b", + StructType.of( + optional(4, "nested1", StringType.get()), + optional(5, "nested2", StringType.get()))), + optional(3, "c", IntegerType.get())); + + Schema targetSchema = + new Schema( + optional(1, "a", StringType.get()), + optional(2, "b", StructType.of(optional(5, "nested2", StringType.get())))); + + UpdateSchema updateApi = loadUpdateApi(existingSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, DROP_COLUMNS); + + Schema newSchema = updateApi.apply(); + assertThat(newSchema.sameSchema(targetSchema)).isTrue(); + } + + @Test + public void testPreserveUnusedColumns() { + Schema existingSchema = + new Schema( + optional(1, "a", StringType.get()), + optional( + 2, + "b", + StructType.of( + optional(4, "nested1", StringType.get()), + optional(5, "nested2", StringType.get()))), + optional(3, "c", IntegerType.get())); + + Schema targetSchema = new Schema(optional(1, "a", StringType.get())); + + UpdateSchema updateApi = loadUpdateApi(existingSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, PRESERVE_COLUMNS); + + Schema newSchema = updateApi.apply(); + assertThat(newSchema.sameSchema(existingSchema)).isTrue(); + } + @Test public void testIdentifyFieldsByName() { Schema existingSchema = @@ -112,7 +164,7 @@ public void testIdentifyFieldsByName() { UpdateSchema updateApi = loadUpdateApi(existingSchema); Schema newSchema = new Schema(Arrays.asList(Types.NestedField.optional(-1, "myField", Types.LongType.get()))); - EvolveSchemaVisitor.visit(updateApi, existingSchema, newSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, newSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().sameSchema(existingSchema)).isTrue(); } @@ -125,7 +177,7 @@ public void testChangeOrderTopLevelPrimitives() { new Schema( Arrays.asList(optional(2, "b", StringType.get()), optional(1, "a", StringType.get()))); UpdateSchema updateApi = loadUpdateApi(existingSchema); - EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -134,7 +186,7 @@ public void testAddTopLevelListOfPrimitives() { for (PrimitiveType primitiveType : primitiveTypes()) { Schema targetSchema = new Schema(optional(1, "aList", ListType.ofOptional(2, primitiveType))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } } @@ -146,7 +198,7 @@ public void testMakeTopLevelListOfPrimitivesOptional() { new Schema(optional(1, "aList", ListType.ofRequired(2, primitiveType))); Schema targetSchema = new Schema(); UpdateSchema updateApi = loadUpdateApi(existingSchema); - EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, PRESERVE_COLUMNS); Schema expectedSchema = new Schema(optional(1, "aList", ListType.ofRequired(2, primitiveType))); assertThat(updateApi.apply().asStruct()).isEqualTo(expectedSchema.asStruct()); @@ -159,7 +211,7 @@ public void testAddTopLevelMapOfPrimitives() { Schema targetSchema = new Schema(optional(1, "aMap", MapType.ofOptional(2, 3, primitiveType, primitiveType))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } } @@ -171,7 +223,7 @@ public void testAddTopLevelStructOfPrimitives() { new Schema( optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), currentSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), currentSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(currentSchema.asStruct()); } } @@ -184,7 +236,7 @@ public void testAddNestedPrimitive() { new Schema( optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } } @@ -199,7 +251,7 @@ public void testMakeNestedPrimitiveOptional() { new Schema( optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } } @@ -210,7 +262,7 @@ public void testAddNestedPrimitives() { Schema targetSchema = new Schema(optional(1, "aStruct", StructType.of(primitiveFields(1, primitiveTypes())))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -240,7 +292,7 @@ public void testAddNestedLists() { ListType.ofOptional( 10, DecimalType.of(11, 20)))))))))))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -279,7 +331,7 @@ public void testAddNestedStruct() { "aString", StringType.get())))))))))))))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -314,7 +366,7 @@ public void testAddNestedMaps() { 12, 13, StringType.get(), StringType.get())))))))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -326,7 +378,11 @@ public void testDetectInvalidTopLevelList() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aList.element: string -> long") .isInstanceOf(IllegalArgumentException.class); } @@ -343,7 +399,11 @@ public void testDetectInvalidTopLevelMapValue() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aMap.value: string -> long") .isInstanceOf(IllegalArgumentException.class); } @@ -358,7 +418,11 @@ public void testDetectInvalidTopLevelMapKey() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aMap.key: string -> uuid") .isInstanceOf(IllegalArgumentException.class); } @@ -370,7 +434,7 @@ public void testTypePromoteIntegerToLong() { Schema targetSchema = new Schema(required(1, "aCol", LongType.get())); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); Schema applied = updateApi.apply(); assertThat(applied.asStruct().fields()).hasSize(1); assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(LongType.get()); @@ -383,7 +447,7 @@ public void testTypePromoteFloatToDouble() { Schema targetSchema = new Schema(required(1, "aCol", DoubleType.get())); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); Schema applied = updateApi.apply(); assertThat(applied.asStruct().fields()).hasSize(1); assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(DoubleType.get()); @@ -396,7 +460,11 @@ public void testInvalidTypePromoteDoubleToFloat() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aCol: double -> float") .isInstanceOf(IllegalArgumentException.class); } @@ -409,7 +477,7 @@ public void testTypePromoteDecimalToFixedScaleWithWiderPrecision() { Schema targetSchema = new Schema(required(1, "aCol", DecimalType.of(22, 1))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -452,7 +520,7 @@ public void testAddPrimitiveToNestedStruct() { optional(6, "time", TimeType.get()))))))))); UpdateSchema updateApi = loadUpdateApi(existingSchema); - EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -464,7 +532,11 @@ public void testReplaceListWithPrimitive() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aColumn: list -> string") .isInstanceOf(IllegalArgumentException.class); } @@ -501,7 +573,7 @@ public void addNewTopLevelStruct() { optional(7, "d1", StructType.of(optional(8, "d2", StringType.get())))))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -553,7 +625,7 @@ public void testAppendNestedStruct() { StringType.get())))))))))))))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -573,7 +645,7 @@ public void testMakeNestedStructOptional() { optional( 3, "s3", StructType.of(optional(4, "s4", StringType.get())))))))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(getNestedSchemaWithOptionalModifier(true).asStruct()) .isEqualTo(updateApi.apply().asStruct()); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java index bf5b9f562f9a..7f91d2f8d585 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java @@ -53,17 +53,22 @@ void testCaching() { catalog.createTable(tableIdentifier, SCHEMA); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - Schema schema1 = cache.schema(tableIdentifier, SCHEMA).resolvedTableSchema(); + Schema schema1 = cache.schema(tableIdentifier, SCHEMA, false).resolvedTableSchema(); assertThat(schema1.sameSchema(SCHEMA)).isTrue(); assertThat( - cache.schema(tableIdentifier, SerializationUtils.clone(SCHEMA)).resolvedTableSchema()) + cache + .schema(tableIdentifier, SerializationUtils.clone(SCHEMA), false) + .resolvedTableSchema()) .isEqualTo(schema1); - assertThat(cache.schema(tableIdentifier, SCHEMA2)).isEqualTo(TableMetadataCache.NOT_FOUND); + assertThat(cache.schema(tableIdentifier, SCHEMA2, false)) + .isEqualTo(TableMetadataCache.NOT_FOUND); - schema1 = cache.schema(tableIdentifier, SCHEMA).resolvedTableSchema(); + schema1 = cache.schema(tableIdentifier, SCHEMA, false).resolvedTableSchema(); assertThat( - cache.schema(tableIdentifier, SerializationUtils.clone(SCHEMA)).resolvedTableSchema()) + cache + .schema(tableIdentifier, SerializationUtils.clone(SCHEMA), false) + .resolvedTableSchema()) .isEqualTo(schema1); } @@ -73,9 +78,9 @@ void testCacheInvalidationAfterSchemaChange() { TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); catalog.createTable(tableIdentifier, SCHEMA); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); - Schema schema1 = cache.schema(tableIdentifier, SCHEMA).resolvedTableSchema(); + Schema schema1 = cache.schema(tableIdentifier, SCHEMA, false).resolvedTableSchema(); assertThat(schema1.sameSchema(SCHEMA)).isTrue(); catalog.dropTable(tableIdentifier); @@ -83,7 +88,7 @@ void testCacheInvalidationAfterSchemaChange() { tableUpdater.update( tableIdentifier, "main", SCHEMA2, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); - Schema schema2 = cache.schema(tableIdentifier, SCHEMA2).resolvedTableSchema(); + Schema schema2 = cache.schema(tableIdentifier, SCHEMA2, false).resolvedTableSchema(); assertThat(schema2.sameSchema(SCHEMA2)).isTrue(); } @@ -111,11 +116,11 @@ void testNoCacheRefreshingBeforeRefreshIntervalElapses() { cache.update(tableIdentifier, table); // Cache schema - Schema schema = cache.schema(tableIdentifier, SCHEMA2).resolvedTableSchema(); + Schema schema = cache.schema(tableIdentifier, SCHEMA2, false).resolvedTableSchema(); assertThat(schema.sameSchema(SCHEMA2)).isTrue(); // Cache schema with fewer fields - TableMetadataCache.ResolvedSchemaInfo schemaInfo = cache.schema(tableIdentifier, SCHEMA); + TableMetadataCache.ResolvedSchemaInfo schemaInfo = cache.schema(tableIdentifier, SCHEMA, false); assertThat(schemaInfo.resolvedTableSchema().sameSchema(SCHEMA2)).isTrue(); assertThat(schemaInfo.compareResult()) .isEqualTo(CompareSchemasVisitor.Result.DATA_CONVERSION_NEEDED); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java index 1d4461698746..bcc5d8064517 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java @@ -56,7 +56,7 @@ void testTableCreation(@TempDir Path tempDir) { catalog.createNamespace(Namespace.of("myNamespace")); TableIdentifier tableIdentifier = TableIdentifier.parse("myNamespace.myTable"); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); String locationOverride = tempDir.toString() + "/custom-path"; Map tableProperties = Map.of("key", "value"); @@ -69,7 +69,8 @@ void testTableCreation(@TempDir Path tempDir) { assertThat(catalog.tableExists(tableIdentifier)).isTrue(); assertThat(catalog.loadTable(tableIdentifier).properties().get("key")).isEqualTo("value"); assertThat(catalog.loadTable(tableIdentifier).location()).isEqualTo(locationOverride); - TableMetadataCache.ResolvedSchemaInfo cachedSchema = cache.schema(tableIdentifier, SCHEMA); + TableMetadataCache.ResolvedSchemaInfo cachedSchema = + cache.schema(tableIdentifier, SCHEMA, false); assertThat(cachedSchema.resolvedTableSchema().sameSchema(SCHEMA)).isTrue(); } @@ -78,7 +79,7 @@ void testTableAlreadyExists() { Catalog catalog = CATALOG_EXTENSION.catalog(); TableIdentifier tableIdentifier = TableIdentifier.parse("myTable"); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); // Make the table non-existent in cache cache.exists(tableIdentifier); @@ -98,7 +99,7 @@ void testBranchCreationAndCaching() { Catalog catalog = CATALOG_EXTENSION.catalog(); TableIdentifier tableIdentifier = TableIdentifier.parse("myTable"); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); catalog.createTable(tableIdentifier, SCHEMA); tableUpdater.update( @@ -116,7 +117,7 @@ void testSpecCreation() { Catalog catalog = CATALOG_EXTENSION.catalog(); TableIdentifier tableIdentifier = TableIdentifier.parse("myTable"); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).bucket("data", 10).build(); tableUpdater.update(tableIdentifier, "main", SCHEMA, spec, TableCreator.DEFAULT); @@ -132,8 +133,8 @@ void testInvalidateOldCacheEntryOnUpdate() { TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); catalog.createTable(tableIdentifier, SCHEMA); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - cache.schema(tableIdentifier, SCHEMA); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + cache.schema(tableIdentifier, SCHEMA, false); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); Schema updated = tableUpdater @@ -146,7 +147,8 @@ void testInvalidateOldCacheEntryOnUpdate() { .f0 .resolvedTableSchema(); assertThat(updated.sameSchema(SCHEMA2)).isTrue(); - assertThat(cache.schema(tableIdentifier, SCHEMA2).resolvedTableSchema().sameSchema(SCHEMA2)) + assertThat( + cache.schema(tableIdentifier, SCHEMA2, false).resolvedTableSchema().sameSchema(SCHEMA2)) .isTrue(); } @@ -156,7 +158,7 @@ void testLastResultInvalidation() { TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); catalog.createTable(tableIdentifier, SCHEMA); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); // Initialize cache tableUpdater.update( @@ -167,7 +169,7 @@ void testLastResultInvalidation() { catalog.createTable(tableIdentifier, SCHEMA2); // Cache still stores the old information - assertThat(cache.schema(tableIdentifier, SCHEMA2).compareResult()) + assertThat(cache.schema(tableIdentifier, SCHEMA2, false).compareResult()) .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); assertThat( @@ -186,4 +188,27 @@ void testLastResultInvalidation() { assertThat(cache.getInternalCache().get(tableIdentifier).inputSchemas()) .doesNotContainKey(SCHEMA2); } + + @Test + void testDropUnusedColumns() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("myTable"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + + final boolean dropUnusedColumns = true; + TableUpdater tableUpdater = new TableUpdater(cache, catalog, dropUnusedColumns); + + catalog.createTable(tableIdentifier, SCHEMA2); + + Tuple2 result = + tableUpdater.update( + tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + + assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); + Schema tableSchema = catalog.loadTable(tableIdentifier).schema(); + assertThat(tableSchema.columns()).hasSize(2); + assertThat(tableSchema.findField("id")).isNotNull(); + assertThat(tableSchema.findField("data")).isNotNull(); + assertThat(tableSchema.findField("extra")).isNull(); + } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java index 41ffa609540b..60561b0f56bf 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/CompareSchemasVisitor.java @@ -43,20 +43,23 @@ public class CompareSchemasVisitor extends SchemaWithPartnerVisitor { private final Schema tableSchema; + private final boolean dropUnusedColumns; - private CompareSchemasVisitor(Schema tableSchema) { + private CompareSchemasVisitor(Schema tableSchema, boolean dropUnusedColumns) { this.tableSchema = tableSchema; + this.dropUnusedColumns = dropUnusedColumns; } public static Result visit(Schema dataSchema, Schema tableSchema) { - return visit(dataSchema, tableSchema, true); + return visit(dataSchema, tableSchema, true, false); } - public static Result visit(Schema dataSchema, Schema tableSchema, boolean caseSensitive) { + public static Result visit( + Schema dataSchema, Schema tableSchema, boolean caseSensitive, boolean dropUnusedColumns) { return visit( dataSchema, -1, - new CompareSchemasVisitor(tableSchema), + new CompareSchemasVisitor(tableSchema, dropUnusedColumns), new PartnerIdByNameAccessors(tableSchema, caseSensitive)); } @@ -70,6 +73,7 @@ public Result schema(Schema dataSchema, Integer tableSchemaId, Result downstream } @Override + @SuppressWarnings("CyclomaticComplexity") public Result struct(Types.StructType struct, Integer tableSchemaId, List fields) { if (tableSchemaId == null) { return Result.SCHEMA_UPDATE_NEEDED; @@ -88,10 +92,10 @@ public Result struct(Types.StructType struct, Integer tableSchemaId, List { private ReadableConfig readableConfig = new Configuration(); private TableCreator tableCreator = TableCreator.DEFAULT; private boolean immediateUpdate = false; + private boolean dropUnusedColumns = false; private int cacheMaximumSize = 100; private long cacheRefreshMs = 1_000; private int inputSchemasPerTableCacheMaximumSize = 10; @@ -314,6 +315,22 @@ public Builder immediateTableUpdate(boolean newImmediateUpdate) { return this; } + /** + * Dropping columns is disabled by default to prevent issues with late or out-of-order data, as + * removed fields cannot be easily restored without data loss. + * + *

    You can opt-in to allow dropping columns. Once a column has been dropped, it is + * technically still possible to write data to that column because Iceberg maintains all past + * table schemas. However, regular queries won't be able to reference the column. If the field + * was to re-appear as part of a new schema, an entirely new column would be added, which apart + * from the name, has nothing in common with the old column, i.e. queries for the new column + * will never return data of the old column. + */ + public Builder dropUnusedColumns(boolean newDropUnusedColumns) { + this.dropUnusedColumns = newDropUnusedColumns; + return this; + } + /** Maximum size of the caches used in Dynamic Sink for table data and serializers. */ public Builder cacheMaxSize(int maxSize) { this.cacheMaximumSize = maxSize; @@ -382,6 +399,7 @@ public DataStreamSink append() { generator, catalogLoader, immediateUpdate, + dropUnusedColumns, cacheMaximumSize, cacheRefreshMs, inputSchemasPerTableCacheMaximumSize, @@ -400,6 +418,7 @@ public DataStreamSink append() { .map( new DynamicTableUpdateOperator( catalogLoader, + dropUnusedColumns, cacheMaximumSize, cacheRefreshMs, inputSchemasPerTableCacheMaximumSize, diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java index bc3a25468d84..427aa6ceafba 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordProcessor.java @@ -40,6 +40,7 @@ class DynamicRecordProcessor extends ProcessFunction generator; private final CatalogLoader catalogLoader; private final boolean immediateUpdate; + private final boolean dropUnusedColumns; private final int cacheMaximumSize; private final long cacheRefreshMs; private final int inputSchemasPerTableCacheMaximumSize; @@ -56,6 +57,7 @@ class DynamicRecordProcessor extends ProcessFunction generator, CatalogLoader catalogLoader, boolean immediateUpdate, + boolean dropUnusedColumns, int cacheMaximumSize, long cacheRefreshMs, int inputSchemasPerTableCacheMaximumSize, @@ -63,6 +65,7 @@ class DynamicRecordProcessor extends ProcessFunction( @@ -106,7 +109,7 @@ public void collect(DynamicRecord data) { TableMetadataCache.ResolvedSchemaInfo foundSchema = exists - ? tableCache.schema(data.tableIdentifier(), data.schema()) + ? tableCache.schema(data.tableIdentifier(), data.schema(), dropUnusedColumns) : TableMetadataCache.NOT_FOUND; PartitionSpec foundSpec = exists ? tableCache.spec(data.tableIdentifier(), data.spec()) : null; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java index 586239b54bca..8f38d4f8be0c 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicTableUpdateOperator.java @@ -38,6 +38,7 @@ class DynamicTableUpdateOperator extends RichMapFunction { private final CatalogLoader catalogLoader; + private final boolean dropUnusedColumns; private final int cacheMaximumSize; private final long cacheRefreshMs; private final int inputSchemasPerTableCacheMaximumSize; @@ -47,11 +48,13 @@ class DynamicTableUpdateOperator DynamicTableUpdateOperator( CatalogLoader catalogLoader, + boolean dropUnusedColumns, int cacheMaximumSize, long cacheRefreshMs, int inputSchemasPerTableCacheMaximumSize, TableCreator tableCreator) { this.catalogLoader = catalogLoader; + this.dropUnusedColumns = dropUnusedColumns; this.cacheMaximumSize = cacheMaximumSize; this.cacheRefreshMs = cacheRefreshMs; this.inputSchemasPerTableCacheMaximumSize = inputSchemasPerTableCacheMaximumSize; @@ -66,7 +69,8 @@ public void open(OpenContext openContext) throws Exception { new TableUpdater( new TableMetadataCache( catalog, cacheMaximumSize, cacheRefreshMs, inputSchemasPerTableCacheMaximumSize), - catalog); + catalog, + dropUnusedColumns); } @Override diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java index ee0549997178..e106cf5754b3 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/EvolveSchemaVisitor.java @@ -21,10 +21,13 @@ import java.util.List; import org.apache.iceberg.Schema; import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.schema.SchemaWithPartnerVisitor; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Visitor class that accumulates the set of changes needed to evolve an existing schema into the @@ -36,30 +39,39 @@ *

  • Adding new columns *
  • Widening the type of existing columsn *
  • Reordering columns + *
  • Dropping columns (when dropUnusedColumns is enabled) * * * We don't support: * *
      - *
    • Dropping columns *
    • Renaming columns *
    * - * The reason is that dropping columns would create issues with late / out of order data. Once we - * drop fields, we wouldn't be able to easily add them back later without losing the associated - * data. Renaming columns is not supported because we compare schemas by name, which doesn't allow - * for renaming without additional hints. + * By default, any columns present in the table but absent from the input schema are marked as + * optional to prevent issues caused by late or out-of-order data. If dropUnusedColumns is enabled, + * these columns are removed instead to ensure a strict one-to-one schema alignment. */ public class EvolveSchemaVisitor extends SchemaWithPartnerVisitor { + private static final Logger LOG = LoggerFactory.getLogger(EvolveSchemaVisitor.class); + private final TableIdentifier identifier; private final UpdateSchema api; private final Schema existingSchema; private final Schema targetSchema; - - private EvolveSchemaVisitor(UpdateSchema api, Schema existingSchema, Schema targetSchema) { + private final boolean dropUnusedColumns; + + private EvolveSchemaVisitor( + TableIdentifier identifier, + UpdateSchema api, + Schema existingSchema, + Schema targetSchema, + boolean dropUnusedColumns) { + this.identifier = identifier; this.api = api; this.existingSchema = existingSchema; this.targetSchema = targetSchema; + this.dropUnusedColumns = dropUnusedColumns; } /** @@ -70,12 +82,18 @@ private EvolveSchemaVisitor(UpdateSchema api, Schema existingSchema, Schema targ * @param api an UpdateSchema for adding changes * @param existingSchema an existing schema * @param targetSchema a new schema to compare with the existing + * @param dropUnusedColumns whether to drop columns not present in target schema */ - public static void visit(UpdateSchema api, Schema existingSchema, Schema targetSchema) { + public static void visit( + TableIdentifier identifier, + UpdateSchema api, + Schema existingSchema, + Schema targetSchema, + boolean dropUnusedColumns) { visit( targetSchema, -1, - new EvolveSchemaVisitor(api, existingSchema, targetSchema), + new EvolveSchemaVisitor(identifier, api, existingSchema, targetSchema, dropUnusedColumns), new CompareSchemasVisitor.PartnerIdByNameAccessors(existingSchema)); } @@ -103,11 +121,16 @@ public Boolean struct(Types.StructType struct, Integer partnerId, List after = columnName; } - // Ensure that unused fields are made optional for (Types.NestedField existingField : partnerStruct.fields()) { if (struct.field(existingField.name()) == null) { - if (existingField.isRequired()) { - this.api.makeColumnOptional(this.existingSchema.findColumnName(existingField.fieldId())); + String columnName = this.existingSchema.findColumnName(existingField.fieldId()); + if (dropUnusedColumns) { + LOG.debug("{}: Dropping column: {}", identifier.name(), columnName); + this.api.deleteColumn(columnName); + } else { + if (existingField.isRequired()) { + this.api.makeColumnOptional(columnName); + } } } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java index 2c08a3486e7c..8a8362a41996 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java @@ -89,8 +89,8 @@ String branch(TableIdentifier identifier, String branch) { return branch(identifier, branch, true); } - ResolvedSchemaInfo schema(TableIdentifier identifier, Schema input) { - return schema(identifier, input, true); + ResolvedSchemaInfo schema(TableIdentifier identifier, Schema input, boolean dropUnusedColumns) { + return schema(identifier, input, true, dropUnusedColumns); } PartitionSpec spec(TableIdentifier identifier, PartitionSpec spec) { @@ -124,7 +124,7 @@ private String branch(TableIdentifier identifier, String branch, boolean allowRe } private ResolvedSchemaInfo schema( - TableIdentifier identifier, Schema input, boolean allowRefresh) { + TableIdentifier identifier, Schema input, boolean allowRefresh, boolean dropUnusedColumns) { CacheItem cached = tableCache.get(identifier); Schema compatible = null; if (cached != null && cached.tableExists) { @@ -139,7 +139,7 @@ private ResolvedSchemaInfo schema( for (Map.Entry tableSchema : cached.tableSchemas.entrySet()) { CompareSchemasVisitor.Result result = - CompareSchemasVisitor.visit(input, tableSchema.getValue(), true); + CompareSchemasVisitor.visit(input, tableSchema.getValue(), true, dropUnusedColumns); if (result == CompareSchemasVisitor.Result.SAME) { ResolvedSchemaInfo newResult = new ResolvedSchemaInfo( @@ -157,7 +157,7 @@ private ResolvedSchemaInfo schema( if (needsRefresh(cached, allowRefresh)) { refreshTable(identifier); - return schema(identifier, input, false); + return schema(identifier, input, false, dropUnusedColumns); } else if (compatible != null) { ResolvedSchemaInfo newResult = new ResolvedSchemaInfo( diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java index cadfe345980c..d8809efbe541 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableUpdater.java @@ -43,10 +43,12 @@ class TableUpdater { private static final Logger LOG = LoggerFactory.getLogger(TableUpdater.class); private final TableMetadataCache cache; private final Catalog catalog; + private final boolean dropUnusedColumns; - TableUpdater(TableMetadataCache cache, Catalog catalog) { + TableUpdater(TableMetadataCache cache, Catalog catalog, boolean dropUnusedColumns) { this.cache = cache; this.catalog = catalog; + this.dropUnusedColumns = dropUnusedColumns; } /** @@ -118,13 +120,15 @@ private void findOrCreateBranch(TableIdentifier identifier, String branch) { private TableMetadataCache.ResolvedSchemaInfo findOrCreateSchema( TableIdentifier identifier, Schema schema) { - TableMetadataCache.ResolvedSchemaInfo fromCache = cache.schema(identifier, schema); + TableMetadataCache.ResolvedSchemaInfo fromCache = + cache.schema(identifier, schema, dropUnusedColumns); if (fromCache.compareResult() != CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED) { return fromCache; } else { Table table = catalog.loadTable(identifier); Schema tableSchema = table.schema(); - CompareSchemasVisitor.Result result = CompareSchemasVisitor.visit(schema, tableSchema, true); + CompareSchemasVisitor.Result result = + CompareSchemasVisitor.visit(schema, tableSchema, true, dropUnusedColumns); switch (result) { case SAME: cache.update(identifier, table); @@ -141,19 +145,20 @@ private TableMetadataCache.ResolvedSchemaInfo findOrCreateSchema( LOG.info( "Triggering schema update for table {} {} to {}", identifier, tableSchema, schema); UpdateSchema updateApi = table.updateSchema(); - EvolveSchemaVisitor.visit(updateApi, tableSchema, schema); + EvolveSchemaVisitor.visit(identifier, updateApi, tableSchema, schema, dropUnusedColumns); try { updateApi.commit(); cache.update(identifier, table); TableMetadataCache.ResolvedSchemaInfo comparisonAfterMigration = - cache.schema(identifier, schema); + cache.schema(identifier, schema, dropUnusedColumns); Schema newSchema = comparisonAfterMigration.resolvedTableSchema(); LOG.info("Table {} schema updated from {} to {}", identifier, tableSchema, newSchema); return comparisonAfterMigration; } catch (CommitFailedException e) { cache.invalidate(identifier); - TableMetadataCache.ResolvedSchemaInfo newSchema = cache.schema(identifier, schema); + TableMetadataCache.ResolvedSchemaInfo newSchema = + cache.schema(identifier, schema, dropUnusedColumns); if (newSchema.compareResult() != CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED) { LOG.debug("Table {} schema updated concurrently to {}", identifier, schema); return newSchema; diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java index 385a354889fb..cc8e6898d2ed 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestCompareSchemasVisitor.java @@ -33,6 +33,9 @@ class TestCompareSchemasVisitor { + private static final boolean DROP_COLUMNS = true; + private static final boolean PRESERVE_COLUMNS = false; + @Test void testSchema() { assertThat( @@ -226,4 +229,63 @@ void testListChanged() { optional(2, "list1", ListType.ofOptional(3, IntegerType.get()))))) .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); } + + @Test + void testDropUnusedColumnsEnabled() { + Schema dataSchema = new Schema(optional(1, "id", IntegerType.get())); + Schema tableSchema = + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "data", StringType.get()), + optional(3, "extra", StringType.get())); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, DROP_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); + } + + @Test + void testDropUnusedColumnsWithRequiredField() { + Schema dataSchema = new Schema(optional(1, "id", IntegerType.get())); + Schema tableSchema = + new Schema(optional(1, "id", IntegerType.get()), required(2, "data", StringType.get())); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, DROP_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); + } + + @Test + void testDropUnusedColumnsWhenInputHasMoreFields() { + Schema dataSchema = + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "data", StringType.get()), + optional(3, "extra", StringType.get())); + Schema tableSchema = new Schema(optional(1, "id", IntegerType.get())); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, DROP_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); + } + + @Test + void testDropUnusedColumnsInNestedStruct() { + Schema dataSchema = + new Schema( + optional(1, "id", IntegerType.get()), + optional(2, "struct1", StructType.of(optional(3, "field1", StringType.get())))); + Schema tableSchema = + new Schema( + optional(1, "id", IntegerType.get()), + optional( + 2, + "struct1", + StructType.of( + optional(3, "field1", StringType.get()), + optional(4, "field2", IntegerType.get())))); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, DROP_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); + + assertThat(CompareSchemasVisitor.visit(dataSchema, tableSchema, true, PRESERVE_COLUMNS)) + .isEqualTo(CompareSchemasVisitor.Result.DATA_CONVERSION_NEEDED); + } } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java index b660d8e285d9..2711df72cab1 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java @@ -822,6 +822,55 @@ void testCommitsOnceWhenConcurrentDuplicateCommit(boolean overwriteMode) throws assertThat(totalAddedRecords).isEqualTo(records.size()); } + @Test + void testOptInDropUnusedColumns() throws Exception { + Schema schema1 = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "extra", Types.StringType.get())); + + Schema schema2 = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.of(DATABASE, "t1"); + catalog.createTable(tableIdentifier, schema1); + + List rows = + Lists.newArrayList( + // Drop columns + new DynamicIcebergDataImpl(schema2, "t1", "main", PartitionSpec.unpartitioned()), + // Re-add columns + new DynamicIcebergDataImpl(schema1, "t1", "main", PartitionSpec.unpartitioned())); + + DataStream dataStream = + env.fromData(rows, TypeInformation.of(new TypeHint<>() {})); + env.setParallelism(1); + + DynamicIcebergSink.forInput(dataStream) + .generator(new Generator()) + .catalogLoader(CATALOG_EXTENSION.catalogLoader()) + .immediateTableUpdate(true) + .dropUnusedColumns(true) + .append(); + + env.execute("Test Drop Unused Columns"); + + Table table = catalog.loadTable(tableIdentifier); + table.refresh(); + + assertThat(table.schema().columns()).hasSize(2); + assertThat(table.schema().findField("id")).isNotNull(); + assertThat(table.schema().findField("data")).isNotNull(); + assertThat(table.schema().findField("extra")).isNull(); + + List records = Lists.newArrayList(IcebergGenerics.read(table).build()); + assertThat(records).hasSize(2); + } + /** * Represents a concurrent duplicate commit during an ongoing commit operation, which can happen * in production scenarios when using REST catalog. diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java index 22655ff99f86..d68dd58c08fc 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicTableUpdateOperator.java @@ -59,6 +59,7 @@ void testDynamicTableUpdateOperatorNewTable() throws Exception { DynamicTableUpdateOperator operator = new DynamicTableUpdateOperator( CATALOG_EXTENSION.catalogLoader(), + false, cacheMaximumSize, cacheRefreshMs, inputSchemaCacheMaximumSize, @@ -92,6 +93,7 @@ void testDynamicTableUpdateOperatorSchemaChange() throws Exception { DynamicTableUpdateOperator operator = new DynamicTableUpdateOperator( CATALOG_EXTENSION.catalogLoader(), + false, cacheMaximumSize, cacheRefreshMs, inputSchemaCacheMaximumSize, @@ -119,4 +121,84 @@ void testDynamicTableUpdateOperatorSchemaChange() throws Exception { assertThat(output2).isEqualTo(output); assertThat(catalog.loadTable(table).schema().schemaId()).isEqualTo(output.schema().schemaId()); } + + @Test + void testDynamicTableUpdateOperatorPreserveUnusedColumns() throws Exception { + int cacheMaximumSize = 10; + int cacheRefreshMs = 1000; + int inputSchemaCacheMaximumSize = 10; + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier table = TableIdentifier.of(TABLE); + + DynamicTableUpdateOperator operator = + new DynamicTableUpdateOperator( + CATALOG_EXTENSION.catalogLoader(), + false, // dropUnusedColumns = false (default) + cacheMaximumSize, + cacheRefreshMs, + inputSchemaCacheMaximumSize, + TableCreator.DEFAULT); + operator.open(null); + + catalog.createTable(table, SCHEMA2); + + DynamicRecordInternal input = + new DynamicRecordInternal( + TABLE, + "branch", + SCHEMA1, + GenericRowData.of(1), + PartitionSpec.unpartitioned(), + 42, + false, + Collections.emptySet()); + DynamicRecordInternal output = operator.map(input); + + Schema tableSchema = catalog.loadTable(table).schema(); + assertThat(tableSchema.columns()).hasSize(2); + assertThat(tableSchema.findField("id")).isNotNull(); + assertThat(tableSchema.findField("data")).isNotNull(); + assertThat(tableSchema.findField("data").isOptional()).isTrue(); + assertThat(input).isEqualTo(output); + } + + @Test + void testDynamicTableUpdateOperatorDropUnusedColumns() throws Exception { + int cacheMaximumSize = 10; + int cacheRefreshMs = 1000; + int inputSchemaCacheMaximumSize = 10; + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier table = TableIdentifier.of(TABLE); + + DynamicTableUpdateOperator operator = + new DynamicTableUpdateOperator( + CATALOG_EXTENSION.catalogLoader(), + // Drop unused columns + true, + cacheMaximumSize, + cacheRefreshMs, + inputSchemaCacheMaximumSize, + TableCreator.DEFAULT); + operator.open(null); + + catalog.createTable(table, SCHEMA2); + + DynamicRecordInternal input = + new DynamicRecordInternal( + TABLE, + "branch", + SCHEMA1, + GenericRowData.of(1), + PartitionSpec.unpartitioned(), + 42, + false, + Collections.emptySet()); + DynamicRecordInternal output = operator.map(input); + + Schema tableSchema = catalog.loadTable(table).schema(); + assertThat(tableSchema.columns()).hasSize(1); + assertThat(tableSchema.findField("id")).isNotNull(); + assertThat(tableSchema.findField("data")).isNull(); + assertThat(input).isEqualTo(output); + } } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java index d416e7ec1fc6..027adc4031bd 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestEvolveSchemaVisitor.java @@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.Schema; import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Type.PrimitiveType; import org.apache.iceberg.types.Types; @@ -48,6 +49,10 @@ public class TestEvolveSchemaVisitor { + private static final TableIdentifier TABLE = TableIdentifier.of("table"); + private static final boolean DROP_COLUMNS = true; + private static final boolean PRESERVE_COLUMNS = false; + private static List primitiveTypes() { return Lists.newArrayList( StringType.get(), @@ -89,7 +94,7 @@ private static Types.NestedField[] primitiveFields( public void testAddTopLevelPrimitives() { Schema targetSchema = new Schema(primitiveFields(0, primitiveTypes())); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(targetSchema.asStruct()).isEqualTo(updateApi.apply().asStruct()); } @@ -99,12 +104,59 @@ public void testMakeTopLevelPrimitivesOptional() { assertThat(existingSchema.columns().stream().allMatch(Types.NestedField::isRequired)).isTrue(); UpdateSchema updateApi = loadUpdateApi(existingSchema); - EvolveSchemaVisitor.visit(updateApi, existingSchema, new Schema()); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, new Schema(), PRESERVE_COLUMNS); Schema newSchema = updateApi.apply(); assertThat(newSchema.asStruct().fields()).hasSize(14); assertThat(newSchema.columns().stream().allMatch(Types.NestedField::isOptional)).isTrue(); } + @Test + public void testDropUnusedColumns() { + Schema existingSchema = + new Schema( + optional(1, "a", StringType.get()), + optional( + 2, + "b", + StructType.of( + optional(4, "nested1", StringType.get()), + optional(5, "nested2", StringType.get()))), + optional(3, "c", IntegerType.get())); + + Schema targetSchema = + new Schema( + optional(1, "a", StringType.get()), + optional(2, "b", StructType.of(optional(5, "nested2", StringType.get())))); + + UpdateSchema updateApi = loadUpdateApi(existingSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, DROP_COLUMNS); + + Schema newSchema = updateApi.apply(); + assertThat(newSchema.sameSchema(targetSchema)).isTrue(); + } + + @Test + public void testPreserveUnusedColumns() { + Schema existingSchema = + new Schema( + optional(1, "a", StringType.get()), + optional( + 2, + "b", + StructType.of( + optional(4, "nested1", StringType.get()), + optional(5, "nested2", StringType.get()))), + optional(3, "c", IntegerType.get())); + + Schema targetSchema = new Schema(optional(1, "a", StringType.get())); + + UpdateSchema updateApi = loadUpdateApi(existingSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, PRESERVE_COLUMNS); + + Schema newSchema = updateApi.apply(); + assertThat(newSchema.sameSchema(existingSchema)).isTrue(); + } + @Test public void testIdentifyFieldsByName() { Schema existingSchema = @@ -112,7 +164,7 @@ public void testIdentifyFieldsByName() { UpdateSchema updateApi = loadUpdateApi(existingSchema); Schema newSchema = new Schema(Arrays.asList(Types.NestedField.optional(-1, "myField", Types.LongType.get()))); - EvolveSchemaVisitor.visit(updateApi, existingSchema, newSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, newSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().sameSchema(existingSchema)).isTrue(); } @@ -125,7 +177,7 @@ public void testChangeOrderTopLevelPrimitives() { new Schema( Arrays.asList(optional(2, "b", StringType.get()), optional(1, "a", StringType.get()))); UpdateSchema updateApi = loadUpdateApi(existingSchema); - EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -134,7 +186,7 @@ public void testAddTopLevelListOfPrimitives() { for (PrimitiveType primitiveType : primitiveTypes()) { Schema targetSchema = new Schema(optional(1, "aList", ListType.ofOptional(2, primitiveType))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } } @@ -146,7 +198,7 @@ public void testMakeTopLevelListOfPrimitivesOptional() { new Schema(optional(1, "aList", ListType.ofRequired(2, primitiveType))); Schema targetSchema = new Schema(); UpdateSchema updateApi = loadUpdateApi(existingSchema); - EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, PRESERVE_COLUMNS); Schema expectedSchema = new Schema(optional(1, "aList", ListType.ofRequired(2, primitiveType))); assertThat(updateApi.apply().asStruct()).isEqualTo(expectedSchema.asStruct()); @@ -159,7 +211,7 @@ public void testAddTopLevelMapOfPrimitives() { Schema targetSchema = new Schema(optional(1, "aMap", MapType.ofOptional(2, 3, primitiveType, primitiveType))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } } @@ -171,7 +223,7 @@ public void testAddTopLevelStructOfPrimitives() { new Schema( optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), currentSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), currentSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(currentSchema.asStruct()); } } @@ -184,7 +236,7 @@ public void testAddNestedPrimitive() { new Schema( optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } } @@ -199,7 +251,7 @@ public void testMakeNestedPrimitiveOptional() { new Schema( optional(1, "aStruct", StructType.of(optional(2, "primitive", primitiveType)))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } } @@ -210,7 +262,7 @@ public void testAddNestedPrimitives() { Schema targetSchema = new Schema(optional(1, "aStruct", StructType.of(primitiveFields(1, primitiveTypes())))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -240,7 +292,7 @@ public void testAddNestedLists() { ListType.ofOptional( 10, DecimalType.of(11, 20)))))))))))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -279,7 +331,7 @@ public void testAddNestedStruct() { "aString", StringType.get())))))))))))))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -314,7 +366,7 @@ public void testAddNestedMaps() { 12, 13, StringType.get(), StringType.get())))))))); UpdateSchema updateApi = loadUpdateApi(new Schema()); - EvolveSchemaVisitor.visit(updateApi, new Schema(), targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, new Schema(), targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -326,7 +378,11 @@ public void testDetectInvalidTopLevelList() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aList.element: string -> long") .isInstanceOf(IllegalArgumentException.class); } @@ -343,7 +399,11 @@ public void testDetectInvalidTopLevelMapValue() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aMap.value: string -> long") .isInstanceOf(IllegalArgumentException.class); } @@ -358,7 +418,11 @@ public void testDetectInvalidTopLevelMapKey() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aMap.key: string -> uuid") .isInstanceOf(IllegalArgumentException.class); } @@ -370,7 +434,7 @@ public void testTypePromoteIntegerToLong() { Schema targetSchema = new Schema(required(1, "aCol", LongType.get())); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); Schema applied = updateApi.apply(); assertThat(applied.asStruct().fields()).hasSize(1); assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(LongType.get()); @@ -383,7 +447,7 @@ public void testTypePromoteFloatToDouble() { Schema targetSchema = new Schema(required(1, "aCol", DoubleType.get())); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); Schema applied = updateApi.apply(); assertThat(applied.asStruct().fields()).hasSize(1); assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(DoubleType.get()); @@ -396,7 +460,11 @@ public void testInvalidTypePromoteDoubleToFloat() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aCol: double -> float") .isInstanceOf(IllegalArgumentException.class); } @@ -409,7 +477,7 @@ public void testTypePromoteDecimalToFixedScaleWithWiderPrecision() { Schema targetSchema = new Schema(required(1, "aCol", DecimalType.of(22, 1))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -452,7 +520,7 @@ public void testAddPrimitiveToNestedStruct() { optional(6, "time", TimeType.get()))))))))); UpdateSchema updateApi = loadUpdateApi(existingSchema); - EvolveSchemaVisitor.visit(updateApi, existingSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, existingSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -464,7 +532,11 @@ public void testReplaceListWithPrimitive() { assertThatThrownBy( () -> EvolveSchemaVisitor.visit( - loadUpdateApi(currentSchema), currentSchema, targetSchema)) + TABLE, + loadUpdateApi(currentSchema), + currentSchema, + targetSchema, + PRESERVE_COLUMNS)) .hasMessage("Cannot change column type: aColumn: list -> string") .isInstanceOf(IllegalArgumentException.class); } @@ -501,7 +573,7 @@ public void addNewTopLevelStruct() { optional(7, "d1", StructType.of(optional(8, "d2", StringType.get())))))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -553,7 +625,7 @@ public void testAppendNestedStruct() { StringType.get())))))))))))))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(updateApi.apply().asStruct()).isEqualTo(targetSchema.asStruct()); } @@ -573,7 +645,7 @@ public void testMakeNestedStructOptional() { optional( 3, "s3", StructType.of(optional(4, "s4", StringType.get())))))))); UpdateSchema updateApi = loadUpdateApi(currentSchema); - EvolveSchemaVisitor.visit(updateApi, currentSchema, targetSchema); + EvolveSchemaVisitor.visit(TABLE, updateApi, currentSchema, targetSchema, PRESERVE_COLUMNS); assertThat(getNestedSchemaWithOptionalModifier(true).asStruct()) .isEqualTo(updateApi.apply().asStruct()); } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java index bf5b9f562f9a..7f91d2f8d585 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java @@ -53,17 +53,22 @@ void testCaching() { catalog.createTable(tableIdentifier, SCHEMA); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - Schema schema1 = cache.schema(tableIdentifier, SCHEMA).resolvedTableSchema(); + Schema schema1 = cache.schema(tableIdentifier, SCHEMA, false).resolvedTableSchema(); assertThat(schema1.sameSchema(SCHEMA)).isTrue(); assertThat( - cache.schema(tableIdentifier, SerializationUtils.clone(SCHEMA)).resolvedTableSchema()) + cache + .schema(tableIdentifier, SerializationUtils.clone(SCHEMA), false) + .resolvedTableSchema()) .isEqualTo(schema1); - assertThat(cache.schema(tableIdentifier, SCHEMA2)).isEqualTo(TableMetadataCache.NOT_FOUND); + assertThat(cache.schema(tableIdentifier, SCHEMA2, false)) + .isEqualTo(TableMetadataCache.NOT_FOUND); - schema1 = cache.schema(tableIdentifier, SCHEMA).resolvedTableSchema(); + schema1 = cache.schema(tableIdentifier, SCHEMA, false).resolvedTableSchema(); assertThat( - cache.schema(tableIdentifier, SerializationUtils.clone(SCHEMA)).resolvedTableSchema()) + cache + .schema(tableIdentifier, SerializationUtils.clone(SCHEMA), false) + .resolvedTableSchema()) .isEqualTo(schema1); } @@ -73,9 +78,9 @@ void testCacheInvalidationAfterSchemaChange() { TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); catalog.createTable(tableIdentifier, SCHEMA); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); - Schema schema1 = cache.schema(tableIdentifier, SCHEMA).resolvedTableSchema(); + Schema schema1 = cache.schema(tableIdentifier, SCHEMA, false).resolvedTableSchema(); assertThat(schema1.sameSchema(SCHEMA)).isTrue(); catalog.dropTable(tableIdentifier); @@ -83,7 +88,7 @@ void testCacheInvalidationAfterSchemaChange() { tableUpdater.update( tableIdentifier, "main", SCHEMA2, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); - Schema schema2 = cache.schema(tableIdentifier, SCHEMA2).resolvedTableSchema(); + Schema schema2 = cache.schema(tableIdentifier, SCHEMA2, false).resolvedTableSchema(); assertThat(schema2.sameSchema(SCHEMA2)).isTrue(); } @@ -111,11 +116,11 @@ void testNoCacheRefreshingBeforeRefreshIntervalElapses() { cache.update(tableIdentifier, table); // Cache schema - Schema schema = cache.schema(tableIdentifier, SCHEMA2).resolvedTableSchema(); + Schema schema = cache.schema(tableIdentifier, SCHEMA2, false).resolvedTableSchema(); assertThat(schema.sameSchema(SCHEMA2)).isTrue(); // Cache schema with fewer fields - TableMetadataCache.ResolvedSchemaInfo schemaInfo = cache.schema(tableIdentifier, SCHEMA); + TableMetadataCache.ResolvedSchemaInfo schemaInfo = cache.schema(tableIdentifier, SCHEMA, false); assertThat(schemaInfo.resolvedTableSchema().sameSchema(SCHEMA2)).isTrue(); assertThat(schemaInfo.compareResult()) .isEqualTo(CompareSchemasVisitor.Result.DATA_CONVERSION_NEEDED); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java index 1d4461698746..bcc5d8064517 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java @@ -56,7 +56,7 @@ void testTableCreation(@TempDir Path tempDir) { catalog.createNamespace(Namespace.of("myNamespace")); TableIdentifier tableIdentifier = TableIdentifier.parse("myNamespace.myTable"); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); String locationOverride = tempDir.toString() + "/custom-path"; Map tableProperties = Map.of("key", "value"); @@ -69,7 +69,8 @@ void testTableCreation(@TempDir Path tempDir) { assertThat(catalog.tableExists(tableIdentifier)).isTrue(); assertThat(catalog.loadTable(tableIdentifier).properties().get("key")).isEqualTo("value"); assertThat(catalog.loadTable(tableIdentifier).location()).isEqualTo(locationOverride); - TableMetadataCache.ResolvedSchemaInfo cachedSchema = cache.schema(tableIdentifier, SCHEMA); + TableMetadataCache.ResolvedSchemaInfo cachedSchema = + cache.schema(tableIdentifier, SCHEMA, false); assertThat(cachedSchema.resolvedTableSchema().sameSchema(SCHEMA)).isTrue(); } @@ -78,7 +79,7 @@ void testTableAlreadyExists() { Catalog catalog = CATALOG_EXTENSION.catalog(); TableIdentifier tableIdentifier = TableIdentifier.parse("myTable"); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); // Make the table non-existent in cache cache.exists(tableIdentifier); @@ -98,7 +99,7 @@ void testBranchCreationAndCaching() { Catalog catalog = CATALOG_EXTENSION.catalog(); TableIdentifier tableIdentifier = TableIdentifier.parse("myTable"); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); catalog.createTable(tableIdentifier, SCHEMA); tableUpdater.update( @@ -116,7 +117,7 @@ void testSpecCreation() { Catalog catalog = CATALOG_EXTENSION.catalog(); TableIdentifier tableIdentifier = TableIdentifier.parse("myTable"); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).bucket("data", 10).build(); tableUpdater.update(tableIdentifier, "main", SCHEMA, spec, TableCreator.DEFAULT); @@ -132,8 +133,8 @@ void testInvalidateOldCacheEntryOnUpdate() { TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); catalog.createTable(tableIdentifier, SCHEMA); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - cache.schema(tableIdentifier, SCHEMA); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + cache.schema(tableIdentifier, SCHEMA, false); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); Schema updated = tableUpdater @@ -146,7 +147,8 @@ void testInvalidateOldCacheEntryOnUpdate() { .f0 .resolvedTableSchema(); assertThat(updated.sameSchema(SCHEMA2)).isTrue(); - assertThat(cache.schema(tableIdentifier, SCHEMA2).resolvedTableSchema().sameSchema(SCHEMA2)) + assertThat( + cache.schema(tableIdentifier, SCHEMA2, false).resolvedTableSchema().sameSchema(SCHEMA2)) .isTrue(); } @@ -156,7 +158,7 @@ void testLastResultInvalidation() { TableIdentifier tableIdentifier = TableIdentifier.parse("default.myTable"); catalog.createTable(tableIdentifier, SCHEMA); TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); - TableUpdater tableUpdater = new TableUpdater(cache, catalog); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); // Initialize cache tableUpdater.update( @@ -167,7 +169,7 @@ void testLastResultInvalidation() { catalog.createTable(tableIdentifier, SCHEMA2); // Cache still stores the old information - assertThat(cache.schema(tableIdentifier, SCHEMA2).compareResult()) + assertThat(cache.schema(tableIdentifier, SCHEMA2, false).compareResult()) .isEqualTo(CompareSchemasVisitor.Result.SCHEMA_UPDATE_NEEDED); assertThat( @@ -186,4 +188,27 @@ void testLastResultInvalidation() { assertThat(cache.getInternalCache().get(tableIdentifier).inputSchemas()) .doesNotContainKey(SCHEMA2); } + + @Test + void testDropUnusedColumns() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("myTable"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + + final boolean dropUnusedColumns = true; + TableUpdater tableUpdater = new TableUpdater(cache, catalog, dropUnusedColumns); + + catalog.createTable(tableIdentifier, SCHEMA2); + + Tuple2 result = + tableUpdater.update( + tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + + assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); + Schema tableSchema = catalog.loadTable(tableIdentifier).schema(); + assertThat(tableSchema.columns()).hasSize(2); + assertThat(tableSchema.findField("id")).isNotNull(); + assertThat(tableSchema.findField("data")).isNotNull(); + assertThat(tableSchema.findField("extra")).isNull(); + } } From 642b852487c9396448334afd1a277286ee885ae8 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 8 Dec 2025 16:40:54 +0100 Subject: [PATCH 062/201] OpenAPI: Make namespace separator configurable by server (#14448) --- open-api/rest-catalog-open-api.yaml | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index c977f9b9afb8..44b165ea6e21 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -263,7 +263,10 @@ paths: An optional namespace, underneath which to list namespaces. If not provided, all top-level namespaces should be listed. For backward compatibility, empty string is treated as absent for now. - If parent is a multipart namespace, the parts must be separated by the unit separator (`0x1F`) byte. + If parent is a multipart namespace, the parts must be separated by the namespace separator as + indicated via the /config override `namespace-separator`, which defaults to the unit separator `0x1F` byte (url encoded `%1F`). + To be compatible with older clients, servers must use both the advertised separator and `0x1F` as valid separators when decoding namespaces. + The `namespace-separator` should be provided in a url encoded form. required: false schema: type: string @@ -1835,7 +1838,10 @@ components: required: true description: A namespace identifier as a single string. - Multipart namespace parts should be separated by the unit separator (`0x1F`) byte. + Multipart namespace parts must be separated by the namespace separator as + indicated via the /config override `namespace-separator`, which defaults to the unit separator `0x1F` byte (url encoded `%1F`). + To be compatible with older clients, servers must use both the advertised separator and `0x1F` as valid separators when decoding namespaces. + The `namespace-separator` should be provided in a url encoded form. schema: type: string examples: From 8901269a21853199e7b297756308cd9e208a0342 Mon Sep 17 00:00:00 2001 From: Prashant Singh <35593236+singhpk234@users.noreply.github.com> Date: Mon, 8 Dec 2025 08:37:42 -0800 Subject: [PATCH 063/201] OpenAPI: Add idempotency key for the mutating plan endpoints (#14730) --------- Co-authored-by: Prashant Singh --- open-api/rest-catalog-open-api.yaml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 44b165ea6e21..a438b8ae3b5c 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -662,6 +662,8 @@ paths: needed by calling cancelPlanning. Cancellation is not necessary after fetchScanTasks has been used to fetch scan tasks for each plan task. operationId: planTableScan + parameters: + - $ref: '#/components/parameters/idempotency-key' requestBody: content: application/json: @@ -769,6 +771,8 @@ paths: - Catalog API summary: Cancels scan planning for a plan-id operationId: cancelPlanning + parameters: + - $ref: '#/components/parameters/idempotency-key' description: > Cancels scan planning for a plan-id. @@ -827,6 +831,8 @@ paths: - Catalog API summary: Fetches result tasks for a plan task operationId: fetchScanTasks + parameters: + - $ref: '#/components/parameters/idempotency-key' description: Fetches result tasks for a plan task. requestBody: content: From bd8d2895847bdce71b119f4cb7a83b2e1a58ea73 Mon Sep 17 00:00:00 2001 From: Adam Szita <40628386+szlta@users.noreply.github.com> Date: Mon, 8 Dec 2025 18:42:44 +0100 Subject: [PATCH 064/201] Hive: Metadata integrity check for encrypted tables (#14685) * [WiP] Hive: Metadata integrity check for encrypted tables Introducing new Hive table property metadata_hash (to be stored exclusively in HMS) that tracks the hash of the current table metadata. It is used in HiveTableOperations to carry out integrity check and ensure that the metadata.json has not been tampered with when table encryption is used. * streaming hash calculation + testing Change-Id: I72dad6d8dbc2338299236e495bc76ba60fab7db8 * adding HashWriter closure logic Change-Id: Ie8d978b022ab67bfa3b3238ebc3c5d4f25d0a843 * adding missing space in warning log message Change-Id: Ie5a0310a1b2eb71ed171c1eaa4c4529ada136565 --- .../iceberg/BaseMetastoreTableOperations.java | 1 + .../org/apache/iceberg/util/HashWriter.java | 78 +++++++++++++++++++ .../apache/iceberg/util/TestHashWriter.java | 75 ++++++++++++++++++ .../iceberg/hive/HMSTablePropertyHelper.java | 44 +++++++++++ .../iceberg/hive/HiveTableOperations.java | 46 +++++++---- .../apache/iceberg/hive/TestHiveCatalog.java | 27 +++++++ .../spark/sql/TestTableEncryption.java | 30 +++++++ 7 files changed, 287 insertions(+), 14 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/util/HashWriter.java create mode 100644 core/src/test/java/org/apache/iceberg/util/TestHashWriter.java diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index bda983a6c170..f1223705c11d 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -46,6 +46,7 @@ public abstract class BaseMetastoreTableOperations extends BaseMetastoreOperatio public static final String TABLE_TYPE_PROP = "table_type"; public static final String ICEBERG_TABLE_TYPE_VALUE = "iceberg"; public static final String METADATA_LOCATION_PROP = "metadata_location"; + public static final String METADATA_HASH_PROP = "metadata_hash"; public static final String PREVIOUS_METADATA_LOCATION_PROP = "previous_metadata_location"; private static final String METADATA_FOLDER_NAME = "metadata"; diff --git a/core/src/main/java/org/apache/iceberg/util/HashWriter.java b/core/src/main/java/org/apache/iceberg/util/HashWriter.java new file mode 100644 index 000000000000..2d0668fe8283 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/util/HashWriter.java @@ -0,0 +1,78 @@ +/* + * 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.util; + +import java.io.IOException; +import java.io.Writer; +import java.nio.ByteBuffer; +import java.nio.CharBuffer; +import java.nio.charset.Charset; +import java.nio.charset.CharsetEncoder; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; + +/** + * {@link java.io.Writer} implementation that uses a hashing function to produce a hash value based + * on the streamed bytes. The output of the writer is not preserved. + */ +public class HashWriter extends Writer { + + private final MessageDigest digest; + private final CharsetEncoder encoder; + private boolean isClosed = false; + + public HashWriter(String hashAlgorithm, Charset charset) throws NoSuchAlgorithmException { + this.digest = MessageDigest.getInstance(hashAlgorithm); + this.encoder = charset.newEncoder(); + } + + @Override + public void write(char[] cbuf, int off, int len) throws IOException { + ensureNotClosed(); + CharBuffer chars = CharBuffer.wrap(cbuf, off, len); + ByteBuffer byteBuffer = encoder.encode(chars); + digest.update(byteBuffer); + } + + @Override + public void flush() throws IOException {} + + @Override + public void close() throws IOException { + isClosed = true; + } + + /** + * Calculates the final hash value. The underlying digest will be reset thus subsequent getHash() + * calls are not permitted. + * + * @return bytes of final hash value + */ + public byte[] getHash() { + ensureNotClosed(); + isClosed = true; + return digest.digest(); + } + + private void ensureNotClosed() { + if (isClosed) { + throw new IllegalStateException("HashWriter is closed."); + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/util/TestHashWriter.java b/core/src/test/java/org/apache/iceberg/util/TestHashWriter.java new file mode 100644 index 000000000000..a1c32e996054 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/util/TestHashWriter.java @@ -0,0 +1,75 @@ +/* + * 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.util; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import com.fasterxml.jackson.core.JsonGenerator; +import java.nio.charset.StandardCharsets; +import java.security.MessageDigest; +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestHashWriter { + + @Test + public void testIncrementalHashCalculation() throws Exception { + HashWriter hashWriter = spy(new HashWriter("SHA-256", StandardCharsets.UTF_8)); + + // Create large enough TableMetadata which will be serialized into JSON in multiple chunks by + // the JSON generator + Map icebergTblProperties = Maps.newHashMap(); + for (int i = 0; i < 300; ++i) { + icebergTblProperties.put("Property Key " + i, "Property Value " + i); + } + Schema schema = new Schema(Types.NestedField.required(1, "col1", Types.StringType.get())); + TableMetadata tableMetadata = + TableMetadata.newTableMetadata( + schema, PartitionSpec.unpartitioned(), null, icebergTblProperties); + + JsonGenerator generator = JsonUtil.factory().createGenerator(hashWriter); + TableMetadataParser.toJson(tableMetadata, generator); + + // Expecting to see 3 write() invocations (and therefore incremental hash calculations) + verify(hashWriter, times(3)).write(any(char[].class), anyInt(), anyInt()); + + // +1 after flushing + generator.flush(); + verify(hashWriter, times(4)).write(any(char[].class), anyInt(), anyInt()); + + // Expected hash is calculated on the whole object i.e. without streaming + byte[] expectedHash = + MessageDigest.getInstance("SHA-256") + .digest(TableMetadataParser.toJson(tableMetadata).getBytes(StandardCharsets.UTF_8)); + assertThat(hashWriter.getHash()).isEqualTo(expectedHash); + assertThatThrownBy(() -> hashWriter.getHash()).hasMessageContaining("HashWriter is closed."); + } +} diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HMSTablePropertyHelper.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HMSTablePropertyHelper.java index 0a177a7190d2..a50210bfc5db 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HMSTablePropertyHelper.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HMSTablePropertyHelper.java @@ -20,7 +20,13 @@ import static org.apache.iceberg.TableProperties.GC_ENABLED; +import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonProcessingException; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.security.NoSuchAlgorithmException; +import java.util.Arrays; +import java.util.Base64; import java.util.Locale; import java.util.Map; import java.util.Optional; @@ -36,10 +42,12 @@ import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.SortOrderParser; import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.HashWriter; import org.apache.iceberg.util.JsonUtil; import org.apache.iceberg.view.ViewMetadata; import org.slf4j.Logger; @@ -101,6 +109,7 @@ public static void updateHmsTableForIcebergTable( metadata.schema(), maxHiveTablePropertySize); setStorageHandler(parameters, hiveEngineEnabled); + setMetadataHash(metadata, parameters); // Set the basic statistics if (summary.get(SnapshotSummary.TOTAL_DATA_FILES_PROP) != null) { @@ -256,6 +265,41 @@ static void setSchema( } } + @VisibleForTesting + static void setMetadataHash(TableMetadata metadata, Map parameters) { + if (parameters.containsKey(TableProperties.ENCRYPTION_TABLE_KEY)) { + byte[] currentHashBytes = hashOf(metadata); + parameters.put( + BaseMetastoreTableOperations.METADATA_HASH_PROP, + Base64.getEncoder().encodeToString(currentHashBytes)); + } + } + + @VisibleForTesting + static void verifyMetadataHash(TableMetadata metadata, String metadataHashFromHMS) { + byte[] currentHashBytes = hashOf(metadata); + byte[] expectedHashBytes = Base64.getDecoder().decode(metadataHashFromHMS); + + if (!Arrays.equals(expectedHashBytes, currentHashBytes)) { + throw new RuntimeException( + String.format( + "The current metadata file %s might have been modified. Hash of metadata loaded from storage differs " + + "from HMS-stored metadata hash.", + metadata.metadataFileLocation())); + } + } + + private static byte[] hashOf(TableMetadata tableMetadata) { + try (HashWriter hashWriter = new HashWriter("SHA-256", StandardCharsets.UTF_8)) { + JsonGenerator generator = JsonUtil.factory().createGenerator(hashWriter); + TableMetadataParser.toJson(tableMetadata, generator); + generator.flush(); + return hashWriter.getHash(); + } catch (NoSuchAlgorithmException | IOException e) { + throw new RuntimeException("Unable to produce hash of table metadata", e); + } + } + private static void setField( Map parameters, String key, String value, long maxHiveTablePropertySize) { if (value.length() <= maxHiveTablePropertySize) { diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java index 4d382f8d388e..62340fe40138 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java @@ -25,6 +25,7 @@ import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.common.StatsSetupConst; import org.apache.hadoop.hive.metastore.IMetaStoreClient; @@ -174,6 +175,7 @@ protected void doRefresh() { String metadataLocation = null; String tableKeyIdFromHMS = null; String dekLengthFromHMS = null; + String metadataHashFromHMS = null; try { Table table = metaClients.run(client -> client.getTable(database, tableName)); @@ -189,6 +191,7 @@ the table key parameter (along with existing snapshots) in the file, making the produce unencrypted files. Table key ID is taken directly from HMS catalog */ tableKeyIdFromHMS = table.getParameters().get(TableProperties.ENCRYPTION_TABLE_KEY); dekLengthFromHMS = table.getParameters().get(TableProperties.ENCRYPTION_DEK_LENGTH); + metadataHashFromHMS = table.getParameters().get(METADATA_HASH_PROP); } catch (NoSuchObjectException e) { if (currentMetadataLocation() != null) { throw new NoSuchTableException("No such table: %s.%s", database, tableName); @@ -207,7 +210,7 @@ the table key parameter (along with existing snapshots) in the file, making the refreshFromMetadataLocation(metadataLocation, metadataRefreshMaxRetries); if (tableKeyIdFromHMS != null) { - checkEncryptionProperties(tableKeyIdFromHMS, dekLengthFromHMS); + checkIntegrityForEncryption(tableKeyIdFromHMS, dekLengthFromHMS, metadataHashFromHMS); tableKeyId = tableKeyIdFromHMS; encryptionDekLength = @@ -245,6 +248,7 @@ the table key parameter (along with existing snapshots) in the file, making the @Override protected void doCommit(TableMetadata base, TableMetadata metadata) { boolean newTable = base == null; + final TableMetadata tableMetadata; encryptionPropsFromMetadata(metadata.properties()); String newMetadataLocation; @@ -257,19 +261,21 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { builder.addEncryptionKey(entry.getValue()); } - newMetadataLocation = writeNewMetadataIfRequired(newTable, builder.build()); + tableMetadata = builder.build(); } else { - newMetadataLocation = writeNewMetadataIfRequired(newTable, metadata); + tableMetadata = metadata; } - boolean hiveEngineEnabled = hiveEngineEnabled(metadata, conf); + newMetadataLocation = writeNewMetadataIfRequired(newTable, tableMetadata); + + boolean hiveEngineEnabled = hiveEngineEnabled(tableMetadata, conf); boolean keepHiveStats = conf.getBoolean(ConfigProperties.KEEP_HIVE_STATS, false); BaseMetastoreOperations.CommitStatus commitStatus = BaseMetastoreOperations.CommitStatus.FAILURE; boolean updateHiveTable = false; - HiveLock lock = lockObject(base != null ? base : metadata); + HiveLock lock = lockObject(base != null ? base : tableMetadata); try { lock.lock(); @@ -293,14 +299,14 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { } else { tbl = newHmsTable( - metadata.property(HiveCatalog.HMS_TABLE_OWNER, HiveHadoopUtil.currentUser())); + tableMetadata.property(HiveCatalog.HMS_TABLE_OWNER, HiveHadoopUtil.currentUser())); LOG.debug("Committing new table: {}", fullName); } tbl.setSd( HiveOperationsBase.storageDescriptor( - metadata.schema(), - metadata.location(), + tableMetadata.schema(), + tableMetadata.location(), hiveEngineEnabled)); // set to pickup any schema changes String metadataLocation = tbl.getParameters().get(METADATA_LOCATION_PROP); @@ -316,7 +322,7 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { if (base != null) { removedProps = base.properties().keySet().stream() - .filter(key -> !metadata.properties().containsKey(key)) + .filter(key -> !tableMetadata.properties().containsKey(key)) .collect(Collectors.toSet()); } @@ -334,7 +340,7 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { HMSTablePropertyHelper.updateHmsTableForIcebergTable( newMetadataLocation, tbl, - metadata, + tableMetadata, removedProps, hiveEngineEnabled, maxHiveTablePropertySize, @@ -391,7 +397,7 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { // issue for example, and triggers this exception. So we need double-check to make sure // this is really a concurrent modification. Hitting this exception means no pending // requests, if any, can succeed later, so it's safe to check status in strict mode - commitStatus = checkCommitStatusStrict(newMetadataLocation, metadata); + commitStatus = checkCommitStatusStrict(newMetadataLocation, tableMetadata); if (commitStatus == BaseMetastoreOperations.CommitStatus.FAILURE) { throw new CommitFailedException( e, "The table %s.%s has been modified concurrently", database, tableName); @@ -402,7 +408,7 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { database, tableName, e); - commitStatus = checkCommitStatus(newMetadataLocation, metadata); + commitStatus = checkCommitStatus(newMetadataLocation, tableMetadata); } switch (commitStatus) { @@ -574,8 +580,20 @@ private void encryptionPropsFromMetadata(Map tableProperties) { } } - private void checkEncryptionProperties(String encryptionKeyIdFromHMS, String dekLengthFromHMS) { - Map propertiesFromMetadata = current().properties(); + private void checkIntegrityForEncryption( + String encryptionKeyIdFromHMS, String dekLengthFromHMS, String metadataHashFromHMS) { + TableMetadata metadata = current(); + if (StringUtils.isNotEmpty(metadataHashFromHMS)) { + HMSTablePropertyHelper.verifyMetadataHash(metadata, metadataHashFromHMS); + return; + } + + LOG.warn( + "Full metadata integrity check skipped because no metadata hash was recorded in HMS for table {}." + + " Falling back to encryption property based check.", + tableName); + + Map propertiesFromMetadata = metadata.properties(); String encryptionKeyIdFromMetadata = propertiesFromMetadata.get(TableProperties.ENCRYPTION_TABLE_KEY); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java index 2bac6082854c..f212d307fe9d 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCatalog.java @@ -26,6 +26,7 @@ import static org.apache.iceberg.TableProperties.CURRENT_SNAPSHOT_TIMESTAMP; import static org.apache.iceberg.TableProperties.DEFAULT_PARTITION_SPEC; import static org.apache.iceberg.TableProperties.DEFAULT_SORT_ORDER; +import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY; import static org.apache.iceberg.TableProperties.SNAPSHOT_COUNT; import static org.apache.iceberg.expressions.Expressions.bucket; import static org.apache.iceberg.types.Types.NestedField.required; @@ -47,6 +48,7 @@ import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.CachingCatalog; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; @@ -1239,4 +1241,29 @@ public void testTableLocationWithTrailingSlashInDatabaseLocation() throws TExcep HIVE_METASTORE_EXTENSION.metastoreClient().dropDatabase(dbName); } } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testMetadataHashing(boolean isTableEncrypted) { + Map hiveTblProperties = Maps.newHashMap(); + if (isTableEncrypted) { + hiveTblProperties.put(ENCRYPTION_TABLE_KEY, "key_id"); + } + + Schema schema = new Schema(Types.NestedField.required(1, "col1", Types.StringType.get())); + TableMetadata tableMetadata = + TableMetadata.newTableMetadata( + schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); + + HMSTablePropertyHelper.setMetadataHash(tableMetadata, hiveTblProperties); + + String base64EncodedHash = + hiveTblProperties.get(BaseMetastoreTableOperations.METADATA_HASH_PROP); + if (isTableEncrypted) { + assertThat(base64EncodedHash).isBase64(); + HMSTablePropertyHelper.verifyMetadataHash(tableMetadata, base64EncodedHash); + } else { + assertThat(base64EncodedHash).isNull(); + } + } } diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java index e35ebf71b5c5..85e7f48b59d7 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java @@ -29,14 +29,20 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.ChecksumFileSystem; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; import org.apache.iceberg.Transaction; import org.apache.iceberg.encryption.Ciphers; import org.apache.iceberg.encryption.UnitestKMS; @@ -53,6 +59,7 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; +import org.mockito.internal.util.collections.Iterables; public class TestTableEncryption extends CatalogTestBase { private static Map appendCatalogEncryptionProperties(Map props) { @@ -162,6 +169,29 @@ public void testInsertAndDelete() { sql("SELECT * FROM %s ORDER BY id", tableName)); } + @TestTemplate + public void testMetadataTamperproofing() throws IOException { + ChecksumFileSystem fs = ((ChecksumFileSystem) FileSystem.newInstance(new Configuration())); + catalog.initialize(catalogName, catalogConfig); + + Table table = catalog.loadTable(tableIdent); + TableMetadata currentMetadata = ((HasTableOperations) table).operations().current(); + Path metadataFile = new Path(currentMetadata.metadataFileLocation()); + Path previousMetadataFile = new Path(Iterables.firstOf(currentMetadata.previousFiles()).file()); + + // manual FS tampering: replacing the current metadata file with a previous one + Path crcPath = fs.getChecksumFile(metadataFile); + fs.delete(crcPath, false); + fs.delete(metadataFile, false); + fs.rename(previousMetadataFile, metadataFile); + + assertThatThrownBy(() -> catalog.loadTable(tableIdent)) + .hasMessageContaining( + String.format( + "The current metadata file %s might have been modified. Hash of metadata loaded from storage differs from HMS-stored metadata hash.", + metadataFile)); + } + @TestTemplate public void testKeyDelete() { assertThatThrownBy( From 344adf9d123cfb45a14d8d0812c1d299753bee76 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 9 Dec 2025 10:09:13 +0100 Subject: [PATCH 065/201] Core: Make namespace separator configurable (#10877) --- .../iceberg/rest/RESTCatalogProperties.java | 2 + .../iceberg/rest/RESTSessionCatalog.java | 9 +- .../org/apache/iceberg/rest/RESTUtil.java | 138 +++++++++++++++--- .../apache/iceberg/rest/ResourcePaths.java | 39 +++-- .../iceberg/rest/RESTCatalogAdapter.java | 13 +- .../apache/iceberg/rest/TestRESTCatalog.java | 12 +- .../org/apache/iceberg/rest/TestRESTUtil.java | 79 +++++++++- .../iceberg/rest/TestResourcePaths.java | 46 ++++++ 8 files changed, 297 insertions(+), 41 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTCatalogProperties.java b/core/src/main/java/org/apache/iceberg/rest/RESTCatalogProperties.java index 264c31436088..e71610622bac 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTCatalogProperties.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTCatalogProperties.java @@ -35,6 +35,8 @@ private RESTCatalogProperties() {} public static final String PAGE_SIZE = "rest-page-size"; + public static final String NAMESPACE_SEPARATOR = "namespace-separator"; + public enum SnapshotMode { ALL, REFS diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index fe0d13217b62..814ed978c4ad 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -161,6 +161,7 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private CloseableGroup closeables = null; private Set endpoints; private Supplier> mutationHeaders = Map::of; + private String namespaceSeparator = null; public RESTSessionCatalog() { this( @@ -263,6 +264,12 @@ public void initialize(String name, Map unresolved) { mergedProps, RESTCatalogProperties.METRICS_REPORTING_ENABLED, RESTCatalogProperties.METRICS_REPORTING_ENABLED_DEFAULT); + this.namespaceSeparator = + PropertyUtil.propertyAsString( + mergedProps, + RESTCatalogProperties.NAMESPACE_SEPARATOR, + RESTUtil.NAMESPACE_SEPARATOR_URLENCODED_UTF_8); + super.initialize(name, mergedProps); } @@ -580,7 +587,7 @@ public List listNamespaces(SessionContext context, Namespace namespac Map queryParams = Maps.newHashMap(); if (!namespace.isEmpty()) { - queryParams.put("parent", RESTUtil.namespaceToQueryParam(namespace)); + queryParams.put("parent", RESTUtil.namespaceToQueryParam(namespace, namespaceSeparator)); } ImmutableList.Builder namespaces = ImmutableList.builder(); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java b/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java index ec02a9dc8459..f4fdf3af26e7 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java @@ -26,29 +26,33 @@ import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Splitter; +import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.UUIDUtil; +@SuppressWarnings("UnicodeEscape") public class RESTUtil { - private static final char NAMESPACE_SEPARATOR = '\u001f'; - private static final String NAMESPACE_ESCAPED_SEPARATOR = "%1F"; - private static final Joiner NAMESPACE_ESCAPED_JOINER = Joiner.on(NAMESPACE_ESCAPED_SEPARATOR); - private static final Splitter NAMESPACE_ESCAPED_SPLITTER = - Splitter.on(NAMESPACE_ESCAPED_SEPARATOR); + /** The namespace separator as Unicode character */ + private static final char NAMESPACE_SEPARATOR_AS_UNICODE = '\u001f'; + + /** The namespace separator as url encoded UTF-8 character */ + static final String NAMESPACE_SEPARATOR_URLENCODED_UTF_8 = "%1F"; /** * @deprecated since 1.11.0, will be removed in 1.12.0; use {@link * RESTUtil#namespaceToQueryParam(Namespace)}} instead. */ - @Deprecated public static final Joiner NAMESPACE_JOINER = Joiner.on(NAMESPACE_SEPARATOR); + @Deprecated + public static final Joiner NAMESPACE_JOINER = Joiner.on(NAMESPACE_SEPARATOR_AS_UNICODE); /** * @deprecated since 1.11.0, will be removed in 1.12.0; use {@link * RESTUtil#namespaceFromQueryParam(String)} instead. */ - @Deprecated public static final Splitter NAMESPACE_SPLITTER = Splitter.on(NAMESPACE_SEPARATOR); + @Deprecated + public static final Splitter NAMESPACE_SPLITTER = Splitter.on(NAMESPACE_SEPARATOR_AS_UNICODE); public static final String IDEMPOTENCY_KEY_HEADER = "Idempotency-Key"; @@ -179,8 +183,31 @@ public static String decodeString(String encoded) { * separated using the unicode character '\u001f' */ public static String namespaceToQueryParam(Namespace namespace) { + return namespaceToQueryParam(namespace, String.valueOf(NAMESPACE_SEPARATOR_AS_UNICODE)); + } + + /** + * This converts the given namespace to a string and separates each part in a multipart namespace + * using the provided unicode separator. Note that this method is different from {@link + * RESTUtil#encodeNamespace(Namespace)}, which uses a UTF-8 escaped separator. + * + *

    {@link #namespaceFromQueryParam(String, String)} should be used to convert the namespace + * string back to a {@link Namespace} instance. + * + * @param namespace The namespace to convert + * @param unicodeNamespaceSeparator The unicode namespace separator to use, such as '\u002e' + * @return The namespace converted to a string where each part in a multipart namespace is + * separated using the given unicode separator + */ + public static String namespaceToQueryParam( + Namespace namespace, String unicodeNamespaceSeparator) { Preconditions.checkArgument(null != namespace, "Invalid namespace: null"); - return RESTUtil.NAMESPACE_JOINER.join(namespace.levels()); + Preconditions.checkArgument( + !Strings.isNullOrEmpty(unicodeNamespaceSeparator), "Invalid separator: null or empty"); + + // decode in case the separator was already encoded with UTF-8 + String separator = URLDecoder.decode(unicodeNamespaceSeparator, StandardCharsets.UTF_8); + return Joiner.on(separator).join(namespace.levels()); } /** @@ -191,13 +218,39 @@ public static String namespaceToQueryParam(Namespace namespace) { * a string. * * @param namespace The namespace to convert - * @return The namespace instance from the given namespace string, where each multipart separator - * ('\u001f') is converted to a separate namespace level + * @return The namespace instance from the given namespace string, where each part in a multipart + * namespace is converted using the unicode separator '\u001f' */ public static Namespace namespaceFromQueryParam(String namespace) { + return namespaceFromQueryParam(namespace, String.valueOf(NAMESPACE_SEPARATOR_AS_UNICODE)); + } + + /** + * This converts a namespace where each part in a multipart namespace has been separated using the + * provided unicode separator to its original {@link Namespace} instance. + * + *

    {@link #namespaceToQueryParam(Namespace, String)} should be used to convert the {@link + * Namespace} to a string. + * + * @param namespace The namespace to convert + * @param unicodeNamespaceSeparator The unicode namespace separator to use, such as '\u002e' + * @return The namespace instance from the given namespace string, where each part in a multipart + * namespace is converted using the given unicode namespace separator + */ + public static Namespace namespaceFromQueryParam( + String namespace, String unicodeNamespaceSeparator) { Preconditions.checkArgument(null != namespace, "Invalid namespace: null"); - return Namespace.of( - RESTUtil.NAMESPACE_SPLITTER.splitToStream(namespace).toArray(String[]::new)); + Preconditions.checkArgument( + !Strings.isNullOrEmpty(unicodeNamespaceSeparator), "Invalid separator: null or empty"); + + // decode in case the separator was already encoded with UTF-8 + String separator = URLDecoder.decode(unicodeNamespaceSeparator, StandardCharsets.UTF_8); + Splitter splitter = + namespace.contains(String.valueOf(NAMESPACE_SEPARATOR_AS_UNICODE)) + ? Splitter.on(NAMESPACE_SEPARATOR_AS_UNICODE) + : Splitter.on(separator); + + return Namespace.of(splitter.splitToStream(namespace).toArray(String[]::new)); } /** @@ -210,17 +263,40 @@ public static Namespace namespaceFromQueryParam(String namespace) { * * @param ns namespace to encode * @return UTF-8 encoded string representing the namespace, suitable for use as a URL parameter + * @deprecated since 1.11.0, will be removed in 1.12.0; use {@link + * RESTUtil#encodeNamespace(Namespace, String)} instead. */ + @Deprecated public static String encodeNamespace(Namespace ns) { - Preconditions.checkArgument(ns != null, "Invalid namespace: null"); - String[] levels = ns.levels(); + return encodeNamespace(ns, NAMESPACE_SEPARATOR_URLENCODED_UTF_8); + } + + /** + * Returns a String representation of a namespace that is suitable for use in a URL / URI. + * + *

    This function needs to be called when a namespace is used as a path variable (or query + * parameter etc.), to format the namespace per the spec. + * + *

    {@link RESTUtil#decodeNamespace(String, String)} should be used to parse the namespace from + * a URL parameter. + * + * @param namespace namespace to encode + * @param separator The namespace separator to be used for encoding. The separator will be used + * as-is and won't be UTF-8 encoded. + * @return UTF-8 encoded string representing the namespace, suitable for use as a URL parameter + */ + public static String encodeNamespace(Namespace namespace, String separator) { + Preconditions.checkArgument(namespace != null, "Invalid namespace: null"); + Preconditions.checkArgument( + !Strings.isNullOrEmpty(separator), "Invalid separator: null or empty"); + String[] levels = namespace.levels(); String[] encodedLevels = new String[levels.length]; for (int i = 0; i < levels.length; i++) { encodedLevels[i] = encodeString(levels[i]); } - return NAMESPACE_ESCAPED_JOINER.join(encodedLevels); + return Joiner.on(separator).join(encodedLevels); } /** @@ -231,10 +307,38 @@ public static String encodeNamespace(Namespace ns) { * * @param encodedNs a namespace to decode * @return a namespace + * @deprecated since 1.11.0, will be removed in 1.12.0; use {@link + * RESTUtil#decodeNamespace(String, String)} instead. */ + @Deprecated public static Namespace decodeNamespace(String encodedNs) { - Preconditions.checkArgument(encodedNs != null, "Invalid namespace: null"); - String[] levels = Iterables.toArray(NAMESPACE_ESCAPED_SPLITTER.split(encodedNs), String.class); + return decodeNamespace(encodedNs, NAMESPACE_SEPARATOR_URLENCODED_UTF_8); + } + + /** + * Takes in a string representation of a namespace as used for a URL parameter and returns the + * corresponding namespace. + * + *

    See also {@link #encodeNamespace} for generating correctly formatted URLs. + * + * @param encodedNamespace a namespace to decode + * @param separator The namespace separator to be used as-is for decoding. This should be the same + * separator that was used when calling {@link RESTUtil#encodeNamespace(Namespace, String)} + * @return a namespace + */ + public static Namespace decodeNamespace(String encodedNamespace, String separator) { + Preconditions.checkArgument(encodedNamespace != null, "Invalid namespace: null"); + Preconditions.checkArgument( + !Strings.isNullOrEmpty(separator), "Invalid separator: null or empty"); + + // use legacy splitter for backwards compatibility in case an old clients encoded the namespace + // with %1F + Splitter splitter = + Splitter.on( + encodedNamespace.contains(NAMESPACE_SEPARATOR_URLENCODED_UTF_8) + ? NAMESPACE_SEPARATOR_URLENCODED_UTF_8 + : separator); + String[] levels = Iterables.toArray(splitter.split(encodedNamespace), String.class); // Decode levels in place for (int i = 0; i < levels.length; i++) { diff --git a/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java b/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java index 275a87e96815..d85b00d02ebe 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java +++ b/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java @@ -22,6 +22,7 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.util.PropertyUtil; public class ResourcePaths { private static final Joiner SLASH = Joiner.on("/").skipNulls(); @@ -50,7 +51,12 @@ public class ResourcePaths { public static final String V1_VIEW_RENAME = "/v1/{prefix}/views/rename"; public static ResourcePaths forCatalogProperties(Map properties) { - return new ResourcePaths(properties.get(PREFIX)); + return new ResourcePaths( + properties.get(PREFIX), + PropertyUtil.propertyAsString( + properties, + RESTCatalogProperties.NAMESPACE_SEPARATOR, + RESTUtil.NAMESPACE_SEPARATOR_URLENCODED_UTF_8)); } public static String config() { @@ -62,9 +68,20 @@ public static String tokens() { } private final String prefix; + private final String namespaceSeparator; + /** + * @deprecated since 1.11.0, will be made private in 1.12.0; use {@link + * ResourcePaths#forCatalogProperties(Map)} instead. + */ + @Deprecated public ResourcePaths(String prefix) { + this(prefix, RESTUtil.NAMESPACE_SEPARATOR_URLENCODED_UTF_8); + } + + private ResourcePaths(String prefix, String namespaceSeparator) { this.prefix = prefix; + this.namespaceSeparator = namespaceSeparator; } public String namespaces() { @@ -72,15 +89,15 @@ public String namespaces() { } public String namespace(Namespace ns) { - return SLASH.join("v1", prefix, "namespaces", RESTUtil.encodeNamespace(ns)); + return SLASH.join("v1", prefix, "namespaces", pathEncode(ns)); } public String namespaceProperties(Namespace ns) { - return SLASH.join("v1", prefix, "namespaces", RESTUtil.encodeNamespace(ns), "properties"); + return SLASH.join("v1", prefix, "namespaces", pathEncode(ns), "properties"); } public String tables(Namespace ns) { - return SLASH.join("v1", prefix, "namespaces", RESTUtil.encodeNamespace(ns), "tables"); + return SLASH.join("v1", prefix, "namespaces", pathEncode(ns), "tables"); } public String table(TableIdentifier ident) { @@ -88,13 +105,13 @@ public String table(TableIdentifier ident) { "v1", prefix, "namespaces", - RESTUtil.encodeNamespace(ident.namespace()), + pathEncode(ident.namespace()), "tables", RESTUtil.encodeString(ident.name())); } public String register(Namespace ns) { - return SLASH.join("v1", prefix, "namespaces", RESTUtil.encodeNamespace(ns), "register"); + return SLASH.join("v1", prefix, "namespaces", pathEncode(ns), "register"); } public String rename() { @@ -106,7 +123,7 @@ public String metrics(TableIdentifier identifier) { "v1", prefix, "namespaces", - RESTUtil.encodeNamespace(identifier.namespace()), + pathEncode(identifier.namespace()), "tables", RESTUtil.encodeString(identifier.name()), "metrics"); @@ -117,7 +134,7 @@ public String commitTransaction() { } public String views(Namespace ns) { - return SLASH.join("v1", prefix, "namespaces", RESTUtil.encodeNamespace(ns), "views"); + return SLASH.join("v1", prefix, "namespaces", pathEncode(ns), "views"); } public String view(TableIdentifier ident) { @@ -125,7 +142,7 @@ public String view(TableIdentifier ident) { "v1", prefix, "namespaces", - RESTUtil.encodeNamespace(ident.namespace()), + pathEncode(ident.namespace()), "views", RESTUtil.encodeString(ident.name())); } @@ -133,4 +150,8 @@ public String view(TableIdentifier ident) { public String renameView() { return SLASH.join("v1", prefix, "views", "rename"); } + + private String pathEncode(Namespace ns) { + return RESTUtil.encodeNamespace(ns, namespaceSeparator); + } } diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java index ff6daa61e35c..586da0d10782 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -81,6 +81,12 @@ /** Adaptor class to translate REST requests into {@link Catalog} API calls. */ public class RESTCatalogAdapter extends BaseHTTPClient { + + @SuppressWarnings("AvoidEscapedUnicodeCharacters") + private static final String NAMESPACE_SEPARATOR_UNICODE = "\u002e"; + + private static final String NAMESPACE_SEPARATOR_URLENCODED_UTF_8 = "%2E"; + private static final Map, Integer> EXCEPTION_ERROR_CODES = ImmutableMap., Integer>builder() .put(IllegalArgumentException.class, 400) @@ -168,13 +174,15 @@ public T handleRequest( Arrays.stream(Route.values()) .map(r -> Endpoint.create(r.method().name(), r.resourcePath())) .collect(Collectors.toList())) + .withOverride( + RESTCatalogProperties.NAMESPACE_SEPARATOR, NAMESPACE_SEPARATOR_URLENCODED_UTF_8) .build()); case LIST_NAMESPACES: if (asNamespaceCatalog != null) { Namespace ns; if (vars.containsKey("parent")) { - ns = RESTUtil.namespaceFromQueryParam(vars.get("parent")); + ns = RESTUtil.namespaceFromQueryParam(vars.get("parent"), NAMESPACE_SEPARATOR_UNICODE); } else { ns = Namespace.empty(); } @@ -645,7 +653,8 @@ public static void configureResponseFromException( } private static Namespace namespaceFromPathVars(Map pathVars) { - return RESTUtil.decodeNamespace(pathVars.get("namespace")); + return RESTUtil.decodeNamespace( + pathVars.get("namespace"), NAMESPACE_SEPARATOR_URLENCODED_UTF_8); } private static TableIdentifier tableIdentFromPathVars(Map pathVars) { diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index 385893ea7130..753d8cb247c2 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -2943,7 +2943,7 @@ public void testNotModified() { assertThatThrownBy(() -> catalog().loadTable(TABLE)).isInstanceOf(NullPointerException.class); TableIdentifier metadataTableIdentifier = - TableIdentifier.of(TABLE.namespace().toString(), TABLE.name(), "partitions"); + TableIdentifier.of(NS.toString(), TABLE.name(), "partitions"); // TODO: This won't throw when client side of freshness-aware loading is implemented assertThatThrownBy(() -> catalog().loadTable(metadataTableIdentifier)) @@ -2956,12 +2956,14 @@ public void testNotModified() { any(), any()); + // RESTCatalogAdapter uses %2E as a namespace separator, and we're verifying here which + // server-side path was called + ResourcePaths paths = + ResourcePaths.forCatalogProperties( + ImmutableMap.of(RESTCatalogProperties.NAMESPACE_SEPARATOR, "%2E")); verify(adapterForRESTServer) .execute( - reqMatcher(HTTPMethod.GET, RESOURCE_PATHS.table(metadataTableIdentifier)), - any(), - any(), - any()); + reqMatcher(HTTPMethod.GET, paths.table(metadataTableIdentifier)), any(), any(), any()); } @Test diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTUtil.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTUtil.java index fe022b351822..1ed732ebc91a 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTUtil.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTUtil.java @@ -26,6 +26,8 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public class TestRESTUtil { @@ -68,18 +70,24 @@ public void testStripTrailingSlash() { } } - @Test - public void testRoundTripUrlEncodeDecodeNamespace() { + @ParameterizedTest + @ValueSource(strings = {"%1F", "%2D", "%2E", "#", "_"}) + public void testRoundTripUrlEncodeDecodeNamespace(String namespaceSeparator) { // Namespace levels and their expected url encoded form Object[][] testCases = new Object[][] { new Object[] {new String[] {"dogs"}, "dogs"}, new Object[] {new String[] {"dogs.named.hank"}, "dogs.named.hank"}, new Object[] {new String[] {"dogs/named/hank"}, "dogs%2Fnamed%2Fhank"}, - new Object[] {new String[] {"dogs", "named", "hank"}, "dogs%1Fnamed%1Fhank"}, + new Object[] { + new String[] {"dogs", "named", "hank"}, + String.format("dogs%snamed%shank", namespaceSeparator, namespaceSeparator) + }, new Object[] { new String[] {"dogs.and.cats", "named", "hank.or.james-westfall"}, - "dogs.and.cats%1Fnamed%1Fhank.or.james-westfall" + String.format( + "dogs.and.cats%snamed%shank.or.james-westfall", + namespaceSeparator, namespaceSeparator), } }; @@ -90,14 +98,35 @@ public void testRoundTripUrlEncodeDecodeNamespace() { Namespace namespace = Namespace.of(levels); // To be placed into a URL path as query parameter or path parameter - assertThat(RESTUtil.encodeNamespace(namespace)).isEqualTo(encodedNs); + assertThat(RESTUtil.encodeNamespace(namespace, namespaceSeparator)).isEqualTo(encodedNs); // Decoded (after pulling as String) from URL - Namespace asNamespace = RESTUtil.decodeNamespace(encodedNs); - assertThat(asNamespace).isEqualTo(namespace); + assertThat(RESTUtil.decodeNamespace(encodedNs, namespaceSeparator)).isEqualTo(namespace); } } + @Test + public void encodeAsOldClientAndDecodeAsNewServer() { + Namespace namespace = Namespace.of("first", "second", "third"); + // old client would call encodeNamespace without specifying a separator + String encodedNamespace = RESTUtil.encodeNamespace(namespace); + assertThat(encodedNamespace).contains(RESTUtil.NAMESPACE_SEPARATOR_URLENCODED_UTF_8); + + // old client would also call namespaceToQueryParam without specifying a separator + String namespaceAsUnicode = RESTUtil.namespaceToQueryParam(namespace); + assertThat(namespaceAsUnicode).contains("\u001f"); + + // newer server would try and decode the namespace with the separator it communicates to clients + String separator = "%2E"; + Namespace decodedNamespace = RESTUtil.decodeNamespace(encodedNamespace, separator); + assertThat(decodedNamespace).isEqualTo(namespace); + + // newer server would try and split the namespace with the separator it communicates to clients + // but should detect whether the namespace contains the legacy separator + assertThat(RESTUtil.namespaceFromQueryParam(namespaceAsUnicode, separator)) + .isEqualTo(namespace); + } + @Test public void testNamespaceUrlEncodeDecodeDoesNotAllowNull() { assertThatExceptionOfType(IllegalArgumentException.class) @@ -212,4 +241,40 @@ public void namespaceFromQueryParam() { assertThat(RESTUtil.namespaceFromQueryParam("one%1Ftwo\u001fns")) .isEqualTo(Namespace.of("one%1Ftwo", "ns")); } + + @Test + public void nullOrEmptyNamespaceSeparator() { + String errorMsg = "Invalid separator: null or empty"; + assertThatThrownBy(() -> RESTUtil.encodeNamespace(Namespace.empty(), null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage(errorMsg); + + assertThatThrownBy(() -> RESTUtil.encodeNamespace(Namespace.empty(), "")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage(errorMsg); + + assertThatThrownBy(() -> RESTUtil.decodeNamespace("namespace", null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage(errorMsg); + + assertThatThrownBy(() -> RESTUtil.decodeNamespace("namespace", "")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage(errorMsg); + + assertThatThrownBy(() -> RESTUtil.namespaceToQueryParam(Namespace.empty(), null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage(errorMsg); + + assertThatThrownBy(() -> RESTUtil.namespaceToQueryParam(Namespace.empty(), "")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage(errorMsg); + + assertThatThrownBy(() -> RESTUtil.namespaceFromQueryParam("namespace", null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage(errorMsg); + + assertThatThrownBy(() -> RESTUtil.namespaceFromQueryParam("namespace", null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage(errorMsg); + } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java b/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java index c2d3fe560345..ebcced1b3766 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java @@ -24,6 +24,8 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public class TestResourcePaths { private final String prefix = "ws/catalog"; @@ -64,6 +66,50 @@ public void testNamespaceWithMultipartNamespace() { assertThat(withoutPrefix.namespace(ns)).isEqualTo("v1/namespaces/n%1Fs"); } + @ParameterizedTest + @ValueSource(strings = {"%1F", "%2D", "%2E"}) + public void testNamespaceWithMultipartNamespace(String namespaceSeparator) { + Namespace ns = Namespace.of("n", "s"); + String namespace = String.format("n%ss", namespaceSeparator); + assertThat( + ResourcePaths.forCatalogProperties( + ImmutableMap.of( + "prefix", + prefix, + RESTCatalogProperties.NAMESPACE_SEPARATOR, + namespaceSeparator)) + .namespace(ns)) + .isEqualTo("v1/ws/catalog/namespaces/" + namespace); + + assertThat( + ResourcePaths.forCatalogProperties( + ImmutableMap.of(RESTCatalogProperties.NAMESPACE_SEPARATOR, namespaceSeparator)) + .namespace(ns)) + .isEqualTo("v1/namespaces/" + namespace); + } + + @ParameterizedTest + @ValueSource(strings = {"%1F", "%2D", "%2E"}) + public void testNamespaceWithDot(String namespaceSeparator) { + Namespace ns = Namespace.of("n.s", "a.b"); + String namespace = String.format("n.s%sa.b", namespaceSeparator); + assertThat( + ResourcePaths.forCatalogProperties( + ImmutableMap.of( + "prefix", + prefix, + RESTCatalogProperties.NAMESPACE_SEPARATOR, + namespaceSeparator)) + .namespace(ns)) + .isEqualTo("v1/ws/catalog/namespaces/" + namespace); + + assertThat( + ResourcePaths.forCatalogProperties( + ImmutableMap.of(RESTCatalogProperties.NAMESPACE_SEPARATOR, namespaceSeparator)) + .namespace(ns)) + .isEqualTo("v1/namespaces/" + namespace); + } + @Test public void testNamespaceProperties() { Namespace ns = Namespace.of("ns"); From b2f3a4ce7bcb21b966cd3928ae964cc98b9cbdf5 Mon Sep 17 00:00:00 2001 From: pvary Date: Tue, 9 Dec 2025 13:31:27 +0100 Subject: [PATCH 066/201] Spark: Backport ORC vectorized reader to use the delete filter (#14794) backports #14746 --- .../vectorized/VectorizedSparkOrcReaders.java | 4 +- .../iceberg/spark/data/TestHelpers.java | 71 +++++++++++++++++ .../data/TestSparkOrcReadMetadataColumns.java | 54 ++++++++++--- .../TestSparkParquetReadMetadataColumns.java | 78 ++----------------- .../vectorized/VectorizedSparkOrcReaders.java | 4 +- .../iceberg/spark/data/TestHelpers.java | 71 +++++++++++++++++ .../data/TestSparkOrcReadMetadataColumns.java | 54 ++++++++++--- .../TestSparkParquetReadMetadataColumns.java | 78 ++----------------- 8 files changed, 250 insertions(+), 164 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java index 76e0ee381143..8dceb075e604 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java @@ -462,7 +462,9 @@ public ColumnVector convert( } else if (field.equals(MetadataColumns.ROW_POSITION)) { fieldVectors.add(new RowPositionColumnVector(batchOffsetInFile)); } else if (field.equals(MetadataColumns.IS_DELETED)) { - fieldVectors.add(new ConstantColumnVector(field.type(), batchSize, false)); + DeletedColumnVector deletedVector = new DeletedColumnVector(field.type()); + deletedVector.setValue(new boolean[batchSize]); + fieldVectors.add(deletedVector); } else { fieldVectors.add( fieldConverters diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 120d6eeb1730..72f9a3660965 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -52,10 +52,16 @@ import org.apache.iceberg.ManifestFile; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableScan; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.deletes.DeleteCounter; +import org.apache.iceberg.deletes.PositionDeleteIndex; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Type; @@ -895,4 +901,69 @@ public static Dataset selectNonDerived(Dataset metadataTable) { public static Types.StructType nonDerivedSchema(Dataset metadataTable) { return SparkSchemaUtil.convert(TestHelpers.selectNonDerived(metadataTable).schema()).asStruct(); } + + public static class CustomizedDeleteFilter extends DeleteFilter { + private final boolean hasDeletes; + + protected CustomizedDeleteFilter( + boolean hasDeletes, Schema tableSchema, Schema projectedSchema) { + super("", List.of(), tableSchema, projectedSchema, new DeleteCounter(), true); + this.hasDeletes = hasDeletes; + } + + @Override + protected StructLike asStructLike(InternalRow record) { + return null; + } + + @Override + protected InputFile getInputFile(String location) { + return null; + } + + @Override + public boolean hasPosDeletes() { + return hasDeletes; + } + + @Override + public PositionDeleteIndex deletedRowPositions() { + PositionDeleteIndex deletedRowPos = new CustomizedPositionDeleteIndex(); + if (hasDeletes) { + deletedRowPos.delete(98, 103); + } + + return deletedRowPos; + } + } + + public static class CustomizedPositionDeleteIndex implements PositionDeleteIndex { + private final Set deleteIndex; + + private CustomizedPositionDeleteIndex() { + deleteIndex = Sets.newHashSet(); + } + + @Override + public void delete(long position) { + deleteIndex.add(position); + } + + @Override + public void delete(long posStart, long posEnd) { + for (long l = posStart; l < posEnd; l++) { + delete(l); + } + } + + @Override + public boolean isDeleted(long position) { + return deleteIndex.contains(position); + } + + @Override + public boolean isEmpty() { + return deleteIndex.isEmpty(); + } + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java index 9d725250d3d2..13acaa1e3a7b 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -36,6 +37,7 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -46,6 +48,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; +import org.apache.iceberg.spark.source.BatchReaderUtil; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.orc.OrcConf; @@ -74,7 +77,11 @@ public class TestSparkOrcReadMetadataColumns { MetadataColumns.ROW_POSITION, MetadataColumns.IS_DELETED); + private static final DeleteFilter NO_DELETES_FILTER = + new TestHelpers.CustomizedDeleteFilter(false, DATA_SCHEMA, PROJECTION_SCHEMA); + private static final int NUM_ROWS = 1000; + private static final int RECORDS_PER_BATCH = 10; private static final List DATA_ROWS; private static final List EXPECTED_ROWS; @@ -128,13 +135,35 @@ public void writeFile() throws IOException { @TestTemplate public void testReadRowNumbers() throws IOException { - readAndValidate(null, null, null, EXPECTED_ROWS); + readAndValidate(null, null, null, EXPECTED_ROWS, NO_DELETES_FILTER); + } + + @TestTemplate + public void testReadRowNumbersWithDelete() throws IOException { + assumeThat(vectorized).isTrue(); + + List expectedRowsAfterDelete = Lists.newArrayList(); + EXPECTED_ROWS.forEach(row -> expectedRowsAfterDelete.add(row.copy())); + // remove row at position 98, 99, 100, 101, 102, this crosses two row groups [0, 100) and [100, + // 200) + for (int i = 98; i <= 102; i++) { + expectedRowsAfterDelete.get(i).update(3, true); + } + + DeleteFilter deleteFilter = + new TestHelpers.CustomizedDeleteFilter(true, DATA_SCHEMA, PROJECTION_SCHEMA); + + readAndValidate(null, null, null, expectedRowsAfterDelete, deleteFilter); } @TestTemplate public void testReadRowNumbersWithFilter() throws IOException { readAndValidate( - Expressions.greaterThanOrEqual("id", 500), null, null, EXPECTED_ROWS.subList(500, 1000)); + Expressions.greaterThanOrEqual("id", 500), + null, + null, + EXPECTED_ROWS.subList(500, 1000), + NO_DELETES_FILTER); } @TestTemplate @@ -157,12 +186,17 @@ public void testReadRowNumbersWithSplits() throws IOException { null, splitOffsets.get(i), splitLengths.get(i), - EXPECTED_ROWS.subList(i * 100, (i + 1) * 100)); + EXPECTED_ROWS.subList(i * 100, (i + 1) * 100), + NO_DELETES_FILTER); } } private void readAndValidate( - Expression filter, Long splitStart, Long splitLength, List expected) + Expression filter, + Long splitStart, + Long splitLength, + List expected, + DeleteFilter deleteFilter) throws IOException { Schema projectionWithoutMetadataFields = TypeUtil.selectNot(PROJECTION_SCHEMA, MetadataColumns.metadataFieldIds()); @@ -173,10 +207,12 @@ private void readAndValidate( if (vectorized) { builder = - builder.createBatchedReaderFunc( - readOrcSchema -> - VectorizedSparkOrcReaders.buildReader( - PROJECTION_SCHEMA, readOrcSchema, ImmutableMap.of())); + builder + .recordsPerBatch(RECORDS_PER_BATCH) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + PROJECTION_SCHEMA, readOrcSchema, ImmutableMap.of())); } else { builder = builder.createReaderFunc( @@ -192,7 +228,7 @@ private void readAndValidate( } if (vectorized) { - reader = batchesToRows(builder.build()); + reader = batchesToRows(BatchReaderUtil.applyDeleteFilter(builder.build(), deleteFilter)); } else { reader = builder.build(); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java index ccd783915c62..e2e5a98ccb8b 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java @@ -26,7 +26,6 @@ import java.io.IOException; import java.util.Iterator; import java.util.List; -import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.Files; @@ -35,21 +34,16 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; import org.apache.iceberg.data.DeleteFilter; -import org.apache.iceberg.deletes.DeleteCounter; -import org.apache.iceberg.deletes.PositionDeleteIndex; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.InputFile; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.parquet.ParquetSchemaUtil; 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.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.spark.source.BatchReaderUtil; @@ -183,7 +177,8 @@ public void testReadRowNumbersWithDelete() throws IOException { Parquet.ReadBuilder builder = Parquet.read(Files.localInput(testFile)).project(PROJECTION_SCHEMA); - DeleteFilter deleteFilter = new TestDeleteFilter(true); + DeleteFilter deleteFilter = + new TestHelpers.CustomizedDeleteFilter(true, DATA_SCHEMA, PROJECTION_SCHEMA); builder.createBatchedReaderFunc( fileSchema -> @@ -194,70 +189,6 @@ public void testReadRowNumbersWithDelete() throws IOException { validate(expectedRowsAfterDelete, builder, deleteFilter); } - private static class TestDeleteFilter extends DeleteFilter { - private final boolean hasDeletes; - - protected TestDeleteFilter(boolean hasDeletes) { - super("", List.of(), DATA_SCHEMA, PROJECTION_SCHEMA, new DeleteCounter(), true); - this.hasDeletes = hasDeletes; - } - - @Override - protected StructLike asStructLike(InternalRow record) { - return null; - } - - @Override - protected InputFile getInputFile(String location) { - return null; - } - - @Override - public boolean hasPosDeletes() { - return hasDeletes; - } - - @Override - public PositionDeleteIndex deletedRowPositions() { - PositionDeleteIndex deletedRowPos = new CustomizedPositionDeleteIndex(); - if (hasDeletes) { - deletedRowPos.delete(98, 103); - } - - return deletedRowPos; - } - } - - private static class CustomizedPositionDeleteIndex implements PositionDeleteIndex { - private final Set deleteIndex; - - private CustomizedPositionDeleteIndex() { - deleteIndex = Sets.newHashSet(); - } - - @Override - public void delete(long position) { - deleteIndex.add(position); - } - - @Override - public void delete(long posStart, long posEnd) { - for (long l = posStart; l < posEnd; l++) { - delete(l); - } - } - - @Override - public boolean isDeleted(long position) { - return deleteIndex.contains(position); - } - - @Override - public boolean isEmpty() { - return deleteIndex.isEmpty(); - } - } - @TestTemplate public void testReadRowNumbersWithFilter() throws IOException { // current iceberg supports row group filter. @@ -314,7 +245,10 @@ private void readAndValidate( builder = builder.split(splitStart, splitLength); } - validate(expected, builder, new TestDeleteFilter(false)); + validate( + expected, + builder, + new TestHelpers.CustomizedDeleteFilter(false, DATA_SCHEMA, PROJECTION_SCHEMA)); } private void validate( diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java index 5f68c233f60d..4f324239881e 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java @@ -462,7 +462,9 @@ public ColumnVector convert( } else if (field.equals(MetadataColumns.ROW_POSITION)) { fieldVectors.add(new RowPositionColumnVector(batchOffsetInFile)); } else if (field.equals(MetadataColumns.IS_DELETED)) { - fieldVectors.add(new ConstantColumnVector(field.type(), batchSize, false)); + DeletedColumnVector deletedVector = new DeletedColumnVector(field.type()); + deletedVector.setValue(new boolean[batchSize]); + fieldVectors.add(deletedVector); } else if (field.type().equals(Types.UnknownType.get())) { fieldVectors.add(new ConstantColumnVector(field.type(), batchSize, null)); } else { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 82ebb1d95016..dae8612f7db6 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -52,10 +52,16 @@ import org.apache.iceberg.ManifestFile; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableScan; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.deletes.DeleteCounter; +import org.apache.iceberg.deletes.PositionDeleteIndex; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Type; @@ -876,4 +882,69 @@ public static Dataset selectNonDerived(Dataset metadataTable) { public static Types.StructType nonDerivedSchema(Dataset metadataTable) { return SparkSchemaUtil.convert(TestHelpers.selectNonDerived(metadataTable).schema()).asStruct(); } + + public static class CustomizedDeleteFilter extends DeleteFilter { + private final boolean hasDeletes; + + protected CustomizedDeleteFilter( + boolean hasDeletes, Schema tableSchema, Schema projectedSchema) { + super("", List.of(), tableSchema, projectedSchema, new DeleteCounter(), true); + this.hasDeletes = hasDeletes; + } + + @Override + protected StructLike asStructLike(InternalRow record) { + return null; + } + + @Override + protected InputFile getInputFile(String location) { + return null; + } + + @Override + public boolean hasPosDeletes() { + return hasDeletes; + } + + @Override + public PositionDeleteIndex deletedRowPositions() { + PositionDeleteIndex deletedRowPos = new CustomizedPositionDeleteIndex(); + if (hasDeletes) { + deletedRowPos.delete(98, 103); + } + + return deletedRowPos; + } + } + + public static class CustomizedPositionDeleteIndex implements PositionDeleteIndex { + private final Set deleteIndex; + + private CustomizedPositionDeleteIndex() { + deleteIndex = Sets.newHashSet(); + } + + @Override + public void delete(long position) { + deleteIndex.add(position); + } + + @Override + public void delete(long posStart, long posEnd) { + for (long l = posStart; l < posEnd; l++) { + delete(l); + } + } + + @Override + public boolean isDeleted(long position) { + return deleteIndex.contains(position); + } + + @Override + public boolean isEmpty() { + return deleteIndex.isEmpty(); + } + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java index 9d725250d3d2..13acaa1e3a7b 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -36,6 +37,7 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -46,6 +48,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; +import org.apache.iceberg.spark.source.BatchReaderUtil; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.orc.OrcConf; @@ -74,7 +77,11 @@ public class TestSparkOrcReadMetadataColumns { MetadataColumns.ROW_POSITION, MetadataColumns.IS_DELETED); + private static final DeleteFilter NO_DELETES_FILTER = + new TestHelpers.CustomizedDeleteFilter(false, DATA_SCHEMA, PROJECTION_SCHEMA); + private static final int NUM_ROWS = 1000; + private static final int RECORDS_PER_BATCH = 10; private static final List DATA_ROWS; private static final List EXPECTED_ROWS; @@ -128,13 +135,35 @@ public void writeFile() throws IOException { @TestTemplate public void testReadRowNumbers() throws IOException { - readAndValidate(null, null, null, EXPECTED_ROWS); + readAndValidate(null, null, null, EXPECTED_ROWS, NO_DELETES_FILTER); + } + + @TestTemplate + public void testReadRowNumbersWithDelete() throws IOException { + assumeThat(vectorized).isTrue(); + + List expectedRowsAfterDelete = Lists.newArrayList(); + EXPECTED_ROWS.forEach(row -> expectedRowsAfterDelete.add(row.copy())); + // remove row at position 98, 99, 100, 101, 102, this crosses two row groups [0, 100) and [100, + // 200) + for (int i = 98; i <= 102; i++) { + expectedRowsAfterDelete.get(i).update(3, true); + } + + DeleteFilter deleteFilter = + new TestHelpers.CustomizedDeleteFilter(true, DATA_SCHEMA, PROJECTION_SCHEMA); + + readAndValidate(null, null, null, expectedRowsAfterDelete, deleteFilter); } @TestTemplate public void testReadRowNumbersWithFilter() throws IOException { readAndValidate( - Expressions.greaterThanOrEqual("id", 500), null, null, EXPECTED_ROWS.subList(500, 1000)); + Expressions.greaterThanOrEqual("id", 500), + null, + null, + EXPECTED_ROWS.subList(500, 1000), + NO_DELETES_FILTER); } @TestTemplate @@ -157,12 +186,17 @@ public void testReadRowNumbersWithSplits() throws IOException { null, splitOffsets.get(i), splitLengths.get(i), - EXPECTED_ROWS.subList(i * 100, (i + 1) * 100)); + EXPECTED_ROWS.subList(i * 100, (i + 1) * 100), + NO_DELETES_FILTER); } } private void readAndValidate( - Expression filter, Long splitStart, Long splitLength, List expected) + Expression filter, + Long splitStart, + Long splitLength, + List expected, + DeleteFilter deleteFilter) throws IOException { Schema projectionWithoutMetadataFields = TypeUtil.selectNot(PROJECTION_SCHEMA, MetadataColumns.metadataFieldIds()); @@ -173,10 +207,12 @@ private void readAndValidate( if (vectorized) { builder = - builder.createBatchedReaderFunc( - readOrcSchema -> - VectorizedSparkOrcReaders.buildReader( - PROJECTION_SCHEMA, readOrcSchema, ImmutableMap.of())); + builder + .recordsPerBatch(RECORDS_PER_BATCH) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + PROJECTION_SCHEMA, readOrcSchema, ImmutableMap.of())); } else { builder = builder.createReaderFunc( @@ -192,7 +228,7 @@ private void readAndValidate( } if (vectorized) { - reader = batchesToRows(builder.build()); + reader = batchesToRows(BatchReaderUtil.applyDeleteFilter(builder.build(), deleteFilter)); } else { reader = builder.build(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java index ccd783915c62..e2e5a98ccb8b 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java @@ -26,7 +26,6 @@ import java.io.IOException; import java.util.Iterator; import java.util.List; -import java.util.Set; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.Files; @@ -35,21 +34,16 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; import org.apache.iceberg.data.DeleteFilter; -import org.apache.iceberg.deletes.DeleteCounter; -import org.apache.iceberg.deletes.PositionDeleteIndex; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.InputFile; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.parquet.ParquetSchemaUtil; 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.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.spark.source.BatchReaderUtil; @@ -183,7 +177,8 @@ public void testReadRowNumbersWithDelete() throws IOException { Parquet.ReadBuilder builder = Parquet.read(Files.localInput(testFile)).project(PROJECTION_SCHEMA); - DeleteFilter deleteFilter = new TestDeleteFilter(true); + DeleteFilter deleteFilter = + new TestHelpers.CustomizedDeleteFilter(true, DATA_SCHEMA, PROJECTION_SCHEMA); builder.createBatchedReaderFunc( fileSchema -> @@ -194,70 +189,6 @@ public void testReadRowNumbersWithDelete() throws IOException { validate(expectedRowsAfterDelete, builder, deleteFilter); } - private static class TestDeleteFilter extends DeleteFilter { - private final boolean hasDeletes; - - protected TestDeleteFilter(boolean hasDeletes) { - super("", List.of(), DATA_SCHEMA, PROJECTION_SCHEMA, new DeleteCounter(), true); - this.hasDeletes = hasDeletes; - } - - @Override - protected StructLike asStructLike(InternalRow record) { - return null; - } - - @Override - protected InputFile getInputFile(String location) { - return null; - } - - @Override - public boolean hasPosDeletes() { - return hasDeletes; - } - - @Override - public PositionDeleteIndex deletedRowPositions() { - PositionDeleteIndex deletedRowPos = new CustomizedPositionDeleteIndex(); - if (hasDeletes) { - deletedRowPos.delete(98, 103); - } - - return deletedRowPos; - } - } - - private static class CustomizedPositionDeleteIndex implements PositionDeleteIndex { - private final Set deleteIndex; - - private CustomizedPositionDeleteIndex() { - deleteIndex = Sets.newHashSet(); - } - - @Override - public void delete(long position) { - deleteIndex.add(position); - } - - @Override - public void delete(long posStart, long posEnd) { - for (long l = posStart; l < posEnd; l++) { - delete(l); - } - } - - @Override - public boolean isDeleted(long position) { - return deleteIndex.contains(position); - } - - @Override - public boolean isEmpty() { - return deleteIndex.isEmpty(); - } - } - @TestTemplate public void testReadRowNumbersWithFilter() throws IOException { // current iceberg supports row group filter. @@ -314,7 +245,10 @@ private void readAndValidate( builder = builder.split(splitStart, splitLength); } - validate(expected, builder, new TestDeleteFilter(false)); + validate( + expected, + builder, + new TestHelpers.CustomizedDeleteFilter(false, DATA_SCHEMA, PROJECTION_SCHEMA)); } private void validate( From 0547af078c9d21b8642a4efcb3cbc85756547f22 Mon Sep 17 00:00:00 2001 From: GuoYu <511955993@qq.com> Date: Tue, 9 Dec 2025 22:56:28 +0800 Subject: [PATCH 067/201] Flink: Fix write unknown type to ORC exception and add ut for unknown type (#14761) --- .../iceberg/flink/data/FlinkOrcWriters.java | 19 +- .../flink/data/FlinkSchemaVisitor.java | 4 +- .../iceberg/flink/TestFlinkUnknownType.java | 179 ++++++++++++++++++ .../flink/source/reader/ReaderUtil.java | 10 +- 4 files changed, 206 insertions(+), 6 deletions(-) create mode 100644 flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUnknownType.java diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java index afce2cda1db1..684842aa099c 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java @@ -30,6 +30,7 @@ import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; import org.apache.iceberg.FieldMetrics; import org.apache.iceberg.data.orc.GenericOrcWriters; import org.apache.iceberg.flink.FlinkRowData; @@ -93,7 +94,19 @@ static OrcValueWriter map( } static OrcValueWriter struct(List> writers, List types) { - return new RowDataWriter(writers, types); + int[] fieldIndexes = new int[writers.size()]; + int fieldIndex = 0; + List logicalTypes = Lists.newArrayList(); + for (int i = 0; i < types.size(); i += 1) { + LogicalType logicalType = types.get(i); + if (!logicalType.is(LogicalTypeRoot.NULL)) { + fieldIndexes[fieldIndex] = i; + fieldIndex += 1; + logicalTypes.add(logicalType); + } + } + + return new RowDataWriter(fieldIndexes, writers, logicalTypes); } private static class StringWriter implements OrcValueWriter { @@ -294,12 +307,12 @@ public Stream> metrics() { static class RowDataWriter extends GenericOrcWriters.StructWriter { private final List fieldGetters; - RowDataWriter(List> writers, List types) { + RowDataWriter(int[] fieldIndexes, List> writers, List types) { super(writers); this.fieldGetters = Lists.newArrayListWithExpectedSize(types.size()); for (int i = 0; i < types.size(); i++) { - fieldGetters.add(FlinkRowData.createFieldGetter(types.get(i), i)); + fieldGetters.add(FlinkRowData.createFieldGetter(types.get(i), fieldIndexes[i])); } } diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java index ba4e1a7a7aec..1440fde3248c 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java @@ -106,7 +106,9 @@ private static T visitRecord( visitor.beforeField(iField); try { - results.add(visit(fieldFlinkType, iField.type(), visitor)); + if (iField.type() != Types.UnknownType.get()) { + results.add(visit(fieldFlinkType, iField.type(), visitor)); + } } finally { visitor.afterField(iField); } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUnknownType.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUnknownType.java new file mode 100644 index 000000000000..aaf0ebda0e2b --- /dev/null +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUnknownType.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; +import org.apache.iceberg.flink.sink.TaskWriterFactory; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.reader.ReaderUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.TaskWriter; +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.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +class TestFlinkUnknownType { + private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; + private static final Schema SCHEMA_WITH_UNKNOWN_COL = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "unknown_col", Types.UnknownType.get()), + Types.NestedField.optional(4, "data1", Types.StringType.get()))); + private static final List EXCEPTED_ROW_DATA = + Lists.newArrayList( + GenericRowData.of(1, StringData.fromString("data"), null, StringData.fromString("data1")), + GenericRowData.of( + 2, StringData.fromString("data"), null, StringData.fromString("data1"))); + private static final List EXPECTED_RECORDS = exceptedRecords(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @TempDir private Path warehouseDir; + + @Parameter private FileFormat fileFormat; + + private Table table; + + @Parameters(name = "fileFormat={0}") + public static Iterable parameters() { + return ImmutableList.of( + new Object[] {FileFormat.PARQUET}, + new Object[] {FileFormat.AVRO}, + new Object[] {FileFormat.ORC}); + } + + @BeforeEach + public void before() { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SCHEMA_WITH_UNKNOWN_COL, + PartitionSpec.unpartitioned(), + null, + ImmutableMap.of("format-version", "3", "write.format.default", fileFormat.name())); + } + + @TestTemplate + void testV3TableUnknownTypeRead() throws Exception { + new GenericAppenderHelper(table, fileFormat, warehouseDir).appendToTable(EXPECTED_RECORDS); + table.refresh(); + + List genericRowData = Lists.newArrayList(); + CloseableIterable combinedScanTasks = table.newScan().planTasks(); + for (CombinedScanTask combinedScanTask : combinedScanTasks) { + DataIterator dataIterator = + ReaderUtil.createDataIterator(combinedScanTask, table.schema(), table.schema()); + while (dataIterator.hasNext()) { + GenericRowData rowData = (GenericRowData) dataIterator.next(); + genericRowData.add( + GenericRowData.of( + rowData.getInt(0), + rowData.getString(1), + rowData.getField(2), + rowData.getString(3))); + } + } + + assertThat(genericRowData).containsExactlyInAnyOrderElementsOf(EXCEPTED_ROW_DATA); + } + + @TestTemplate + void testV3TableUnknownTypeWrite() throws Exception { + try (TaskWriter taskWriter = createTaskWriter()) { + for (GenericRowData rowData : EXCEPTED_ROW_DATA) { + taskWriter.write(rowData); + } + + taskWriter.close(); + AppendFiles appendFiles = table.newAppend(); + for (DataFile dataFile : taskWriter.dataFiles()) { + appendFiles.appendFile(dataFile); + } + + appendFiles.commit(); + List records = SimpleDataUtil.tableRecords(table); + assertThat(records).containsExactlyInAnyOrderElementsOf(exceptedRecords()); + } + } + + private TaskWriter createTaskWriter() { + RowType flinkWriteType = FlinkSchemaUtil.convert(table.schema()); + TaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + table, flinkWriteType, TARGET_FILE_SIZE, fileFormat, table.properties(), null, false); + taskWriterFactory.initialize(1, 1); + return taskWriterFactory.create(); + } + + private static List exceptedRecords() { + GenericRecord record = GenericRecord.create(SCHEMA_WITH_UNKNOWN_COL); + ImmutableList.Builder builder = ImmutableList.builder(); + EXCEPTED_ROW_DATA.forEach( + recordData -> { + GenericRecord copy = record.copy(); + for (int i = 0; i < recordData.getArity(); i++) { + Object field = recordData.getField(i); + if (field instanceof StringData) { + copy.set(i, recordData.getField(i).toString()); + } else { + copy.set(i, recordData.getField(i)); + } + } + + builder.add(copy); + }); + + return builder.build(); + } +} diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index 332244732928..3b094ba02298 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.List; import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseCombinedScanTask; import org.apache.iceberg.BaseFileScanTask; import org.apache.iceberg.CombinedScanTask; @@ -78,11 +79,16 @@ public static FileScanTask createFileTask( } public static DataIterator createDataIterator(CombinedScanTask combinedTask) { + return createDataIterator(combinedTask, TestFixtures.SCHEMA, TestFixtures.SCHEMA); + } + + public static DataIterator createDataIterator( + CombinedScanTask combinedTask, Schema tableSchema, Schema projectSchema) { return new DataIterator<>( new RowDataFileScanTaskReader( - TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()), + tableSchema, projectSchema, null, true, Collections.emptyList()), combinedTask, - new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), + new HadoopFileIO(new Configuration()), PlaintextEncryptionManager.instance()); } From 7e41316832458217548c9b5ff296da28d3e0ce33 Mon Sep 17 00:00:00 2001 From: GuoYu <511955993@qq.com> Date: Wed, 10 Dec 2025 00:02:46 +0800 Subject: [PATCH 068/201] Flink: Backport fix write unknown type to ORC exception and add ut for unknown type (#14806) backport of #14761 --- .../iceberg/flink/data/FlinkOrcWriters.java | 19 +- .../flink/data/FlinkSchemaVisitor.java | 4 +- .../iceberg/flink/TestFlinkUnknownType.java | 179 ++++++++++++++++++ .../flink/source/reader/ReaderUtil.java | 10 +- .../iceberg/flink/data/FlinkOrcWriters.java | 19 +- .../flink/data/FlinkSchemaVisitor.java | 4 +- .../iceberg/flink/TestFlinkUnknownType.java | 179 ++++++++++++++++++ .../flink/source/reader/ReaderUtil.java | 10 +- 8 files changed, 412 insertions(+), 12 deletions(-) create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUnknownType.java create mode 100644 flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUnknownType.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java index afce2cda1db1..684842aa099c 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java @@ -30,6 +30,7 @@ import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; import org.apache.iceberg.FieldMetrics; import org.apache.iceberg.data.orc.GenericOrcWriters; import org.apache.iceberg.flink.FlinkRowData; @@ -93,7 +94,19 @@ static OrcValueWriter map( } static OrcValueWriter struct(List> writers, List types) { - return new RowDataWriter(writers, types); + int[] fieldIndexes = new int[writers.size()]; + int fieldIndex = 0; + List logicalTypes = Lists.newArrayList(); + for (int i = 0; i < types.size(); i += 1) { + LogicalType logicalType = types.get(i); + if (!logicalType.is(LogicalTypeRoot.NULL)) { + fieldIndexes[fieldIndex] = i; + fieldIndex += 1; + logicalTypes.add(logicalType); + } + } + + return new RowDataWriter(fieldIndexes, writers, logicalTypes); } private static class StringWriter implements OrcValueWriter { @@ -294,12 +307,12 @@ public Stream> metrics() { static class RowDataWriter extends GenericOrcWriters.StructWriter { private final List fieldGetters; - RowDataWriter(List> writers, List types) { + RowDataWriter(int[] fieldIndexes, List> writers, List types) { super(writers); this.fieldGetters = Lists.newArrayListWithExpectedSize(types.size()); for (int i = 0; i < types.size(); i++) { - fieldGetters.add(FlinkRowData.createFieldGetter(types.get(i), i)); + fieldGetters.add(FlinkRowData.createFieldGetter(types.get(i), fieldIndexes[i])); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java index ba4e1a7a7aec..1440fde3248c 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java @@ -106,7 +106,9 @@ private static T visitRecord( visitor.beforeField(iField); try { - results.add(visit(fieldFlinkType, iField.type(), visitor)); + if (iField.type() != Types.UnknownType.get()) { + results.add(visit(fieldFlinkType, iField.type(), visitor)); + } } finally { visitor.afterField(iField); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUnknownType.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUnknownType.java new file mode 100644 index 000000000000..aaf0ebda0e2b --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUnknownType.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; +import org.apache.iceberg.flink.sink.TaskWriterFactory; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.reader.ReaderUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.TaskWriter; +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.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +class TestFlinkUnknownType { + private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; + private static final Schema SCHEMA_WITH_UNKNOWN_COL = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "unknown_col", Types.UnknownType.get()), + Types.NestedField.optional(4, "data1", Types.StringType.get()))); + private static final List EXCEPTED_ROW_DATA = + Lists.newArrayList( + GenericRowData.of(1, StringData.fromString("data"), null, StringData.fromString("data1")), + GenericRowData.of( + 2, StringData.fromString("data"), null, StringData.fromString("data1"))); + private static final List EXPECTED_RECORDS = exceptedRecords(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @TempDir private Path warehouseDir; + + @Parameter private FileFormat fileFormat; + + private Table table; + + @Parameters(name = "fileFormat={0}") + public static Iterable parameters() { + return ImmutableList.of( + new Object[] {FileFormat.PARQUET}, + new Object[] {FileFormat.AVRO}, + new Object[] {FileFormat.ORC}); + } + + @BeforeEach + public void before() { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SCHEMA_WITH_UNKNOWN_COL, + PartitionSpec.unpartitioned(), + null, + ImmutableMap.of("format-version", "3", "write.format.default", fileFormat.name())); + } + + @TestTemplate + void testV3TableUnknownTypeRead() throws Exception { + new GenericAppenderHelper(table, fileFormat, warehouseDir).appendToTable(EXPECTED_RECORDS); + table.refresh(); + + List genericRowData = Lists.newArrayList(); + CloseableIterable combinedScanTasks = table.newScan().planTasks(); + for (CombinedScanTask combinedScanTask : combinedScanTasks) { + DataIterator dataIterator = + ReaderUtil.createDataIterator(combinedScanTask, table.schema(), table.schema()); + while (dataIterator.hasNext()) { + GenericRowData rowData = (GenericRowData) dataIterator.next(); + genericRowData.add( + GenericRowData.of( + rowData.getInt(0), + rowData.getString(1), + rowData.getField(2), + rowData.getString(3))); + } + } + + assertThat(genericRowData).containsExactlyInAnyOrderElementsOf(EXCEPTED_ROW_DATA); + } + + @TestTemplate + void testV3TableUnknownTypeWrite() throws Exception { + try (TaskWriter taskWriter = createTaskWriter()) { + for (GenericRowData rowData : EXCEPTED_ROW_DATA) { + taskWriter.write(rowData); + } + + taskWriter.close(); + AppendFiles appendFiles = table.newAppend(); + for (DataFile dataFile : taskWriter.dataFiles()) { + appendFiles.appendFile(dataFile); + } + + appendFiles.commit(); + List records = SimpleDataUtil.tableRecords(table); + assertThat(records).containsExactlyInAnyOrderElementsOf(exceptedRecords()); + } + } + + private TaskWriter createTaskWriter() { + RowType flinkWriteType = FlinkSchemaUtil.convert(table.schema()); + TaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + table, flinkWriteType, TARGET_FILE_SIZE, fileFormat, table.properties(), null, false); + taskWriterFactory.initialize(1, 1); + return taskWriterFactory.create(); + } + + private static List exceptedRecords() { + GenericRecord record = GenericRecord.create(SCHEMA_WITH_UNKNOWN_COL); + ImmutableList.Builder builder = ImmutableList.builder(); + EXCEPTED_ROW_DATA.forEach( + recordData -> { + GenericRecord copy = record.copy(); + for (int i = 0; i < recordData.getArity(); i++) { + Object field = recordData.getField(i); + if (field instanceof StringData) { + copy.set(i, recordData.getField(i).toString()); + } else { + copy.set(i, recordData.getField(i)); + } + } + + builder.add(copy); + }); + + return builder.build(); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index 332244732928..3b094ba02298 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.List; import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseCombinedScanTask; import org.apache.iceberg.BaseFileScanTask; import org.apache.iceberg.CombinedScanTask; @@ -78,11 +79,16 @@ public static FileScanTask createFileTask( } public static DataIterator createDataIterator(CombinedScanTask combinedTask) { + return createDataIterator(combinedTask, TestFixtures.SCHEMA, TestFixtures.SCHEMA); + } + + public static DataIterator createDataIterator( + CombinedScanTask combinedTask, Schema tableSchema, Schema projectSchema) { return new DataIterator<>( new RowDataFileScanTaskReader( - TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()), + tableSchema, projectSchema, null, true, Collections.emptyList()), combinedTask, - new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), + new HadoopFileIO(new Configuration()), PlaintextEncryptionManager.instance()); } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java index afce2cda1db1..684842aa099c 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java @@ -30,6 +30,7 @@ import org.apache.flink.table.data.StringData; import org.apache.flink.table.data.TimestampData; import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeRoot; import org.apache.iceberg.FieldMetrics; import org.apache.iceberg.data.orc.GenericOrcWriters; import org.apache.iceberg.flink.FlinkRowData; @@ -93,7 +94,19 @@ static OrcValueWriter map( } static OrcValueWriter struct(List> writers, List types) { - return new RowDataWriter(writers, types); + int[] fieldIndexes = new int[writers.size()]; + int fieldIndex = 0; + List logicalTypes = Lists.newArrayList(); + for (int i = 0; i < types.size(); i += 1) { + LogicalType logicalType = types.get(i); + if (!logicalType.is(LogicalTypeRoot.NULL)) { + fieldIndexes[fieldIndex] = i; + fieldIndex += 1; + logicalTypes.add(logicalType); + } + } + + return new RowDataWriter(fieldIndexes, writers, logicalTypes); } private static class StringWriter implements OrcValueWriter { @@ -294,12 +307,12 @@ public Stream> metrics() { static class RowDataWriter extends GenericOrcWriters.StructWriter { private final List fieldGetters; - RowDataWriter(List> writers, List types) { + RowDataWriter(int[] fieldIndexes, List> writers, List types) { super(writers); this.fieldGetters = Lists.newArrayListWithExpectedSize(types.size()); for (int i = 0; i < types.size(); i++) { - fieldGetters.add(FlinkRowData.createFieldGetter(types.get(i), i)); + fieldGetters.add(FlinkRowData.createFieldGetter(types.get(i), fieldIndexes[i])); } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java index ba4e1a7a7aec..1440fde3248c 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java @@ -106,7 +106,9 @@ private static T visitRecord( visitor.beforeField(iField); try { - results.add(visit(fieldFlinkType, iField.type(), visitor)); + if (iField.type() != Types.UnknownType.get()) { + results.add(visit(fieldFlinkType, iField.type(), visitor)); + } } finally { visitor.afterField(iField); } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUnknownType.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUnknownType.java new file mode 100644 index 000000000000..aaf0ebda0e2b --- /dev/null +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUnknownType.java @@ -0,0 +1,179 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; +import org.apache.iceberg.flink.sink.TaskWriterFactory; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.reader.ReaderUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.TaskWriter; +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.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +class TestFlinkUnknownType { + private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; + private static final Schema SCHEMA_WITH_UNKNOWN_COL = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "unknown_col", Types.UnknownType.get()), + Types.NestedField.optional(4, "data1", Types.StringType.get()))); + private static final List EXCEPTED_ROW_DATA = + Lists.newArrayList( + GenericRowData.of(1, StringData.fromString("data"), null, StringData.fromString("data1")), + GenericRowData.of( + 2, StringData.fromString("data"), null, StringData.fromString("data1"))); + private static final List EXPECTED_RECORDS = exceptedRecords(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @TempDir private Path warehouseDir; + + @Parameter private FileFormat fileFormat; + + private Table table; + + @Parameters(name = "fileFormat={0}") + public static Iterable parameters() { + return ImmutableList.of( + new Object[] {FileFormat.PARQUET}, + new Object[] {FileFormat.AVRO}, + new Object[] {FileFormat.ORC}); + } + + @BeforeEach + public void before() { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SCHEMA_WITH_UNKNOWN_COL, + PartitionSpec.unpartitioned(), + null, + ImmutableMap.of("format-version", "3", "write.format.default", fileFormat.name())); + } + + @TestTemplate + void testV3TableUnknownTypeRead() throws Exception { + new GenericAppenderHelper(table, fileFormat, warehouseDir).appendToTable(EXPECTED_RECORDS); + table.refresh(); + + List genericRowData = Lists.newArrayList(); + CloseableIterable combinedScanTasks = table.newScan().planTasks(); + for (CombinedScanTask combinedScanTask : combinedScanTasks) { + DataIterator dataIterator = + ReaderUtil.createDataIterator(combinedScanTask, table.schema(), table.schema()); + while (dataIterator.hasNext()) { + GenericRowData rowData = (GenericRowData) dataIterator.next(); + genericRowData.add( + GenericRowData.of( + rowData.getInt(0), + rowData.getString(1), + rowData.getField(2), + rowData.getString(3))); + } + } + + assertThat(genericRowData).containsExactlyInAnyOrderElementsOf(EXCEPTED_ROW_DATA); + } + + @TestTemplate + void testV3TableUnknownTypeWrite() throws Exception { + try (TaskWriter taskWriter = createTaskWriter()) { + for (GenericRowData rowData : EXCEPTED_ROW_DATA) { + taskWriter.write(rowData); + } + + taskWriter.close(); + AppendFiles appendFiles = table.newAppend(); + for (DataFile dataFile : taskWriter.dataFiles()) { + appendFiles.appendFile(dataFile); + } + + appendFiles.commit(); + List records = SimpleDataUtil.tableRecords(table); + assertThat(records).containsExactlyInAnyOrderElementsOf(exceptedRecords()); + } + } + + private TaskWriter createTaskWriter() { + RowType flinkWriteType = FlinkSchemaUtil.convert(table.schema()); + TaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + table, flinkWriteType, TARGET_FILE_SIZE, fileFormat, table.properties(), null, false); + taskWriterFactory.initialize(1, 1); + return taskWriterFactory.create(); + } + + private static List exceptedRecords() { + GenericRecord record = GenericRecord.create(SCHEMA_WITH_UNKNOWN_COL); + ImmutableList.Builder builder = ImmutableList.builder(); + EXCEPTED_ROW_DATA.forEach( + recordData -> { + GenericRecord copy = record.copy(); + for (int i = 0; i < recordData.getArity(); i++) { + Object field = recordData.getField(i); + if (field instanceof StringData) { + copy.set(i, recordData.getField(i).toString()); + } else { + copy.set(i, recordData.getField(i)); + } + } + + builder.add(copy); + }); + + return builder.build(); + } +} diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index 332244732928..3b094ba02298 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -26,6 +26,7 @@ import java.util.Collections; import java.util.List; import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseCombinedScanTask; import org.apache.iceberg.BaseFileScanTask; import org.apache.iceberg.CombinedScanTask; @@ -78,11 +79,16 @@ public static FileScanTask createFileTask( } public static DataIterator createDataIterator(CombinedScanTask combinedTask) { + return createDataIterator(combinedTask, TestFixtures.SCHEMA, TestFixtures.SCHEMA); + } + + public static DataIterator createDataIterator( + CombinedScanTask combinedTask, Schema tableSchema, Schema projectSchema) { return new DataIterator<>( new RowDataFileScanTaskReader( - TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()), + tableSchema, projectSchema, null, true, Collections.emptyList()), combinedTask, - new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), + new HadoopFileIO(new Configuration()), PlaintextEncryptionManager.instance()); } From c8f4d5fee230dc9f240149938ecad84a9a979b5e Mon Sep 17 00:00:00 2001 From: pvary Date: Tue, 9 Dec 2025 20:30:53 +0100 Subject: [PATCH 069/201] Spark: Add comet reader test (#14807) --- .../TestParquetCometVectorizedScan.java | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java new file mode 100644 index 000000000000..5e1d88f8bcd4 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java @@ -0,0 +1,33 @@ +/* + * 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.junit.jupiter.api.BeforeAll; + +public class TestParquetCometVectorizedScan extends TestParquetScan { + @BeforeAll + public static void setComet() { + ScanTestBase.spark.conf().set("spark.sql.iceberg.parquet.reader-type", "COMET"); + } + + @Override + protected boolean vectorized() { + return true; + } +} From 03415dff66c7ba4efd34cea2bb7f3dbcb40bb67e Mon Sep 17 00:00:00 2001 From: pvary Date: Wed, 10 Dec 2025 00:13:05 +0100 Subject: [PATCH 070/201] Spark: Backport add comet reader test (#14809) --- .../TestParquetCometVectorizedScan.java | 33 +++++++++++++++++++ .../TestParquetCometVectorizedScan.java | 33 +++++++++++++++++++ 2 files changed, 66 insertions(+) create mode 100644 spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java new file mode 100644 index 000000000000..5e1d88f8bcd4 --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java @@ -0,0 +1,33 @@ +/* + * 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.junit.jupiter.api.BeforeAll; + +public class TestParquetCometVectorizedScan extends TestParquetScan { + @BeforeAll + public static void setComet() { + ScanTestBase.spark.conf().set("spark.sql.iceberg.parquet.reader-type", "COMET"); + } + + @Override + protected boolean vectorized() { + return true; + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java new file mode 100644 index 000000000000..5e1d88f8bcd4 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java @@ -0,0 +1,33 @@ +/* + * 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.junit.jupiter.api.BeforeAll; + +public class TestParquetCometVectorizedScan extends TestParquetScan { + @BeforeAll + public static void setComet() { + ScanTestBase.spark.conf().set("spark.sql.iceberg.parquet.reader-type", "COMET"); + } + + @Override + protected boolean vectorized() { + return true; + } +} From c6ba7a4c48cf1280eeff35829492fd7e6a5f6787 Mon Sep 17 00:00:00 2001 From: Beerelly Prudhvi Maharishi Date: Wed, 10 Dec 2025 05:18:18 +0530 Subject: [PATCH 071/201] GCS: Integrate GCSAnalyticsCore Library (#14333) GCS: add support for GCS analytics core library Co-authored-by: Ajay Yadav --- build.gradle | 3 + gcp-bundle/LICENSE | 6 + gcp-bundle/NOTICE | 23 ++ gcp-bundle/build.gradle | 1 + .../apache/iceberg/gcp/gcs/TestGcsFileIO.java | 246 ++++++++++++++++ .../org/apache/iceberg/gcp/GCPProperties.java | 10 + .../apache/iceberg/gcp/gcs/BaseGCSFile.java | 14 +- .../apache/iceberg/gcp/gcs/GCSInputFile.java | 57 +++- .../apache/iceberg/gcp/gcs/GCSOutputFile.java | 17 +- .../gcp/gcs/GcsInputStreamWrapper.java | 112 ++++++++ .../iceberg/gcp/gcs/PrefixedStorage.java | 69 ++++- .../iceberg/gcp/gcs/TestGcsInputFile.java | 270 ++++++++++++++++++ .../gcp/gcs/TestGcsInputStreamWrapper.java | 146 ++++++++++ .../iceberg/gcp/gcs/TestPrefixedStorage.java | 40 +++ gradle/libs.versions.toml | 2 + 15 files changed, 997 insertions(+), 19 deletions(-) create mode 100644 gcp/src/integration/java/org/apache/iceberg/gcp/gcs/TestGcsFileIO.java create mode 100644 gcp/src/main/java/org/apache/iceberg/gcp/gcs/GcsInputStreamWrapper.java create mode 100644 gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestGcsInputFile.java create mode 100644 gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestGcsInputStreamWrapper.java diff --git a/build.gradle b/build.gradle index 3220f8ba8d98..d740e6fb53e6 100644 --- a/build.gradle +++ b/build.gradle @@ -730,6 +730,7 @@ project(':iceberg-gcp') { compileOnly platform(libs.google.libraries.bom) compileOnly "com.google.cloud:google-cloud-storage" compileOnly "com.google.cloud:google-cloud-kms" + compileOnly(libs.gcs.analytics.core) testImplementation "com.google.cloud:google-cloud-nio" @@ -746,6 +747,8 @@ project(':iceberg-gcp') { testImplementation libs.mockserver.netty testImplementation libs.mockserver.client.java testImplementation libs.mockito.junit.jupiter + testImplementation libs.testcontainers + testImplementation libs.testcontainers.junit.jupiter } sourceSets { diff --git a/gcp-bundle/LICENSE b/gcp-bundle/LICENSE index 66d297673446..de9a4007e68f 100644 --- a/gcp-bundle/LICENSE +++ b/gcp-bundle/LICENSE @@ -547,6 +547,12 @@ License: Apache 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt -------------------------------------------------------------------------------- +Group: com.google.cloud.gcs.analytics Name: gcs-analytics-core Version: 1.1.2 +Project URL: https://github.com/GoogleCloudPlatform/gcs-analytics-core +License (from POM): Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + Group: com.google.escapevelocity Name: escapevelocity Version: 0.9 Project URL: https://github.com/google/escapevelocity License: Apache 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt diff --git a/gcp-bundle/NOTICE b/gcp-bundle/NOTICE index 2532bf7c1529..5ea9500ec678 100644 --- a/gcp-bundle/NOTICE +++ b/gcp-bundle/NOTICE @@ -290,6 +290,29 @@ NOTICE for Group: com.fasterxml.jackson.core Name: jackson-databind Version: 2 -------------------------------------------------------------------------------- +NOTICE for Group: com.google.cloud.gcs.analytics Name: gcs-analytics-core Version: 1.1.2 + +| # GCS Analytics Core +| +| GCS Analytics Core is a Java library designed to optimize analytics workloads on +| Google Cloud Storage (GCS). +| +| Copyright Google LLC +| +| Licensed 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. + +-------------------------------------------------------------------------------- + NOTICE for Group: com.google.escapevelocity Name: escapevelocity Version: 0.9 | Apache Velocity diff --git a/gcp-bundle/build.gradle b/gcp-bundle/build.gradle index 99e4359bd6d0..6ebe05ccdbce 100644 --- a/gcp-bundle/build.gradle +++ b/gcp-bundle/build.gradle @@ -29,6 +29,7 @@ project(":iceberg-gcp-bundle") { implementation "com.google.cloud:google-cloud-bigquery" implementation "com.google.cloud:google-cloud-core" implementation "com.google.cloud:google-cloud-kms" + implementation libs.gcs.analytics.core } shadowJar { diff --git a/gcp/src/integration/java/org/apache/iceberg/gcp/gcs/TestGcsFileIO.java b/gcp/src/integration/java/org/apache/iceberg/gcp/gcs/TestGcsFileIO.java new file mode 100644 index 000000000000..b377d24c6042 --- /dev/null +++ b/gcp/src/integration/java/org/apache/iceberg/gcp/gcs/TestGcsFileIO.java @@ -0,0 +1,246 @@ +/* + * 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.gcp.gcs; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.github.dockerjava.api.model.ExposedPort; +import com.github.dockerjava.api.model.HostConfig; +import com.github.dockerjava.api.model.PortBinding; +import com.github.dockerjava.api.model.Ports.Binding; +import com.google.cloud.NoCredentials; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobId; +import com.google.cloud.storage.BlobInfo; +import com.google.cloud.storage.BucketInfo; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageException; +import com.google.cloud.storage.StorageOptions; +import java.io.IOException; +import java.io.InputStream; +import java.time.Duration; +import java.util.List; +import java.util.Random; +import java.util.stream.Collectors; +import org.apache.iceberg.gcp.GCPProperties; +import org.apache.iceberg.io.FileInfo; +import org.apache.iceberg.io.IOUtil; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; +import org.testcontainers.junit.jupiter.Container; +import org.testcontainers.junit.jupiter.Testcontainers; + +@Testcontainers +public class TestGcsFileIO { + + private static final String BUCKET = "test-bucket"; + private static final String PROJECT_ID = "test-project"; + private static final int GCS_EMULATOR_PORT = 4443; + private static final Random RANDOM = new Random(1); + + @Container + private static final GenericContainer GCS_EMULATOR = + new GenericContainer<>("fsouza/fake-gcs-server:latest") + .withExposedPorts(GCS_EMULATOR_PORT) + .withCreateContainerCmdModifier( + cmd -> + cmd.withHostConfig( + new HostConfig() + .withPortBindings( + new PortBinding( + Binding.bindPort(GCS_EMULATOR_PORT), + new ExposedPort(GCS_EMULATOR_PORT))))) + .withCommand( + "-scheme", + "http", + "-external-url", + String.format("http://localhost:%d", GCS_EMULATOR_PORT)) + .waitingFor( + new HttpWaitStrategy() + .forPort(GCS_EMULATOR_PORT) + .forPath("/storage/v1/b") + .forStatusCode(200) + .withStartupTimeout(Duration.ofMinutes(2))); + + private GCSFileIO fileIO; + private static Storage storage; + + @BeforeAll + public static void beforeClass() { + String endpoint = String.format("http://localhost:%d", GCS_EMULATOR_PORT); + StorageOptions options = + StorageOptions.newBuilder() + .setProjectId(PROJECT_ID) + .setHost(endpoint) + .setCredentials(NoCredentials.getInstance()) + .build(); + storage = options.getService(); + storage.create(BucketInfo.of(BUCKET)); + } + + @AfterAll + public static void afterClass() { + if (storage != null) { + storage.delete(BUCKET); + } + } + + @BeforeEach + public void before() { + fileIO = new GCSFileIO(() -> storage); + fileIO.initialize(ImmutableMap.of()); + for (Blob blob : storage.list(BUCKET).iterateAll()) { + storage.delete(blob.getBlobId()); + } + } + + @AfterEach + public void after() { + for (Blob blob : storage.list(BUCKET).iterateAll()) { + storage.delete(blob.getBlobId()); + } + } + + @Test + public void newInputFileGcsAnalyticsCoreDisabled() throws IOException { + String location = String.format("gs://%s/path/to/file.txt", BUCKET); + byte[] expected = new byte[1024 * 1024]; + RANDOM.nextBytes(expected); + storage.create(BlobInfo.newBuilder(BlobId.fromGsUtilUri(location)).build(), expected); + InputFile in = fileIO.newInputFile(location); + byte[] actual = new byte[1024 * 1024]; + + try (InputStream is = in.newStream()) { + IOUtil.readFully(is, actual, 0, expected.length); + } + + assertThat(actual).isEqualTo(expected); + } + + @Test + public void newInputFileGcsAnalyticsCoreEnabled() throws IOException { + String location = String.format("gs://%s/path/to/file.txt", BUCKET); + byte[] expected = new byte[1024 * 1024]; + RANDOM.nextBytes(expected); + storage.create(BlobInfo.newBuilder(BlobId.fromGsUtilUri(location)).build(), expected); + fileIO.initialize( + ImmutableMap.of( + GCPProperties.GCS_ANALYTICS_CORE_ENABLED, "true", + GCPProperties.GCS_NO_AUTH, "true", + GCPProperties.GCS_SERVICE_HOST, + String.format("http://localhost:%d", GCS_EMULATOR_PORT))); + InputFile in = fileIO.newInputFile(location); + byte[] actual = new byte[1024 * 1024]; + + InputStream inputStream = in.newStream(); + try (InputStream is = inputStream) { + IOUtil.readFully(is, actual, 0, expected.length); + } + + assertThat(inputStream).isInstanceOf(GcsInputStreamWrapper.class); + assertThat(actual).isEqualTo(expected); + } + + @Test + public void deleteFiles() { + String prefix = "delete-files"; + List locations = Lists.newArrayList(); + for (int i = 0; i < 10; i++) { + String location = String.format("gs://%s/%s/file-%d.txt", BUCKET, prefix, i); + locations.add(location); + storage.create(BlobInfo.newBuilder(BlobId.fromGsUtilUri(location)).build(), new byte[] {1}); + } + + fileIO.deleteFiles(locations); + + for (String location : locations) { + assertThat(fileIO.newInputFile(location).exists()).isFalse(); + } + } + + @Test + public void listPrefix() { + String prefix = "list-prefix"; + String dir1 = String.format("gs://%s/%s/d1", BUCKET, prefix); + String dir2 = String.format("gs://%s/%s/d2", BUCKET, prefix); + storage.create( + BlobInfo.newBuilder(BlobId.fromGsUtilUri(dir1 + "/f1.txt")).build(), new byte[] {1}); + storage.create( + BlobInfo.newBuilder(BlobId.fromGsUtilUri(dir1 + "/f2.txt")).build(), new byte[] {1}); + storage.create( + BlobInfo.newBuilder(BlobId.fromGsUtilUri(dir2 + "/f3.txt")).build(), new byte[] {1}); + + List files = + Lists.newArrayList(fileIO.listPrefix(String.format("gs://%s/%s/", BUCKET, prefix))); + List paths = files.stream().map(FileInfo::location).collect(Collectors.toList()); + + assertThat(files).hasSize(3); + assertThat(paths).contains(dir1 + "/f1.txt", dir1 + "/f2.txt", dir2 + "/f3.txt"); + } + + @Test + public void deletePrefix() { + String prefixToDelete = String.format("gs://%s/delete-prefix/", BUCKET); + storage.create( + BlobInfo.newBuilder(BlobId.fromGsUtilUri(prefixToDelete + "f1.txt")).build(), + new byte[] {1}); + storage.create( + BlobInfo.newBuilder(BlobId.fromGsUtilUri(prefixToDelete + "f2.txt")).build(), + new byte[] {1}); + + int filesBeforeDelete = Lists.newArrayList(fileIO.listPrefix(prefixToDelete)).size(); + + fileIO.deletePrefix(prefixToDelete); + + assertThat(filesBeforeDelete).isEqualTo(2); + assertThat(Lists.newArrayList(fileIO.listPrefix(prefixToDelete))).isEmpty(); + } + + @Test + public void readMissingLocation() { + String location = String.format("gs://%s/path/to/data.parquet", BUCKET); + InputFile in = fileIO.newInputFile(location); + + assertThatThrownBy(() -> in.newStream().read()) + .isInstanceOf(IOException.class) + .hasCauseInstanceOf(StorageException.class) + .hasMessageContaining("404 Not Found"); + } + + @Test + public void deleteFile() { + String location = String.format("gs://%s/path/to/file.txt", BUCKET); + storage.create(BlobInfo.newBuilder(BlobId.fromGsUtilUri(location)).build(), new byte[] {1}); + InputFile in = fileIO.newInputFile(location); + assertThat(in.exists()).as("File should exist before delete").isTrue(); + + fileIO.deleteFile(in); + + assertThat(fileIO.newInputFile(location).exists()).isFalse(); + } +} diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java b/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java index d91601125c74..52abd850031a 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java @@ -56,6 +56,9 @@ public class GCPProperties implements Serializable { /** Configure the batch size used when deleting multiple files from a given GCS bucket */ public static final String GCS_DELETE_BATCH_SIZE = "gcs.delete.batch-size"; + /** Controls whether analytics core library is enabled or not. Defaults to false. */ + public static final String GCS_ANALYTICS_CORE_ENABLED = "gcs.analytics-core.enabled"; + /** * Max possible batch size for deletion. Currently, a max of 100 keys is advised, so we default to * a number below that. https://cloud.google.com/storage/docs/batch @@ -80,6 +83,7 @@ public class GCPProperties implements Serializable { private Date gcsOAuth2TokenExpiresAt; private String gcsOauth2RefreshCredentialsEndpoint; private boolean gcsOauth2RefreshCredentialsEnabled; + private boolean gcsAnalyticsCoreEnabled; private int gcsDeleteBatchSize = GCS_DELETE_BATCH_SIZE_DEFAULT; @@ -128,6 +132,8 @@ public GCPProperties(Map properties) { gcsDeleteBatchSize = PropertyUtil.propertyAsInt( properties, GCS_DELETE_BATCH_SIZE, GCS_DELETE_BATCH_SIZE_DEFAULT); + gcsAnalyticsCoreEnabled = + PropertyUtil.propertyAsBoolean(properties, GCS_ANALYTICS_CORE_ENABLED, false); } public Optional channelReadChunkSize() { @@ -189,4 +195,8 @@ public boolean oauth2RefreshCredentialsEnabled() { public Map properties() { return allProperties; } + + public boolean isGcsAnalyticsCoreEnabled() { + return gcsAnalyticsCoreEnabled; + } } diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/BaseGCSFile.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/BaseGCSFile.java index d34ea63eee82..4ece6a215615 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/BaseGCSFile.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/BaseGCSFile.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.gcp.gcs; +import com.google.cloud.gcs.analyticscore.client.GcsFileSystem; import com.google.cloud.storage.Blob; import com.google.cloud.storage.BlobId; import com.google.cloud.storage.Storage; @@ -27,13 +28,20 @@ abstract class BaseGCSFile { private final Storage storage; + private final GcsFileSystem gcsFileSystem; private final GCPProperties gcpProperties; private final BlobId blobId; private Blob metadata; private final MetricsContext metrics; - BaseGCSFile(Storage storage, BlobId blobId, GCPProperties gcpProperties, MetricsContext metrics) { + BaseGCSFile( + Storage storage, + GcsFileSystem gcsFileSystem, + BlobId blobId, + GCPProperties gcpProperties, + MetricsContext metrics) { this.storage = storage; + this.gcsFileSystem = gcsFileSystem; this.blobId = blobId; this.gcpProperties = gcpProperties; this.metrics = metrics; @@ -47,6 +55,10 @@ Storage storage() { return storage; } + GcsFileSystem gcsFileSystem() { + return gcsFileSystem; + } + URI uri() { return URI.create(blobId.toGsUtilUri()); } diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputFile.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputFile.java index 3022f1ecb451..497af03bcdaa 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputFile.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputFile.java @@ -18,14 +18,25 @@ */ package org.apache.iceberg.gcp.gcs; +import com.google.cloud.gcs.analyticscore.client.GcsFileInfo; +import com.google.cloud.gcs.analyticscore.client.GcsFileSystem; +import com.google.cloud.gcs.analyticscore.client.GcsItemId; +import com.google.cloud.gcs.analyticscore.client.GcsItemInfo; +import com.google.cloud.gcs.analyticscore.core.GoogleCloudStorageInputStream; import com.google.cloud.storage.BlobId; import com.google.cloud.storage.Storage; +import java.io.IOException; +import java.net.URI; import org.apache.iceberg.gcp.GCPProperties; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.SeekableInputStream; import org.apache.iceberg.metrics.MetricsContext; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class GCSInputFile extends BaseGCSFile implements InputFile { + private static final Logger LOG = LoggerFactory.getLogger(GCSInputFile.class); private Long blobSize; static GCSInputFile fromLocation( @@ -37,6 +48,7 @@ static GCSInputFile fromLocation( String location, long length, PrefixedStorage storage, MetricsContext metrics) { return new GCSInputFile( storage.storage(), + storage.gcsFileSystem(), BlobId.fromGsUtilUri(location), length > 0 ? length : null, storage.gcpProperties(), @@ -45,11 +57,12 @@ static GCSInputFile fromLocation( GCSInputFile( Storage storage, + GcsFileSystem gcsFileSystem, BlobId blobId, Long blobSize, GCPProperties gcpProperties, MetricsContext metrics) { - super(storage, blobId, gcpProperties, metrics); + super(storage, gcsFileSystem, blobId, gcpProperties, metrics); this.blobSize = blobSize; } @@ -64,6 +77,48 @@ public long getLength() { @Override public SeekableInputStream newStream() { + if (gcpProperties().isGcsAnalyticsCoreEnabled()) { + try { + return newGoogleCloudStorageInputStream(); + } catch (IOException e) { + LOG.error( + "Failed to create GCS analytics core input stream for {}, falling back to default.", + uri(), + e); + } + } + return new GCSInputStream(storage(), blobId(), blobSize, gcpProperties(), metrics()); } + + private SeekableInputStream newGoogleCloudStorageInputStream() throws IOException { + if (null == blobSize) { + return new GcsInputStreamWrapper( + GoogleCloudStorageInputStream.create(gcsFileSystem(), gcsItemId()), metrics()); + } + + return new GcsInputStreamWrapper( + GoogleCloudStorageInputStream.create(gcsFileSystem(), gcsFileInfo()), metrics()); + } + + private GcsItemId gcsItemId() { + BlobId blobId = blobId(); + GcsItemId.Builder builder = + GcsItemId.builder().setBucketName(blobId.getBucket()).setObjectName(blobId.getName()); + if (blobId.getGeneration() != null) { + builder.setContentGeneration(blobId.getGeneration()); + } + + return builder.build(); + } + + private GcsFileInfo gcsFileInfo() { + GcsItemId itemId = gcsItemId(); + GcsItemInfo itemInfo = GcsItemInfo.builder().setItemId(itemId).setSize(getLength()).build(); + return GcsFileInfo.builder() + .setItemInfo(itemInfo) + .setUri(URI.create(blobId().toGsUtilUri())) + .setAttributes(ImmutableMap.of()) + .build(); + } } diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputFile.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputFile.java index 34195aa71415..aa9f038db778 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputFile.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputFile.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.gcp.gcs; +import com.google.cloud.gcs.analyticscore.client.GcsFileSystem; import com.google.cloud.storage.BlobId; import com.google.cloud.storage.Storage; import java.io.IOException; @@ -34,12 +35,20 @@ class GCSOutputFile extends BaseGCSFile implements OutputFile { static GCSOutputFile fromLocation( String location, PrefixedStorage storage, MetricsContext metrics) { return new GCSOutputFile( - storage.storage(), BlobId.fromGsUtilUri(location), storage.gcpProperties(), metrics); + storage.storage(), + storage.gcsFileSystem(), + BlobId.fromGsUtilUri(location), + storage.gcpProperties(), + metrics); } GCSOutputFile( - Storage storage, BlobId blobId, GCPProperties gcpProperties, MetricsContext metrics) { - super(storage, blobId, gcpProperties, metrics); + Storage storage, + GcsFileSystem gcsFileSystem, + BlobId blobId, + GCPProperties gcpProperties, + MetricsContext metrics) { + super(storage, gcsFileSystem, blobId, gcpProperties, metrics); } /** @@ -68,6 +77,6 @@ public PositionOutputStream createOrOverwrite() { @Override public InputFile toInputFile() { - return new GCSInputFile(storage(), blobId(), null, gcpProperties(), metrics()); + return new GCSInputFile(storage(), gcsFileSystem(), blobId(), null, gcpProperties(), metrics()); } } diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GcsInputStreamWrapper.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GcsInputStreamWrapper.java new file mode 100644 index 000000000000..2e1dfdd73c08 --- /dev/null +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GcsInputStreamWrapper.java @@ -0,0 +1,112 @@ +/* + * 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.gcp.gcs; + +import com.google.api.client.util.Preconditions; +import com.google.cloud.gcs.analyticscore.client.GcsObjectRange; +import com.google.cloud.gcs.analyticscore.core.GoogleCloudStorageInputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.function.IntFunction; +import java.util.stream.Collectors; +import org.apache.iceberg.io.FileIOMetricsContext; +import org.apache.iceberg.io.FileRange; +import org.apache.iceberg.io.RangeReadable; +import org.apache.iceberg.io.SeekableInputStream; +import org.apache.iceberg.metrics.Counter; +import org.apache.iceberg.metrics.MetricsContext; + +class GcsInputStreamWrapper extends SeekableInputStream implements RangeReadable { + private final Counter readBytes; + private final Counter readOperations; + private final GoogleCloudStorageInputStream stream; + + GcsInputStreamWrapper(GoogleCloudStorageInputStream stream, MetricsContext metrics) { + Preconditions.checkArgument(null != stream, "Invalid input stream : null"); + this.stream = stream; + this.readBytes = metrics.counter(FileIOMetricsContext.READ_BYTES, MetricsContext.Unit.BYTES); + this.readOperations = metrics.counter(FileIOMetricsContext.READ_OPERATIONS); + } + + @Override + public long getPos() throws IOException { + return stream.getPos(); + } + + @Override + public void seek(long newPos) throws IOException { + stream.seek(newPos); + } + + @Override + public int read() throws IOException { + int readByte = stream.read(); + readBytes.increment(); + readOperations.increment(); + return readByte; + } + + @Override + public int read(byte[] b) throws IOException { + return read(b, 0, b.length); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + int bytesRead = stream.read(b, off, len); + if (bytesRead > 0) { + readBytes.increment(bytesRead); + } + readOperations.increment(); + return bytesRead; + } + + @Override + public void readFully(long position, byte[] buffer, int offset, int length) throws IOException { + stream.readFully(position, buffer, offset, length); + } + + @Override + public int readTail(byte[] buffer, int offset, int length) throws IOException { + return stream.readTail(buffer, offset, length); + } + + @Override + public void readVectored(List ranges, IntFunction allocate) + throws IOException { + List objectRanges = + ranges.stream() + .map( + fileRange -> + GcsObjectRange.builder() + .setOffset(fileRange.offset()) + .setLength(fileRange.length()) + .setByteBufferFuture(fileRange.byteBuffer()) + .build()) + .collect(Collectors.toList()); + + stream.readVectored(objectRanges, allocate); + } + + @Override + public void close() throws IOException { + stream.close(); + } +} diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/PrefixedStorage.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/PrefixedStorage.java index e9db60b149da..1f60db346d65 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/PrefixedStorage.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/PrefixedStorage.java @@ -19,7 +19,12 @@ package org.apache.iceberg.gcp.gcs; import com.google.api.gax.rpc.FixedHeaderProvider; +import com.google.auth.Credentials; import com.google.cloud.NoCredentials; +import com.google.cloud.gcs.analyticscore.client.GcsFileSystem; +import com.google.cloud.gcs.analyticscore.client.GcsFileSystemImpl; +import com.google.cloud.gcs.analyticscore.client.GcsFileSystemOptions; +import com.google.cloud.gcs.analyticscore.core.GcsAnalyticsCoreOptions; import com.google.cloud.storage.Storage; import com.google.cloud.storage.StorageOptions; import java.io.IOException; @@ -41,6 +46,8 @@ class PrefixedStorage implements AutoCloseable { private SerializableSupplier storage; private CloseableGroup closeableGroup; private transient volatile Storage storageClient; + private final SerializableSupplier gcsFileSystemSupplier; + private transient volatile GcsFileSystem gcsFileSystem; PrefixedStorage( String storagePrefix, Map properties, SerializableSupplier storage) { @@ -50,7 +57,7 @@ class PrefixedStorage implements AutoCloseable { this.storagePrefix = storagePrefix; this.storage = storage; this.gcpProperties = new GCPProperties(properties); - + this.closeableGroup = new CloseableGroup(); if (null == storage) { this.storage = () -> { @@ -64,23 +71,16 @@ class PrefixedStorage implements AutoCloseable { gcpProperties.clientLibToken().ifPresent(builder::setClientLibToken); gcpProperties.serviceHost().ifPresent(builder::setHost); - // Google Cloud APIs default to automatically detect the credentials to use, which is - // in most cases the convenient way, especially in GCP. - // See javadoc of com.google.auth.oauth2.GoogleCredentials.getApplicationDefault() - if (gcpProperties.noAuth()) { - // Explicitly allow "no credentials" for testing purposes - builder.setCredentials(NoCredentials.getInstance()); - } - - if (gcpProperties.oauth2Token().isPresent()) { - this.closeableGroup = new CloseableGroup(); - builder.setCredentials( - GCPAuthUtils.oauth2CredentialsFromGcpProperties(gcpProperties, closeableGroup)); + Credentials credentials = credentials(gcpProperties); + if (credentials != null) { + builder.setCredentials(credentials); } return builder.build().getService(); }; } + + this.gcsFileSystemSupplier = gcsFileSystemSupplier(properties); } public String storagePrefix() { @@ -118,4 +118,47 @@ public void close() { storage = null; } } + + GcsFileSystem gcsFileSystem() { + if (gcsFileSystem == null) { + synchronized (this) { + if (gcsFileSystem == null) { + this.gcsFileSystem = gcsFileSystemSupplier.get(); + this.closeableGroup.addCloseable(gcsFileSystem); + } + } + } + + return this.gcsFileSystem; + } + + private Credentials credentials(GCPProperties properties) { + // Google Cloud APIs default to automatically detect the credentials to use, which is + // in most cases the convenient way, especially in GCP. + // See javadoc of com.google.auth.oauth2.GoogleCredentials.getApplicationDefault() + if (properties.oauth2Token().isPresent()) { + return GCPAuthUtils.oauth2CredentialsFromGcpProperties(properties, closeableGroup); + } else if (properties.noAuth()) { + // Explicitly allow "no credentials" for testing purposes + return NoCredentials.getInstance(); + } else { + return null; + } + } + + private SerializableSupplier gcsFileSystemSupplier( + Map properties) { + ImmutableMap.Builder propertiesWithUserAgent = + new ImmutableMap.Builder() + .putAll(properties) + .put("gcs.user-agent", GCS_FILE_IO_USER_AGENT); + GcsAnalyticsCoreOptions gcsAnalyticsCoreOptions = + new GcsAnalyticsCoreOptions("gcs.", propertiesWithUserAgent.build()); + GcsFileSystemOptions fileSystemOptions = gcsAnalyticsCoreOptions.getGcsFileSystemOptions(); + Credentials credentials = credentials(new GCPProperties(properties)); + return () -> + credentials == null + ? new GcsFileSystemImpl(fileSystemOptions) + : new GcsFileSystemImpl(credentials, fileSystemOptions); + } } diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestGcsInputFile.java b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestGcsInputFile.java new file mode 100644 index 000000000000..e58e814bd8d4 --- /dev/null +++ b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestGcsInputFile.java @@ -0,0 +1,270 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.gcp.gcs; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockConstruction; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; + +import com.google.cloud.gcs.analyticscore.client.GcsFileInfo; +import com.google.cloud.gcs.analyticscore.client.GcsFileSystem; +import com.google.cloud.gcs.analyticscore.client.GcsItemId; +import com.google.cloud.gcs.analyticscore.client.GcsItemInfo; +import com.google.cloud.gcs.analyticscore.core.GcsAnalyticsCoreOptions; +import com.google.cloud.gcs.analyticscore.core.GoogleCloudStorageInputStream; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobId; +import com.google.cloud.storage.Storage; +import java.io.IOException; +import java.net.URI; +import java.util.Collections; +import org.apache.iceberg.gcp.GCPProperties; +import org.apache.iceberg.io.SeekableInputStream; +import org.apache.iceberg.metrics.MetricsContext; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.MockedConstruction; +import org.mockito.MockedStatic; + +public class TestGcsInputFile { + + private static final String TEST_BUCKET = "TEST_BUCKET"; + private static final String KEY = "file/path/a.dat"; + private static final String LOCATION = "gs://" + TEST_BUCKET + "/" + KEY; + private static final long FILE_SIZE = 1024L; + + private Storage storage; + private GcsFileSystem gcsFileSystem; + private GcsAnalyticsCoreOptions gcsAnalyticsCoreOptions; + private PrefixedStorage prefixedStorage; + private GCPProperties gcpProperties; + private MetricsContext metricsContext; + private Blob blob; + + @BeforeEach + public void before() { + storage = mock(Storage.class); + gcsFileSystem = mock(GcsFileSystem.class); + prefixedStorage = mock(PrefixedStorage.class); + gcsAnalyticsCoreOptions = new GcsAnalyticsCoreOptions("", Collections.emptyMap()); + gcpProperties = new GCPProperties(); + metricsContext = MetricsContext.nullMetrics(); + blob = mock(Blob.class); + when(prefixedStorage.storage()).thenReturn(storage); + when(prefixedStorage.gcsFileSystem()).thenReturn(gcsFileSystem); + when(prefixedStorage.gcpProperties()).thenReturn(gcpProperties); + when(gcsFileSystem.getFileSystemOptions()) + .thenReturn(gcsAnalyticsCoreOptions.getGcsFileSystemOptions()); + when(storage.get(any(BlobId.class))).thenReturn(blob); + when(blob.getSize()).thenReturn(FILE_SIZE); + } + + @Test + public void fromLocation() { + GCSInputFile inputFile = GCSInputFile.fromLocation(LOCATION, prefixedStorage, metricsContext); + + assertThat(inputFile.blobId()).isEqualTo(BlobId.fromGsUtilUri(LOCATION)); + assertThat(inputFile.getLength()).isEqualTo(FILE_SIZE); + } + + @Test + public void fromLocationWithLength() { + GCSInputFile inputFile = + GCSInputFile.fromLocation(LOCATION, FILE_SIZE, prefixedStorage, metricsContext); + + assertThat(inputFile.blobId()).isEqualTo(BlobId.fromGsUtilUri(LOCATION)); + assertThat(inputFile.getLength()).isEqualTo(FILE_SIZE); + } + + @Test + public void getLength() { + when(blob.getSize()).thenReturn(FILE_SIZE); + GCSInputFile inputFile = + new GCSInputFile( + storage, + gcsFileSystem, + BlobId.fromGsUtilUri(LOCATION), + null, + gcpProperties, + metricsContext); + + assertThat(inputFile.getLength()).isEqualTo(FILE_SIZE); + } + + @Test + public void getLengthCached() { + GCSInputFile inputFile = + new GCSInputFile( + storage, + gcsFileSystem, + BlobId.fromGsUtilUri(LOCATION), + FILE_SIZE, + gcpProperties, + metricsContext); + + assertThat(inputFile.getLength()).isEqualTo(FILE_SIZE); + } + + @Test + public void newStreamGcsAnalyticsCoreEnabled() throws IOException { + GCPProperties enabledGcpProperties = + new GCPProperties(ImmutableMap.of(GCPProperties.GCS_ANALYTICS_CORE_ENABLED, "true")); + BlobId blobId = BlobId.fromGsUtilUri(LOCATION); + GcsItemId itemId = + GcsItemId.builder() + .setBucketName(blobId.getBucket()) + .setObjectName(blobId.getName()) + .build(); + GcsItemInfo itemInfo = GcsItemInfo.builder().setItemId(itemId).setSize(FILE_SIZE).build(); + GcsFileInfo gcsFileInfo = + GcsFileInfo.builder() + .setItemInfo(itemInfo) + .setUri(URI.create(LOCATION)) + .setAttributes(ImmutableMap.of()) + .build(); + try (MockedStatic mocked = + mockStatic(GoogleCloudStorageInputStream.class)) { + mocked + .when(() -> GoogleCloudStorageInputStream.create(gcsFileSystem, gcsFileInfo)) + .thenReturn(mock(GoogleCloudStorageInputStream.class)); + + GCSInputFile inputFile = + new GCSInputFile( + storage, + gcsFileSystem, + BlobId.fromGsUtilUri(LOCATION), + FILE_SIZE, + enabledGcpProperties, + metricsContext); + + try (SeekableInputStream stream = inputFile.newStream()) { + assertThat(stream).isInstanceOf(GcsInputStreamWrapper.class); + } + } + } + + @Test + public void newStreamGcsAnalyticsCoreEnabledObjectSizeNull() throws IOException { + GCPProperties enabledGcpProperties = + new GCPProperties(ImmutableMap.of(GCPProperties.GCS_ANALYTICS_CORE_ENABLED, "true")); + BlobId blobId = BlobId.fromGsUtilUri(LOCATION); + GcsItemId itemId = + GcsItemId.builder() + .setBucketName(blobId.getBucket()) + .setObjectName(blobId.getName()) + .build(); + try (MockedStatic mocked = + mockStatic(GoogleCloudStorageInputStream.class)) { + mocked + .when(() -> GoogleCloudStorageInputStream.create(gcsFileSystem, itemId)) + .thenReturn(mock(GoogleCloudStorageInputStream.class)); + + GCSInputFile inputFile = + new GCSInputFile( + storage, + gcsFileSystem, + BlobId.fromGsUtilUri(LOCATION), + null, + enabledGcpProperties, + metricsContext); + + try (SeekableInputStream stream = inputFile.newStream()) { + assertThat(stream).isInstanceOf(GcsInputStreamWrapper.class); + } + } + } + + @Test + public void newStreamGcsAnalyticsCoreDisabled() throws IOException { + GCSInputFile inputFile = + new GCSInputFile( + storage, + gcsFileSystem, + BlobId.fromGsUtilUri(LOCATION), + FILE_SIZE, + gcpProperties, + metricsContext); + + try (MockedConstruction mocked = + mockConstruction( + GCSInputStream.class, + (mock, context) -> { + assertThat(context.arguments()).hasSize(5); + assertThat(context.arguments().get(0)).isEqualTo(storage); + assertThat(context.arguments().get(1)).isEqualTo(BlobId.fromGsUtilUri(LOCATION)); + assertThat(context.arguments().get(2)).isEqualTo(FILE_SIZE); + assertThat(context.arguments().get(3)).isEqualTo(gcpProperties); + assertThat(context.arguments().get(4)).isEqualTo(metricsContext); + })) { + try (SeekableInputStream stream = inputFile.newStream()) { + assertThat(stream).isInstanceOf(GCSInputStream.class); + assertThat(mocked.constructed()).hasSize(1); + } + } + } + + @Test + public void newStreamAnalyticsCoreInitializationFailed() throws IOException { + GCPProperties enabledGcpProperties = + new GCPProperties(ImmutableMap.of(GCPProperties.GCS_ANALYTICS_CORE_ENABLED, "true")); + BlobId blobId = BlobId.fromGsUtilUri(LOCATION); + GcsItemId itemId = + GcsItemId.builder() + .setBucketName(blobId.getBucket()) + .setObjectName(blobId.getName()) + .build(); + + try (MockedStatic mocked = + mockStatic(GoogleCloudStorageInputStream.class)) { + mocked + .when(() -> GoogleCloudStorageInputStream.create(gcsFileSystem, itemId)) + .thenThrow(new IOException("GCS connector failed")); + + GCSInputFile inputFile = + new GCSInputFile( + storage, + gcsFileSystem, + BlobId.fromGsUtilUri(LOCATION), + null, + enabledGcpProperties, + metricsContext); + try (MockedConstruction inputStreamMocked = + mockConstruction( + GCSInputStream.class, + (mock, context) -> { + assertThat(context.arguments()).hasSize(5); + assertThat(context.arguments().get(0)).isEqualTo(storage); + assertThat(context.arguments().get(1)).isEqualTo(BlobId.fromGsUtilUri(LOCATION)); + assertThat(context.arguments().get(2)).isEqualTo(null); + assertThat(context.arguments().get(3)).isEqualTo(enabledGcpProperties); + assertThat(context.arguments().get(4)).isEqualTo(metricsContext); + })) { + SeekableInputStream stream = inputFile.newStream(); + assertThat(stream).isInstanceOf(GCSInputStream.class); + assertThat(inputStreamMocked.constructed()).hasSize(1); + mocked.verify(() -> GoogleCloudStorageInputStream.create(gcsFileSystem, itemId)); + stream.close(); + } + } + } +} diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestGcsInputStreamWrapper.java b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestGcsInputStreamWrapper.java new file mode 100644 index 000000000000..2320037bd017 --- /dev/null +++ b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestGcsInputStreamWrapper.java @@ -0,0 +1,146 @@ +/* + * 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.gcp.gcs; + +import com.google.cloud.gcs.analyticscore.client.GcsObjectRange; +import com.google.cloud.gcs.analyticscore.core.GoogleCloudStorageInputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.function.IntFunction; +import org.apache.iceberg.io.FileRange; +import org.apache.iceberg.metrics.MetricsContext; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.jupiter.MockitoExtension; + +@ExtendWith(MockitoExtension.class) +public class TestGcsInputStreamWrapper { + + @Mock private GoogleCloudStorageInputStream googleCloudStorageInputStream; + + private GcsInputStreamWrapper inputStreamWrapper; + + @BeforeEach + public void before() { + inputStreamWrapper = + new GcsInputStreamWrapper(googleCloudStorageInputStream, MetricsContext.nullMetrics()); + } + + @Test + public void getPos() throws IOException { + inputStreamWrapper.getPos(); + + Mockito.verify(googleCloudStorageInputStream).getPos(); + } + + @Test + public void seek() throws IOException { + long newPos = 1234L; + inputStreamWrapper.seek(newPos); + + Mockito.verify(googleCloudStorageInputStream).seek(newPos); + } + + @Test + public void read() throws IOException { + inputStreamWrapper.read(); + + Mockito.verify(googleCloudStorageInputStream).read(); + } + + @Test + public void readByteArray() throws IOException { + byte[] buffer = new byte[1024]; + + inputStreamWrapper.read(buffer); + + Mockito.verify(googleCloudStorageInputStream).read(buffer, 0, buffer.length); + } + + @Test + public void readByteArrayWithOffset() throws IOException { + byte[] buffer = new byte[1024]; + int off = 10; + int len = 100; + + inputStreamWrapper.read(buffer, off, len); + + Mockito.verify(googleCloudStorageInputStream).read(buffer, off, len); + } + + @Test + public void readFully() throws IOException { + long position = 123L; + byte[] buffer = new byte[1024]; + int offset = 10; + int length = 100; + + inputStreamWrapper.readFully(position, buffer, offset, length); + + Mockito.verify(googleCloudStorageInputStream).readFully(position, buffer, offset, length); + } + + @Test + public void readTail() throws IOException { + byte[] buffer = new byte[1024]; + int offset = 10; + int length = 100; + + inputStreamWrapper.readTail(buffer, offset, length); + + Mockito.verify(googleCloudStorageInputStream).readTail(buffer, offset, length); + } + + @Test + public void readVectored() throws IOException { + CompletableFuture future1 = new CompletableFuture<>(); + CompletableFuture future2 = new CompletableFuture<>(); + List ranges = + List.of(new FileRange(future1, 10L, 100), new FileRange(future2, 0, 50)); + IntFunction allocate = ByteBuffer::allocate; + + inputStreamWrapper.readVectored(ranges, allocate); + List objectRanges = + List.of( + GcsObjectRange.builder() + .setOffset(10) + .setLength(100) + .setByteBufferFuture(future1) + .build(), + GcsObjectRange.builder() + .setOffset(0) + .setLength(50) + .setByteBufferFuture(future2) + .build()); + + Mockito.verify(googleCloudStorageInputStream).readVectored(objectRanges, allocate); + } + + @Test + public void close() throws IOException { + inputStreamWrapper.close(); + + Mockito.verify(googleCloudStorageInputStream).close(); + } +} diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestPrefixedStorage.java b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestPrefixedStorage.java index 595f025ae9a0..cfabb2f65edd 100644 --- a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestPrefixedStorage.java +++ b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestPrefixedStorage.java @@ -21,6 +21,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import com.google.cloud.gcs.analyticscore.client.GcsClientOptions; +import com.google.cloud.gcs.analyticscore.client.GcsFileSystem; +import com.google.cloud.gcs.analyticscore.client.GcsFileSystemOptions; +import com.google.cloud.gcs.analyticscore.client.GcsReadOptions; import java.util.Map; import org.apache.iceberg.EnvironmentContext; import org.apache.iceberg.gcp.GCPProperties; @@ -69,4 +73,40 @@ public void userAgentPrefix() { assertThat(storage.storage().getOptions().getUserAgent()) .isEqualTo("gcsfileio/" + EnvironmentContext.get()); } + + @Test + public void gcsFileSystem() { + Map properties = + ImmutableMap.of( + GCPProperties.GCS_PROJECT_ID, "myProject", + GCPProperties.GCS_USER_PROJECT, "userProject", + GCPProperties.GCS_CLIENT_LIB_TOKEN, "gccl", + GCPProperties.GCS_SERVICE_HOST, "example.com", + GCPProperties.GCS_DECRYPTION_KEY, "decryptionKey", + GCPProperties.GCS_ENCRYPTION_KEY, "encryptionKey", + GCPProperties.GCS_CHANNEL_READ_CHUNK_SIZE, "1024"); + PrefixedStorage storage = new PrefixedStorage("gs://bucket", properties, null); + GcsFileSystemOptions expectedOptions = + GcsFileSystemOptions.builder() + .setGcsClientOptions( + GcsClientOptions.builder() + .setProjectId("myProject") + .setClientLibToken("gccl") + .setServiceHost("example.com") + .setUserAgent("gcsfileio/" + EnvironmentContext.get()) + .setGcsReadOptions( + GcsReadOptions.builder() + .setChunkSize(1024) + .setDecryptionKey("decryptionKey") + .setUserProjectId("userProject") + .build()) + .build()) + .build(); + + GcsFileSystem fileSystem = storage.gcsFileSystem(); + + assertThat(fileSystem).isNotNull(); + assertThat(fileSystem.getGcsClient()).isNotNull(); + assertThat(fileSystem.getFileSystemOptions()).isEqualTo(expectedOptions); + } } diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index a99206dba3f2..314a1e0471a1 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -50,6 +50,7 @@ flink120 = { strictly = "1.20.1"} flink20 = { strictly = "2.0.0"} flink21 = { strictly = "2.1.0"} google-libraries-bom = "26.72.0" +gcs-analytics-core = "1.2.1" guava = "33.5.0-jre" hadoop3 = "3.4.2" httpcomponents-httpclient5 = "5.5.1" @@ -133,6 +134,7 @@ flink21-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwiza flink21-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink21" } flink21-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink21" } google-libraries-bom = { module = "com.google.cloud:libraries-bom", version.ref = "google-libraries-bom" } +gcs-analytics-core = { module = "com.google.cloud.gcs.analytics:gcs-analytics-core", version.ref = "gcs-analytics-core" } guava-guava = { module = "com.google.guava:guava", version.ref = "guava" } hadoop3-client = { module = "org.apache.hadoop:hadoop-client", version.ref = "hadoop3" } hadoop3-common = { module = "org.apache.hadoop:hadoop-common", version.ref = "hadoop3" } From 0cc337ace306033dff1db2775a8046c1a58214f5 Mon Sep 17 00:00:00 2001 From: Prashant Singh <35593236+singhpk234@users.noreply.github.com> Date: Tue, 9 Dec 2025 22:34:26 -0800 Subject: [PATCH 072/201] Core: REST Scan Planning Task Implementation (#13400) Co-authored-by: Prashant Singh --- build.gradle | 1 + .../org/apache/iceberg/TableScanContext.java | 2 +- .../apache/iceberg/rest/ErrorHandlers.java | 50 + .../iceberg/rest/RESTCatalogProperties.java | 4 + .../iceberg/rest/RESTSessionCatalog.java | 41 + .../org/apache/iceberg/rest/RESTTable.java | 70 ++ .../apache/iceberg/rest/RESTTableScan.java | 282 +++++ .../apache/iceberg/rest/ResourcePaths.java | 34 + .../apache/iceberg/rest/ScanTaskIterable.java | 249 ++++ .../java/org/apache/iceberg/TestBase.java | 27 +- .../iceberg/rest/RESTCatalogAdapter.java | 12 +- .../iceberg/rest/TestRESTScanPlanning.java | 1003 +++++++++++++++++ .../iceberg/rest/TestResourcePaths.java | 63 ++ 13 files changed, 1832 insertions(+), 6 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/rest/RESTTable.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/ScanTaskIterable.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java diff --git a/build.gradle b/build.gradle index d740e6fb53e6..1a18887b21c7 100644 --- a/build.gradle +++ b/build.gradle @@ -377,6 +377,7 @@ project(':iceberg-core') { implementation libs.jackson.databind implementation libs.caffeine implementation libs.roaringbitmap + implementation libs.failsafe compileOnly(libs.hadoop3.client) { exclude group: 'org.apache.avro', module: 'avro' exclude group: 'org.slf4j', module: 'slf4j-log4j12' diff --git a/core/src/main/java/org/apache/iceberg/TableScanContext.java b/core/src/main/java/org/apache/iceberg/TableScanContext.java index faa1c264d5f1..e40899556e11 100644 --- a/core/src/main/java/org/apache/iceberg/TableScanContext.java +++ b/core/src/main/java/org/apache/iceberg/TableScanContext.java @@ -35,7 +35,7 @@ /** Context object with optional arguments for a TableScan. */ @Value.Immutable -abstract class TableScanContext { +public abstract class TableScanContext { @Nullable public abstract Long snapshotId(); diff --git a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java index 0c21fed4de54..b1575035fcc0 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java @@ -26,6 +26,8 @@ import org.apache.iceberg.exceptions.ForbiddenException; import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.exceptions.NoSuchPlanIdException; +import org.apache.iceberg.exceptions.NoSuchPlanTaskException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.exceptions.NotAuthorizedException; @@ -73,6 +75,14 @@ public static Consumer tableCommitHandler() { return CommitErrorHandler.INSTANCE; } + public static Consumer planErrorHandler() { + return PlanErrorHandler.INSTANCE; + } + + public static Consumer planTaskHandler() { + return PlanTaskErrorHandler.INSTANCE; + } + public static Consumer defaultErrorHandler() { return DefaultErrorHandler.INSTANCE; } @@ -125,6 +135,46 @@ public void accept(ErrorResponse error) { } } + /** Plan level error handler. */ + private static class PlanErrorHandler extends DefaultErrorHandler { + private static final ErrorHandler INSTANCE = new PlanErrorHandler(); + + @Override + public void accept(ErrorResponse error) { + if (error.code() == 404) { + if (NoSuchNamespaceException.class.getSimpleName().equals(error.type())) { + throw new NoSuchNamespaceException("%s", error.message()); + } else if (NoSuchTableException.class.getSimpleName().equals(error.type())) { + throw new NoSuchTableException("%s", error.message()); + } else { + throw new NoSuchPlanIdException("%s", error.message()); + } + } + + super.accept(error); + } + } + + /** PlanTask level error handler. */ + private static class PlanTaskErrorHandler extends DefaultErrorHandler { + private static final ErrorHandler INSTANCE = new PlanTaskErrorHandler(); + + @Override + public void accept(ErrorResponse error) { + if (error.code() == 404) { + if (NoSuchNamespaceException.class.getSimpleName().equals(error.type())) { + throw new NoSuchNamespaceException("%s", error.message()); + } else if (NoSuchTableException.class.getSimpleName().equals(error.type())) { + throw new NoSuchTableException("%s", error.message()); + } else { + throw new NoSuchPlanTaskException("%s", error.message()); + } + } + + super.accept(error); + } + } + /** View commit error handler. */ private static class ViewCommitErrorHandler extends DefaultErrorHandler { private static final ErrorHandler INSTANCE = new ViewCommitErrorHandler(); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTCatalogProperties.java b/core/src/main/java/org/apache/iceberg/rest/RESTCatalogProperties.java index e71610622bac..79617b2982ff 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTCatalogProperties.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTCatalogProperties.java @@ -37,6 +37,10 @@ private RESTCatalogProperties() {} public static final String NAMESPACE_SEPARATOR = "namespace-separator"; + // Enable planning on the REST server side + public static final String REST_SCAN_PLANNING_ENABLED = "rest-scan-planning-enabled"; + public static final boolean REST_SCAN_PLANNING_ENABLED_DEFAULT = false; + public enum SnapshotMode { ALL, REFS diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 814ed978c4ad..3f3626589566 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -41,6 +41,7 @@ import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; import org.apache.iceberg.Transaction; import org.apache.iceberg.Transactions; import org.apache.iceberg.catalog.BaseViewSessionCatalog; @@ -158,6 +159,7 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private MetricsReporter reporter = null; private boolean reportingViaRestEnabled; private Integer pageSize = null; + private boolean restScanPlanningEnabled; private CloseableGroup closeables = null; private Set endpoints; private Supplier> mutationHeaders = Map::of; @@ -270,6 +272,11 @@ public void initialize(String name, Map unresolved) { RESTCatalogProperties.NAMESPACE_SEPARATOR, RESTUtil.NAMESPACE_SEPARATOR_URLENCODED_UTF_8); + this.restScanPlanningEnabled = + PropertyUtil.propertyAsBoolean( + mergedProps, + RESTCatalogProperties.REST_SCAN_PLANNING_ENABLED, + RESTCatalogProperties.REST_SCAN_PLANNING_ENABLED_DEFAULT); super.initialize(name, mergedProps); } @@ -476,6 +483,13 @@ public Table loadTable(SessionContext context, TableIdentifier identifier) { trackFileIO(ops); + RESTTable restTable = restTableForScanPlanning(ops, finalIdentifier, tableClient); + // RestTable should be only be returned for non-metadata tables, because client would + // not have access to metadata files for example manifests, since all it needs is catalog. + if (restTable != null) { + return restTable; + } + BaseTable table = new BaseTable( ops, @@ -488,6 +502,23 @@ public Table loadTable(SessionContext context, TableIdentifier identifier) { return table; } + private RESTTable restTableForScanPlanning( + TableOperations ops, TableIdentifier finalIdentifier, RESTClient restClient) { + // server supports remote planning endpoint and server / client wants to do server side planning + if (endpoints.contains(Endpoint.V1_SUBMIT_TABLE_SCAN_PLAN) && restScanPlanningEnabled) { + return new RESTTable( + ops, + fullTableName(finalIdentifier), + metricsReporter(paths.metrics(finalIdentifier), restClient), + restClient, + Map::of, + finalIdentifier, + paths, + endpoints); + } + return null; + } + private void trackFileIO(RESTTableOperations ops) { if (io != ops.io()) { fileIOTracker.track(ops); @@ -556,6 +587,11 @@ public Table registerTable( trackFileIO(ops); + RESTTable restTable = restTableForScanPlanning(ops, ident, tableClient); + if (restTable != null) { + return restTable; + } + return new BaseTable( ops, fullTableName(ident), metricsReporter(paths.metrics(ident), tableClient)); } @@ -820,6 +856,11 @@ public Table create() { trackFileIO(ops); + RESTTable restTable = restTableForScanPlanning(ops, ident, tableClient); + if (restTable != null) { + return restTable; + } + return new BaseTable( ops, fullTableName(ident), metricsReporter(paths.metrics(ident), tableClient)); } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTTable.java b/core/src/main/java/org/apache/iceberg/rest/RESTTable.java new file mode 100644 index 000000000000..5565122aaa6f --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/RESTTable.java @@ -0,0 +1,70 @@ +/* + * 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.rest; + +import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.ImmutableTableScanContext; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.metrics.MetricsReporter; + +class RESTTable extends BaseTable { + private final RESTClient client; + private final Supplier> headers; + private final MetricsReporter reporter; + private final ResourcePaths resourcePaths; + private final TableIdentifier tableIdentifier; + private final Set supportedEndpoints; + + RESTTable( + TableOperations ops, + String name, + MetricsReporter reporter, + RESTClient client, + Supplier> headers, + TableIdentifier tableIdentifier, + ResourcePaths resourcePaths, + Set supportedEndpoints) { + super(ops, name, reporter); + this.reporter = reporter; + this.client = client; + this.headers = headers; + this.tableIdentifier = tableIdentifier; + this.resourcePaths = resourcePaths; + this.supportedEndpoints = supportedEndpoints; + } + + @Override + public TableScan newScan() { + return new RESTTableScan( + this, + schema(), + ImmutableTableScanContext.builder().metricsReporter(reporter).build(), + client, + headers.get(), + operations(), + tableIdentifier, + resourcePaths, + supportedEndpoints); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java b/core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java new file mode 100644 index 000000000000..741f6e549779 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java @@ -0,0 +1,282 @@ +/* + * 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.rest; + +import dev.failsafe.Failsafe; +import dev.failsafe.FailsafeException; +import dev.failsafe.RetryPolicy; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.DataTableScan; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.TableScanContext; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.requests.PlanTableScanRequest; +import org.apache.iceberg.rest.responses.FetchPlanningResultResponse; +import org.apache.iceberg.rest.responses.PlanTableScanResponse; +import org.apache.iceberg.types.Types; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class RESTTableScan extends DataTableScan { + private static final Logger LOG = LoggerFactory.getLogger(RESTTableScan.class); + private static final long MIN_SLEEP_MS = 1000; // Initial delay + private static final long MAX_SLEEP_MS = 60 * 1000; // Max backoff delay (1 minute) + private static final int MAX_ATTEMPTS = 10; // Max number of poll checks + private static final long MAX_WAIT_TIME_MS = 5 * 60 * 1000; // Total maximum duration (5 minutes) + private static final double SCALE_FACTOR = 2.0; // Exponential scale factor + + private final RESTClient client; + private final Map headers; + private final TableOperations operations; + private final Table table; + private final ResourcePaths resourcePaths; + private final TableIdentifier tableIdentifier; + private final Set supportedEndpoints; + private final ParserContext parserContext; + private String currentPlanId = null; + + RESTTableScan( + Table table, + Schema schema, + TableScanContext context, + RESTClient client, + Map headers, + TableOperations operations, + TableIdentifier tableIdentifier, + ResourcePaths resourcePaths, + Set supportedEndpoints) { + super(table, schema, context); + this.table = table; + this.client = client; + this.headers = headers; + this.operations = operations; + this.tableIdentifier = tableIdentifier; + this.resourcePaths = resourcePaths; + this.supportedEndpoints = supportedEndpoints; + this.parserContext = + ParserContext.builder() + .add("specsById", table.specs()) + .add("caseSensitive", context().caseSensitive()) + .build(); + } + + @Override + protected TableScan newRefinedScan( + Table refinedTable, Schema refinedSchema, TableScanContext refinedContext) { + return new RESTTableScan( + refinedTable, + refinedSchema, + refinedContext, + client, + headers, + operations, + tableIdentifier, + resourcePaths, + supportedEndpoints); + } + + @Override + public CloseableIterable planFiles() { + Long startSnapshotId = context().fromSnapshotId(); + Long endSnapshotId = context().toSnapshotId(); + Long snapshotId = snapshotId(); + List selectedColumns = + schema().columns().stream().map(Types.NestedField::name).collect(Collectors.toList()); + + List statsFields = null; + if (columnsToKeepStats() != null) { + statsFields = + columnsToKeepStats().stream() + .map(columnId -> schema().findColumnName(columnId)) + .collect(Collectors.toList()); + } + + PlanTableScanRequest.Builder builder = + PlanTableScanRequest.builder() + .withSelect(selectedColumns) + .withFilter(filter()) + .withCaseSensitive(isCaseSensitive()) + .withStatsFields(statsFields); + + if (startSnapshotId != null && endSnapshotId != null) { + builder + .withStartSnapshotId(startSnapshotId) + .withEndSnapshotId(endSnapshotId) + .withUseSnapshotSchema(true); + } else if (snapshotId != null) { + boolean useSnapShotSchema = snapshotId != table.currentSnapshot().snapshotId(); + builder.withSnapshotId(snapshotId).withUseSnapshotSchema(useSnapShotSchema); + } + + return planTableScan(builder.build()); + } + + private CloseableIterable planTableScan(PlanTableScanRequest planTableScanRequest) { + PlanTableScanResponse response = + client.post( + resourcePaths.planTableScan(tableIdentifier), + planTableScanRequest, + PlanTableScanResponse.class, + headers, + ErrorHandlers.tableErrorHandler(), + stringStringMap -> {}, + parserContext); + + PlanStatus planStatus = response.planStatus(); + switch (planStatus) { + case COMPLETED: + currentPlanId = response.planId(); + return scanTasksIterable(response.planTasks(), response.fileScanTasks()); + case SUBMITTED: + Endpoint.check(supportedEndpoints, Endpoint.V1_FETCH_TABLE_SCAN_PLAN); + return fetchPlanningResult(response.planId()); + case FAILED: + throw new IllegalStateException( + String.format( + "Received status: %s for planId: %s", PlanStatus.FAILED, response.planId())); + case CANCELLED: + throw new IllegalStateException( + String.format( + "Received status: %s for planId: %s", PlanStatus.CANCELLED, response.planId())); + default: + throw new IllegalStateException( + String.format("Invalid planStatus: %s for planId: %s", planStatus, response.planId())); + } + } + + private CloseableIterable fetchPlanningResult(String planId) { + currentPlanId = planId; + + RetryPolicy retryPolicy = + RetryPolicy.builder() + .handleResultIf(response -> response.planStatus() == PlanStatus.SUBMITTED) + .withBackoff( + Duration.ofMillis(MIN_SLEEP_MS), Duration.ofMillis(MAX_SLEEP_MS), SCALE_FACTOR) + .withJitter(0.1) // Add jitter up to 10% of the calculated delay + .withMaxAttempts(MAX_ATTEMPTS) + .withMaxDuration(Duration.ofMillis(MAX_WAIT_TIME_MS)) + .onFailedAttempt( + e -> { + // Log when a retry occurs + LOG.debug( + "Plan {} still SUBMITTED (Attempt {}/{}). Previous attempt took {} ms.", + planId, + e.getAttemptCount(), + MAX_ATTEMPTS, + e.getElapsedAttemptTime().toMillis()); + }) + .onFailure( + e -> { + LOG.warn( + "Polling for plan {} failed due to: {}", + planId, + e.getException().getMessage()); + cancelPlan(); + }) + .build(); + + try { + FetchPlanningResultResponse response = + Failsafe.with(retryPolicy) + .get( + () -> + client.get( + resourcePaths.plan(tableIdentifier, planId), + headers, + FetchPlanningResultResponse.class, + headers, + ErrorHandlers.planErrorHandler(), + parserContext)); + Preconditions.checkState( + response.planStatus() == PlanStatus.COMPLETED, + "Plan finished with unexpected status %s for planId: %s", + response.planStatus(), + planId); + + return scanTasksIterable(response.planTasks(), response.fileScanTasks()); + } catch (FailsafeException e) { + // FailsafeException is thrown when retries are exhausted (Max Attempts/Duration) + // Cleanup is handled by the .onFailure() hook, so we just wrap and rethrow. + throw new IllegalStateException( + String.format("Polling timed out or exceeded max attempts for planId: %s.", planId), e); + } catch (Exception e) { + // Catch any immediate non-retryable exceptions (e.g., I/O errors, auth errors) + try { + cancelPlan(); + } catch (Exception cancelException) { + // Ignore cancellation failures during exception handling + e.addSuppressed(cancelException); + } + throw e; + } + } + + private CloseableIterable scanTasksIterable( + List planTasks, List fileScanTasks) { + if (planTasks != null && !planTasks.isEmpty()) { + Endpoint.check(supportedEndpoints, Endpoint.V1_FETCH_TABLE_SCAN_PLAN_TASKS); + } + + return CloseableIterable.whenComplete( + new ScanTaskIterable( + planTasks, + fileScanTasks == null ? List.of() : fileScanTasks, + client, + resourcePaths, + tableIdentifier, + headers, + planExecutor(), + parserContext), + this::cancelPlan); + } + + @VisibleForTesting + @SuppressWarnings("checkstyle:RegexpMultiline") + public boolean cancelPlan() { + String planId = currentPlanId; + if (planId == null || !supportedEndpoints.contains(Endpoint.V1_CANCEL_TABLE_SCAN_PLAN)) { + return false; + } + + try { + client.delete( + resourcePaths.plan(tableIdentifier, planId), + Map.of(), + null, + headers, + ErrorHandlers.planErrorHandler()); + currentPlanId = null; + return true; + } catch (Exception e) { + // Plan might have already completed or failed, which is acceptable + return false; + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java b/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java index d85b00d02ebe..231a966f8062 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java +++ b/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java @@ -151,6 +151,40 @@ public String renameView() { return SLASH.join("v1", prefix, "views", "rename"); } + public String planTableScan(TableIdentifier ident) { + return SLASH.join( + "v1", + prefix, + "namespaces", + pathEncode(ident.namespace()), + "tables", + RESTUtil.encodeString(ident.name()), + "plan"); + } + + public String plan(TableIdentifier ident, String planId) { + return SLASH.join( + "v1", + prefix, + "namespaces", + pathEncode(ident.namespace()), + "tables", + RESTUtil.encodeString(ident.name()), + "plan", + RESTUtil.encodeString(planId)); + } + + public String fetchScanTasks(TableIdentifier ident) { + return SLASH.join( + "v1", + prefix, + "namespaces", + pathEncode(ident.namespace()), + "tables", + RESTUtil.encodeString(ident.name()), + "tasks"); + } + private String pathEncode(Namespace ns) { return RESTUtil.encodeNamespace(ns, namespaceSeparator); } diff --git a/core/src/main/java/org/apache/iceberg/rest/ScanTaskIterable.java b/core/src/main/java/org/apache/iceberg/rest/ScanTaskIterable.java new file mode 100644 index 000000000000..a03be784da93 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/ScanTaskIterable.java @@ -0,0 +1,249 @@ +/* + * 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.rest; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.rest.requests.FetchScanTasksRequest; +import org.apache.iceberg.rest.responses.FetchScanTasksResponse; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class ScanTaskIterable implements CloseableIterable { + + private static final Logger LOG = LoggerFactory.getLogger(ScanTaskIterable.class); + private static final int DEFAULT_TASK_QUEUE_CAPACITY = 1000; + private static final long QUEUE_POLL_TIMEOUT_MS = 100; + private static final int WORKER_POOL_SIZE = Math.max(1, ThreadPools.WORKER_THREAD_POOL_SIZE / 4); + private final BlockingQueue taskQueue; + private final ConcurrentLinkedQueue initialFileScanTasks; + private final ConcurrentLinkedQueue planTasks; + private final AtomicInteger activeWorkers = new AtomicInteger(0); + private final AtomicBoolean shutdown = new AtomicBoolean(false); + private final ExecutorService executorService; + private final RESTClient client; + private final ResourcePaths resourcePaths; + private final TableIdentifier tableIdentifier; + private final Map headers; + private final ParserContext parserContext; + + ScanTaskIterable( + List initialPlanTasks, + List initialFileScanTasks, + RESTClient client, + ResourcePaths resourcePaths, + TableIdentifier tableIdentifier, + Map headers, + ExecutorService executorService, + ParserContext parserContext) { + + this.taskQueue = new LinkedBlockingQueue<>(DEFAULT_TASK_QUEUE_CAPACITY); + this.planTasks = new ConcurrentLinkedQueue<>(); + // Initialize initial file scan tasks queue so that multiple workers can poll produce from it. + this.initialFileScanTasks = new ConcurrentLinkedQueue<>(initialFileScanTasks); + + this.client = client; + this.resourcePaths = resourcePaths; + this.tableIdentifier = tableIdentifier; + this.headers = headers; + this.executorService = executorService; + this.parserContext = parserContext; + + if (initialPlanTasks != null && !initialPlanTasks.isEmpty()) { + planTasks.addAll(initialPlanTasks); + } else if (initialFileScanTasks.isEmpty()) { + // nothing to do, no need to spawn workers. + return; + } + + submitFixedWorkers(); + } + + private void submitFixedWorkers() { + if (planTasks.isEmpty() && initialFileScanTasks.isEmpty()) { + // nothing to do + return; + } + + // need to spawn at least one worker to enqueue initial file scan tasks + int numWorkers = Math.min(WORKER_POOL_SIZE, Math.max(planTasks.size(), 1)); + + for (int i = 0; i < numWorkers; i++) { + executorService.execute(new PlanTaskWorker()); + } + } + + @Override + public CloseableIterator iterator() { + return new ScanTasksIterator(); + } + + @Override + public void close() throws IOException {} + + private class PlanTaskWorker implements Runnable { + + @Override + public void run() { + activeWorkers.incrementAndGet(); + + try { + while (!shutdown.get()) { + String planTask = planTasks.poll(); + if (planTask == null) { + // if there are no more plan tasks, see if we can just add any remaining initial + // file scan tasks before exiting. + while (!initialFileScanTasks.isEmpty()) { + FileScanTask initialFileScanTask = initialFileScanTasks.poll(); + if (initialFileScanTask != null) { + taskQueue.put(initialFileScanTask); + } + } + return; + } + + processPlanTask(planTask); + } + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (Exception e) { + throw new RuntimeException("Worker failed processing planTask", e); + } finally { + int remaining = activeWorkers.decrementAndGet(); + + if (remaining == 0 + && !planTasks.isEmpty() + && !shutdown.get() + && !initialFileScanTasks.isEmpty()) { + executorService.execute(new PlanTaskWorker()); + } + } + } + + private void processPlanTask(String planTask) throws InterruptedException { + FetchScanTasksResponse response = fetchScanTasks(planTask); + // immediately add any new plan tasks to the queue so the idle workers can pick them up + if (response.planTasks() != null) { + planTasks.addAll(response.planTasks()); + } + + // before blocking on the task queue, check for shutdown again + if (shutdown.get()) { + return; + } + + // Now since the network IO is done, first add any initial file scan tasks + while (!initialFileScanTasks.isEmpty()) { + FileScanTask initialFileScanTask = initialFileScanTasks.poll(); + if (initialFileScanTask != null) { + taskQueue.put(initialFileScanTask); + } + } + + if (response.fileScanTasks() != null) { + for (FileScanTask task : response.fileScanTasks()) { + taskQueue.put(task); + } + } + } + + private FetchScanTasksResponse fetchScanTasks(String planTask) { + FetchScanTasksRequest request = new FetchScanTasksRequest(planTask); + + return client.post( + resourcePaths.fetchScanTasks(tableIdentifier), + request, + FetchScanTasksResponse.class, + headers, + ErrorHandlers.planTaskHandler(), + stringStringMap -> {}, + parserContext); + } + } + + private class ScanTasksIterator implements CloseableIterator { + private FileScanTask nextTask = null; + + @Override + public boolean hasNext() { + if (nextTask != null) { + return true; + } + + while (true) { + if (isDone()) { + return false; + } + + try { + nextTask = taskQueue.poll(QUEUE_POLL_TIMEOUT_MS, TimeUnit.MILLISECONDS); + if (nextTask != null) { + return true; + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return false; + } + } + } + + @Override + public FileScanTask next() { + if (!hasNext()) { + throw new NoSuchElementException("No more scan tasks available"); + } + FileScanTask result = nextTask; + nextTask = null; + return result; + } + + @Override + public void close() { + shutdown.set(true); + LOG.info( + "ScanTasksIterator is closing. Clearing {} queued tasks and {} plan tasks.", + taskQueue.size(), + planTasks.size()); + taskQueue.clear(); + planTasks.clear(); + } + + private boolean isDone() { + return taskQueue.isEmpty() + && planTasks.isEmpty() + && activeWorkers.get() == 0 + && initialFileScanTasks.isEmpty(); + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index 0929c1bd37e6..ddac3709443d 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -91,6 +91,15 @@ public class TestBase { .withPartitionPath("data_bucket=0") // easy way to set partition data for now .withRecordCount(1) .build(); + public static final DeleteFile FILE_A_EQUALITY_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes(1) // delete on column 1 (id column) + .withPath("/path/to/data-a-equality-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("data_bucket=0") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_DV = FileMetadata.deleteFileBuilder(SPEC) .ofPositionDeletes() @@ -127,6 +136,14 @@ public class TestBase { .withPartitionPath("data_bucket=1") // easy way to set partition data for now .withRecordCount(1) .build(); + public static final DeleteFile FILE_B_EQUALITY_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes(1) // delete on column 1 (id column) + .withPath("/path/to/data-b-equality-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("data_bucket=1") // same partition as FILE_B + .withRecordCount(1) + .build(); static final DeleteFile FILE_B_DV = FileMetadata.deleteFileBuilder(SPEC) .ofPositionDeletes() @@ -154,6 +171,14 @@ public class TestBase { .withPartitionPath("data_bucket=2") // easy way to set partition data for now .withRecordCount(1) .build(); + public static final DeleteFile FILE_C_EQUALITY_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes(1) // delete on column 1 (id column) + .withPath("/path/to/data-c-equality-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("data_bucket=2") // same partition as FILE_C + .withRecordCount(1) + .build(); static final DataFile FILE_D = DataFiles.builder(SPEC) .withPath("/path/to/data-d.parquet") @@ -170,7 +195,7 @@ public class TestBase { .withPartitionPath("data_bucket=3") // easy way to set partition data for now .withRecordCount(1) .build(); - static final DataFile FILE_WITH_STATS = + public static final DataFile FILE_WITH_STATS = DataFiles.builder(SPEC) .withPath("/path/to/data-with-stats.parquet") .withMetrics( diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java index 586da0d10782..8d59ee0393fa 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -112,7 +112,7 @@ public class RESTCatalogAdapter extends BaseHTTPClient { private final ViewCatalog asViewCatalog; private AuthSession authSession = AuthSession.EMPTY; - private final PlanningBehavior planningBehavior = planningBehavior(); + private PlanningBehavior planningBehavior; public RESTCatalogAdapter(Catalog catalog) { this.catalog = catalog; @@ -321,8 +321,8 @@ public T handleRequest( catalog, ident, request, - planningBehavior::shouldPlanTableScanAsync, - scan -> planningBehavior.numberFileScanTasksPerPlanTask())); + planningBehavior()::shouldPlanTableScanAsync, + scan -> planningBehavior().numberFileScanTasksPerPlanTask())); } case FETCH_PLANNING_RESULT: @@ -603,7 +603,11 @@ default boolean shouldPlanTableScanAsync(Scan scan) { } protected PlanningBehavior planningBehavior() { - return new PlanningBehavior() {}; + return this.planningBehavior == null ? new PlanningBehavior() {} : planningBehavior; + } + + protected void setPlanningBehavior(PlanningBehavior behavior) { + this.planningBehavior = behavior; } @Override diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java new file mode 100644 index 000000000000..ba13d1e3c1c4 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java @@ -0,0 +1,1003 @@ +/* + * 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.rest; + +import static org.apache.iceberg.TestBase.FILE_A; +import static org.apache.iceberg.TestBase.FILE_A_DELETES; +import static org.apache.iceberg.TestBase.FILE_A_EQUALITY_DELETES; +import static org.apache.iceberg.TestBase.FILE_B; +import static org.apache.iceberg.TestBase.FILE_B_DELETES; +import static org.apache.iceberg.TestBase.FILE_B_EQUALITY_DELETES; +import static org.apache.iceberg.TestBase.FILE_C; +import static org.apache.iceberg.TestBase.FILE_C_EQUALITY_DELETES; +import static org.apache.iceberg.TestBase.SCHEMA; +import static org.apache.iceberg.TestBase.SPEC; +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 com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectReader; +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Scan; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SessionCatalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +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.Lists; +import org.apache.iceberg.rest.responses.ConfigResponse; +import org.apache.iceberg.rest.responses.ErrorResponse; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.gzip.GzipHandler; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.mockito.Mockito; + +public class TestRESTScanPlanning { + private static final ObjectMapper MAPPER = RESTObjectMapper.mapper(); + private static final Namespace NS = Namespace.of("ns"); + + private InMemoryCatalog backendCatalog; + private Server httpServer; + private RESTCatalogAdapter adapterForRESTServer; + private ParserContext parserContext; + @TempDir private Path temp; + private RESTCatalog restCatalogWithScanPlanning; + + @BeforeEach + public void setupCatalogs() throws Exception { + File warehouse = temp.toFile(); + this.backendCatalog = new InMemoryCatalog(); + this.backendCatalog.initialize( + "in-memory", + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse.getAbsolutePath())); + + adapterForRESTServer = + Mockito.spy( + new RESTCatalogAdapter(backendCatalog) { + @Override + public T execute( + HTTPRequest request, + Class responseType, + Consumer errorHandler, + Consumer> responseHeaders) { + if (ResourcePaths.config().equals(request.path())) { + return castResponse( + responseType, + ConfigResponse.builder() + .withEndpoints( + Arrays.stream(Route.values()) + .map(r -> Endpoint.create(r.method().name(), r.resourcePath())) + .collect(Collectors.toList())) + .withOverrides( + ImmutableMap.of( + RESTCatalogProperties.REST_SCAN_PLANNING_ENABLED, "true")) + .build()); + } + Object body = roundTripSerialize(request.body(), "request"); + HTTPRequest req = ImmutableHTTPRequest.builder().from(request).body(body).build(); + T response = super.execute(req, responseType, errorHandler, responseHeaders); + return roundTripSerialize(response, "response"); + } + }); + + ServletContextHandler servletContext = + new ServletContextHandler(ServletContextHandler.NO_SESSIONS); + servletContext.addServlet( + new ServletHolder(new RESTCatalogServlet(adapterForRESTServer)), "/*"); + servletContext.setHandler(new GzipHandler()); + + this.httpServer = new Server(new InetSocketAddress(InetAddress.getLoopbackAddress(), 0)); + httpServer.setHandler(servletContext); + httpServer.start(); + + // Initialize catalog with scan planning enabled + this.restCatalogWithScanPlanning = initCatalog("prod-with-scan-planning", ImmutableMap.of()); + } + + @AfterEach + public void teardownCatalogs() throws Exception { + if (restCatalogWithScanPlanning != null) { + restCatalogWithScanPlanning.close(); + } + + if (backendCatalog != null) { + backendCatalog.close(); + } + + if (httpServer != null) { + httpServer.stop(); + httpServer.join(); + } + } + + // ==================== Helper Methods ==================== + + private RESTCatalog initCatalog(String catalogName, Map additionalProperties) { + RESTCatalog catalog = + new RESTCatalog( + SessionCatalog.SessionContext.createEmpty(), + (config) -> + HTTPClient.builder(config) + .uri(config.get(CatalogProperties.URI)) + .withHeaders(RESTUtil.configHeaders(config)) + .build()); + catalog.setConf(new Configuration()); + Map properties = + ImmutableMap.of( + CatalogProperties.URI, + httpServer.getURI().toString(), + CatalogProperties.FILE_IO_IMPL, + "org.apache.iceberg.inmemory.InMemoryFileIO"); + catalog.initialize( + catalogName, + ImmutableMap.builder() + .putAll(properties) + .putAll(additionalProperties) + .build()); + return catalog; + } + + @SuppressWarnings("unchecked") + private T roundTripSerialize(T payload, String description) { + if (payload == null) { + return null; + } + + try { + if (payload instanceof RESTMessage) { + RESTMessage message = (RESTMessage) payload; + ObjectReader reader = MAPPER.readerFor(message.getClass()); + if (parserContext != null && !parserContext.isEmpty()) { + reader = reader.with(parserContext.toInjectableValues()); + } + return reader.readValue(MAPPER.writeValueAsString(message)); + } else { + // use Map so that Jackson doesn't try to instantiate ImmutableMap from payload.getClass() + return (T) MAPPER.readValue(MAPPER.writeValueAsString(payload), Map.class); + } + } catch (JsonProcessingException e) { + throw new RuntimeException( + String.format("Failed to serialize and deserialize %s: %s", description, payload), e); + } + } + + private void setParserContext(Table table) { + parserContext = + ParserContext.builder().add("specsById", table.specs()).add("caseSensitive", false).build(); + } + + private RESTCatalog scanPlanningCatalog() { + return restCatalogWithScanPlanning; + } + + private void configurePlanningBehavior( + Function configurator) { + TestPlanningBehavior.Builder builder = TestPlanningBehavior.builder(); + adapterForRESTServer.setPlanningBehavior(configurator.apply(builder).build()); + } + + private Table createTableWithScanPlanning(RESTCatalog catalog, String tableName) { + return createTableWithScanPlanning(catalog, TableIdentifier.of(NS, tableName)); + } + + private Table createTableWithScanPlanning(RESTCatalog catalog, TableIdentifier identifier) { + catalog.createNamespace(identifier.namespace()); + return catalog.buildTable(identifier, SCHEMA).withPartitionSpec(SPEC).create(); + } + + private RESTTable restTableFor(RESTCatalog catalog, String tableName) { + Table table = createTableWithScanPlanning(catalog, tableName); + table.newAppend().appendFile(FILE_A).commit(); + assertThat(table).isInstanceOf(RESTTable.class); + return (RESTTable) table; + } + + private RESTTableScan restTableScanFor(Table table) { + assertThat(table).isInstanceOf(RESTTable.class); + RESTTable restTable = (RESTTable) table; + TableScan scan = restTable.newScan(); + assertThat(scan).isInstanceOf(RESTTableScan.class); + return (RESTTableScan) scan; + } + + // ==================== Test Planning Behavior ==================== + + /** Enum for parameterized tests to test both synchronous and asynchronous planning modes. */ + private enum PlanningMode + implements Function { + SYNCHRONOUS(TestPlanningBehavior.Builder::synchronous), + ASYNCHRONOUS(TestPlanningBehavior.Builder::asynchronous); + + private final Function configurer; + + PlanningMode(Function configurer) { + this.configurer = configurer; + } + + @Override + public TestPlanningBehavior.Builder apply(TestPlanningBehavior.Builder builder) { + return this.configurer.apply(builder); + } + } + + private static class TestPlanningBehavior implements RESTCatalogAdapter.PlanningBehavior { + private final boolean asyncPlanning; + private final int tasksPerPage; + + private TestPlanningBehavior(boolean asyncPlanning, int tasksPerPage) { + this.asyncPlanning = asyncPlanning; + this.tasksPerPage = tasksPerPage; + } + + private static Builder builder() { + return new Builder(); + } + + @Override + public boolean shouldPlanTableScanAsync(Scan scan) { + return asyncPlanning; + } + + @Override + public int numberFileScanTasksPerPlanTask() { + return tasksPerPage; + } + + protected static class Builder { + private boolean asyncPlanning; + private int tasksPerPage; + + Builder asyncPlanning(boolean async) { + asyncPlanning = async; + return this; + } + + Builder tasksPerPage(int tasks) { + tasksPerPage = tasks; + return this; + } + + // Convenience methods for common test scenarios + Builder synchronous() { + return asyncPlanning(false).tasksPerPage(100); + } + + Builder synchronousWithPagination() { + return asyncPlanning(false).tasksPerPage(1); + } + + Builder asynchronous() { + return asyncPlanning(true).tasksPerPage(100); + } + + TestPlanningBehavior build() { + return new TestPlanningBehavior(asyncPlanning, tasksPerPage); + } + } + } + + @ParameterizedTest + @EnumSource(PlanningMode.class) + void scanPlanningWithAllTasksInSingleResponse( + Function planMode) + throws IOException { + configurePlanningBehavior(planMode); + Table table = restTableFor(scanPlanningCatalog(), "all_tasks_table"); + setParserContext(table); + + // Verify actual data file is returned with correct count + try (CloseableIterable iterable = table.newScan().planFiles()) { + List tasks = Lists.newArrayList(iterable); + + assertThat(tasks).hasSize(1); + assertThat(tasks.get(0).file().location()).isEqualTo(FILE_A.location()); + assertThat(tasks.get(0).deletes()).isEmpty(); + } + } + + @Test + public void nestedPlanTaskPagination() throws IOException { + // Configure: synchronous planning with very small pages (creates nested plan task structure) + configurePlanningBehavior(TestPlanningBehavior.Builder::synchronousWithPagination); + + Table table = restTableFor(scanPlanningCatalog(), "nested_plan_task_table"); + // add one more files for proper pagination + table.newFastAppend().appendFile(FILE_B).commit(); + setParserContext(table); + + // Verify actual data file is returned via nested plan task fetching with correct count + try (CloseableIterable iterable = table.newScan().planFiles()) { + List tasks = Lists.newArrayList(iterable); + assertThat(tasks).hasSize(2); + assertThat(tasks) + .anySatisfy(task -> assertThat(task.file().location()).isEqualTo(FILE_A.location())); + assertThat(tasks) + .anySatisfy(task -> assertThat(task.file().location()).isEqualTo(FILE_B.location())); + assertThat(tasks.get(0).deletes()).isEmpty(); + assertThat(tasks.get(1).deletes()).isEmpty(); + } + } + + @Test + public void cancelPlanMethodAvailability() { + configurePlanningBehavior(TestPlanningBehavior.Builder::synchronousWithPagination); + RESTTable table = restTableFor(scanPlanningCatalog(), "cancel_method_table"); + RESTTableScan restTableScan = restTableScanFor(table); + + // Test that cancelPlan method is available and callable + // When no plan is active, it should return false + assertThat(restTableScan.cancelPlan()).isFalse(); + + // Verify the method doesn't throw exceptions when called multiple times + assertThat(restTableScan.cancelPlan()).isFalse(); + } + + @Test + public void iterableCloseTriggersCancel() throws IOException { + configurePlanningBehavior(TestPlanningBehavior.Builder::asynchronous); + RESTTable restTable = restTableFor(scanPlanningCatalog(), "iterable_close_test"); + setParserContext(restTable); + + TableScan scan = restTable.newScan(); + assertThat(scan).isInstanceOf(RESTTableScan.class); + RESTTableScan restTableScan = (RESTTableScan) scan; + + // Get the iterable + CloseableIterable iterable = restTableScan.planFiles(); + + // call cancelPlan before closing the iterable + boolean cancelled = restTableScan.cancelPlan(); + assertThat(cancelled).isTrue(); + + // Verify we can close the iterable without exceptions + // This tests that cancellation callbacks are properly wired through + iterable.close(); + } + + @ParameterizedTest + @EnumSource(MetadataTableType.class) + public void metadataTablesWithRemotePlanning(MetadataTableType type) throws IOException { + assumeThat(type) + .as("POSITION_DELETES table does not implement newScan() method") + .isNotEqualTo(MetadataTableType.POSITION_DELETES); + + configurePlanningBehavior(TestPlanningBehavior.Builder::synchronous); + RESTTable table = restTableFor(scanPlanningCatalog(), "metadata_tables_test"); + table.newAppend().appendFile(FILE_B).commit(); + table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_B_EQUALITY_DELETES).commit(); + setParserContext(table); + // RESTTable should be only be returned for non-metadata tables, because client would + // not have access to metadata files for example manifests, since all it needs is file scan + // tasks, this test just verifies that metadata tables can be scanned with RESTTable. + Table metadataTableInstance = MetadataTableUtils.createMetadataTableInstance(table, type); + assertThat(metadataTableInstance).isNotNull(); + assertThat(metadataTableInstance.newScan().planFiles()).isNotEmpty(); + } + + @ParameterizedTest + @EnumSource(PlanningMode.class) + void remoteScanPlanningWithEmptyTable( + Function planMode) { + configurePlanningBehavior(planMode); + Table table = createTableWithScanPlanning(scanPlanningCatalog(), "empty_table_test"); + setParserContext(table); + assertThat(table.newScan().planFiles()).isEmpty(); + } + + @ParameterizedTest + @EnumSource(PlanningMode.class) + @Disabled("Pruning files based on columns is not yet supported in REST scan planning") + void remoteScanPlanningWithNonExistentColumn( + Function planMode) { + configurePlanningBehavior(planMode); + Table table = restTableFor(scanPlanningCatalog(), "non-existent_column"); + setParserContext(table); + assertThat(table.newScan().select("non-existent-column").planFiles()).isEmpty(); + } + + @ParameterizedTest + @EnumSource(PlanningMode.class) + void incrementalScan( + Function planMode) { + configurePlanningBehavior(planMode); + Table table = restTableFor(scanPlanningCatalog(), "incremental_scan"); + setParserContext(table); + + // Add second file to the table + table.newAppend().appendFile(FILE_B).commit(); + long startSnapshotId = table.currentSnapshot().snapshotId(); + // Add third file to the table + table.newAppend().appendFile(FILE_C).commit(); + long endSnapshotId = table.currentSnapshot().snapshotId(); + assertThat( + table + .newIncrementalAppendScan() + .fromSnapshotInclusive(startSnapshotId) + .toSnapshot(endSnapshotId) + .planFiles()) + .hasSize(2) + .extracting(task -> task.file().location()) + .contains(FILE_C.location(), FILE_B.location()); + } + + @ParameterizedTest + @EnumSource(PlanningMode.class) + void remoteScanPlanningWithPositionDeletes( + Function planMode) + throws IOException { + configurePlanningBehavior(planMode); + Table table = restTableFor(scanPlanningCatalog(), "position_deletes_test"); + setParserContext(table); + + // Add position deletes that correspond to FILE_A (which was added in table creation) + table.newRowDelta().addDeletes(FILE_A_DELETES).commit(); + + // Ensure we have a RESTTable with server-side planning enabled + assertThat(table).isInstanceOf(RESTTable.class); + + // Execute scan planning - should handle position deletes correctly + try (CloseableIterable iterable = table.newScan().planFiles()) { + List tasks = Lists.newArrayList(iterable); + + // Verify we get tasks back (specific count depends on implementation) + assertThat(tasks).hasSize(1); // 1 data file: FILE_A + + // Verify specific task content and delete file associations + FileScanTask taskWithDeletes = + assertThat(tasks) + .filteredOn(task -> !task.deletes().isEmpty()) + .first() + .as("Expected at least one task with delete files") + .actual(); + + assertThat(taskWithDeletes.file().location()).isEqualTo(FILE_A.location()); + assertThat(taskWithDeletes.deletes()).hasSize(1); // 1 delete file: FILE_A_DELETES + assertThat(taskWithDeletes.deletes().get(0).location()).isEqualTo(FILE_A_DELETES.location()); + } + } + + @ParameterizedTest + @EnumSource(PlanningMode.class) + void remoteScanPlanningWithEqualityDeletes( + Function planMode) + throws IOException { + configurePlanningBehavior(planMode); + Table table = restTableFor(scanPlanningCatalog(), "equality_deletes_test"); + setParserContext(table); + + // Add equality deletes that correspond to FILE_A + table.newRowDelta().addDeletes(FILE_A_EQUALITY_DELETES).commit(); + + // Execute scan planning - should handle equality deletes correctly + try (CloseableIterable iterable = table.newScan().planFiles()) { + List tasks = Lists.newArrayList(iterable); + + // Verify the task count and file paths + assertThat(tasks).hasSize(1); // 1 data file: FILE_A + + // Verify specific task content and equality delete file associations + FileScanTask taskWithDeletes = + assertThat(tasks) + .filteredOn(task -> !task.deletes().isEmpty()) + .first() + .as("Expected at least one task with delete files") + .actual(); + + assertThat(taskWithDeletes.file().location()).isEqualTo(FILE_A.location()); + assertThat(taskWithDeletes.deletes()).hasSize(1); + assertThat(taskWithDeletes.deletes().get(0).location()) + .isEqualTo(FILE_A_EQUALITY_DELETES.location()); + } + } + + @ParameterizedTest + @EnumSource(PlanningMode.class) + void remoteScanPlanningWithMixedDeletes( + Function planMode) + throws IOException { + configurePlanningBehavior(planMode); + Table table = restTableFor(scanPlanningCatalog(), "mixed_deletes_test"); + setParserContext(table); + + // Add both position and equality deletes in separate commits + table.newRowDelta().addDeletes(FILE_A_DELETES).commit(); + table + .newRowDelta() + .addDeletes(FILE_B_EQUALITY_DELETES) + .commit(); // Equality deletes for different partition + + // Execute scan planning - should handle mixed delete types correctly + try (CloseableIterable iterable = table.newScan().planFiles()) { + List tasks = Lists.newArrayList(iterable); + + // Verify task count: FILE_A only (FILE_B_EQUALITY_DELETES is in different partition) + assertThat(tasks).hasSize(1); // 1 data file: FILE_A + + // Verify FILE_A with position deletes (FILE_B_EQUALITY_DELETES not associated since no + // FILE_B) + FileScanTask fileATask = + assertThat(tasks) + .filteredOn(task -> task.file().location().equals(FILE_A.location())) + .first() + .as("Expected FILE_A in scan tasks") + .actual(); + + assertThat(fileATask.deletes()).hasSize(1); + assertThat(fileATask.deletes().get(0).location()).isEqualTo(FILE_A_DELETES.location()); + } + } + + @ParameterizedTest + @EnumSource(PlanningMode.class) + void remoteScanPlanningWithMultipleDeleteFiles( + Function planMode) + throws IOException { + configurePlanningBehavior(planMode); + Table table = restTableFor(scanPlanningCatalog(), "multiple_deletes_test"); + setParserContext(table); + + // Add FILE_B and FILE_C to the table (FILE_A is already added during table creation) + table.newAppend().appendFile(FILE_B).appendFile(FILE_C).commit(); + + // Add multiple delete files corresponding to FILE_A, FILE_B, FILE_C + table + .newRowDelta() + .addDeletes(FILE_A_DELETES) + .addDeletes(FILE_B_DELETES) + .addDeletes(FILE_C_EQUALITY_DELETES) + .commit(); + + // Execute scan planning with multiple delete files + try (CloseableIterable iterable = table.newScan().planFiles()) { + List tasks = Lists.newArrayList(iterable); + + // Verify we get tasks back (should have 3 data files: FILE_A, FILE_B, FILE_C) + assertThat(tasks).hasSize(3); // 3 data files + + // Verify FILE_A with position deletes + FileScanTask fileATask = + assertThat(tasks) + .filteredOn(task -> task.file().location().equals(FILE_A.location())) + .first() + .as("Expected FILE_A in scan tasks") + .actual(); + assertThat(fileATask.deletes()).isNotEmpty(); // Has delete files + assertThat(fileATask.deletes().stream().map(DeleteFile::location)) + .contains(FILE_A_DELETES.location()); // FILE_A_DELETES is present + + // Verify FILE_B with position deletes + FileScanTask fileBTask = + assertThat(tasks) + .filteredOn(task -> task.file().location().equals(FILE_B.location())) + .first() + .as("Expected FILE_B in scan tasks") + .actual(); + assertThat(fileBTask.deletes()).isNotEmpty(); // Has delete files + assertThat(fileBTask.deletes().stream().map(DeleteFile::location)) + .contains(FILE_B_DELETES.location()); // FILE_B_DELETES is present + + // Verify FILE_C with equality deletes + FileScanTask fileCTask = + assertThat(tasks) + .filteredOn(task -> task.file().location().equals(FILE_C.location())) + .first() + .as("Expected FILE_C in scan tasks") + .actual(); + assertThat(fileCTask.deletes()).isNotEmpty(); // Has delete files + assertThat(fileCTask.deletes().stream().map(DeleteFile::location)) + .contains(FILE_C_EQUALITY_DELETES.location()); // FILE_C_EQUALITY_DELETES is present + } + } + + @ParameterizedTest + @EnumSource(PlanningMode.class) + void remoteScanPlanningWithDeletesAndFiltering( + Function planMode) + throws IOException { + configurePlanningBehavior(planMode); + Table table = restTableFor(scanPlanningCatalog(), "deletes_filtering_test"); + setParserContext(table); + + // Add FILE_B to have more data for filtering + table.newAppend().appendFile(FILE_B).commit(); + + // Add equality delete for FILE_B + table.newRowDelta().addDeletes(FILE_B_EQUALITY_DELETES).commit(); + + // Create a filtered scan and execute scan planning with filtering and deletes + try (CloseableIterable iterable = + table.newScan().filter(Expressions.lessThan("id", 4)).planFiles()) { + List tasks = Lists.newArrayList(iterable); + + // Verify scan planning works with both filtering and deletes + assertThat(tasks).hasSize(2); // 2 data files: FILE_A, FILE_B + + // FILE_A should have no delete files + FileScanTask fileATask = + assertThat(tasks) + .filteredOn(task -> task.file().location().equals(FILE_A.location())) + .first() + .as("Expected FILE_A in scan tasks") + .actual(); + assertThat(fileATask.deletes()).isEmpty(); // 0 delete files for FILE_A + + // FILE_B should have FILE_B_EQUALITY_DELETES + FileScanTask fileBTask = + assertThat(tasks) + .filteredOn(task -> task.file().location().equals(FILE_B.location())) + .first() + .as("Expected FILE_B in scan tasks") + .actual(); + assertThat(fileBTask.deletes()).hasSize(1); // 1 delete file: FILE_B_EQUALITY_DELETES + assertThat(fileBTask.deletes().get(0).location()) + .isEqualTo(FILE_B_EQUALITY_DELETES.location()); + } + } + + @ParameterizedTest + @EnumSource(PlanningMode.class) + void remoteScanPlanningDeletesCancellation( + Function planMode) + throws IOException { + configurePlanningBehavior(planMode); + Table table = restTableFor(scanPlanningCatalog(), "deletes_cancellation_test"); + setParserContext(table); + + // Add deletes to make the scenario more complex + table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A_EQUALITY_DELETES).commit(); + + RESTTableScan restTableScan = restTableScanFor(table); + + // Get the iterable (which may involve async planning with deletes) + try (CloseableIterable iterable = restTableScan.planFiles(); + CloseableIterator iterator = iterable.iterator()) { + // Test cancellation works with delete files present + // Resources will be closed automatically + } + + // Verify cancellation method is still accessible + assertThat(restTableScan.cancelPlan()).isFalse(); // No active plan at this point + } + + @ParameterizedTest + @EnumSource(PlanningMode.class) + void remoteScanPlanningWithTimeTravel( + Function planMode) { + // Test server-side scan planning with time travel (snapshot-based queries) + // Verify that snapshot IDs are correctly passed through the REST API + // and that historical scans return the correct files and deletes + configurePlanningBehavior(planMode); + + // Create table and add FILE_A (snapshot 1) + Table table = restTableFor(scanPlanningCatalog(), "snapshot_scan_test"); + setParserContext(table); + table.refresh(); + long snapshot1Id = table.currentSnapshot().snapshotId(); + + // Add FILE_B (snapshot 2) + table.newAppend().appendFile(FILE_B).commit(); + table.refresh(); + long snapshot2Id = table.currentSnapshot().snapshotId(); + assertThat(snapshot2Id).isNotEqualTo(snapshot1Id); + + // Add FILE_C and deletes (snapshots 3 and 4) + table.newAppend().appendFile(FILE_C).commit(); + table.newRowDelta().addDeletes(FILE_A_DELETES).commit(); + table.refresh(); + long snapshot4Id = table.currentSnapshot().snapshotId(); + assertThat(snapshot4Id).isNotEqualTo(snapshot2Id); + + // Test 1: Scan at snapshot 1 (should only see FILE_A, no deletes) + TableScan scan1 = table.newScan().useSnapshot(snapshot1Id); + CloseableIterable iterable1 = scan1.planFiles(); + List tasks1 = Lists.newArrayList(iterable1); + + assertThat(tasks1).hasSize(1); // Only FILE_A exists at snapshot 1 + assertThat(tasks1.get(0).file().location()).isEqualTo(FILE_A.location()); + assertThat(tasks1.get(0).deletes()).isEmpty(); // No deletes at snapshot 1 + + // Test 2: Scan at snapshot 2 (should see FILE_A and FILE_B, no deletes) + TableScan scan2 = table.newScan().useSnapshot(snapshot2Id); + CloseableIterable iterable2 = scan2.planFiles(); + List tasks2 = Lists.newArrayList(iterable2); + + assertThat(tasks2).hasSize(2); // FILE_A and FILE_B exist at snapshot 2 + assertThat(tasks2) + .map(task -> task.file().location()) + .containsExactlyInAnyOrder(FILE_A.location(), FILE_B.location()); + assertThat(tasks2).allMatch(task -> task.deletes().isEmpty()); // No deletes at snapshot 2 + + // Test 3: Scan at current snapshot (should see FILE_A, FILE_B, FILE_C, and FILE_A has deletes) + TableScan scan3 = table.newScan().useSnapshot(snapshot4Id); + CloseableIterable iterable3 = scan3.planFiles(); + List tasks3 = Lists.newArrayList(iterable3); + + assertThat(tasks3).hasSize(3); // All 3 data files exist at snapshot 4 + assertThat(tasks3) + .map(task -> task.file().location()) + .containsExactlyInAnyOrder(FILE_A.location(), FILE_B.location(), FILE_C.location()); + + // Verify FILE_A has deletes at snapshot 4 + FileScanTask fileATask = + assertThat(tasks3) + .filteredOn(task -> task.file().location().equals(FILE_A.location())) + .first() + .as("Expected FILE_A in scan tasks") + .actual(); + assertThat(fileATask.deletes()).hasSize(1); // FILE_A_DELETES present at snapshot 4 + assertThat(fileATask.deletes().get(0).location()).isEqualTo(FILE_A_DELETES.location()); + + // Verify FILE_B and FILE_C have no deletes at snapshot 4 + assertThat(tasks3) + .filteredOn( + task -> + task.file().location().equals(FILE_B.location()) + || task.file().location().equals(FILE_C.location())) + .allMatch(task -> task.deletes().isEmpty()); + } + + @ParameterizedTest + @EnumSource(PlanningMode.class) + public void scanPlanningWithMultiplePartitionSpecs() throws IOException { + configurePlanningBehavior(TestPlanningBehavior.Builder::synchronous); + + RESTTable table = restTableFor(scanPlanningCatalog(), "multiple_partition_specs"); + table.newFastAppend().appendFile(FILE_B).commit(); + + // Evolve partition spec to bucket by id with 8 buckets instead of 16 + table.updateSpec().removeField("data_bucket").addField(Expressions.bucket("data", 8)).commit(); + + // Create data file with new partition spec (spec-id=1) + PartitionSpec newSpec = table.spec(); + assertThat(newSpec.specId()).isEqualTo(1); + + DataFile fileWithNewSpec = + DataFiles.builder(newSpec) + .withPath("/path/to/data-new-spec.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("data_bucket_8=3") // 8-bucket partition + .withRecordCount(2) + .build(); + + table.newFastAppend().appendFile(fileWithNewSpec).commit(); + setParserContext(table); + + // Scan table - should return all 3 files despite different partition specs + try (CloseableIterable iterable = table.newScan().planFiles()) { + List tasks = Lists.newArrayList(iterable); + + // Verify all 3 files are present + assertThat(tasks).hasSize(3); + assertThat(tasks) + .map(task -> task.file().location()) + .containsExactlyInAnyOrder( + FILE_A.location(), FILE_B.location(), fileWithNewSpec.location()); + + // Verify files have correct partition spec IDs + assertThat(tasks) + .filteredOn( + task -> + task.file().location().equals(FILE_A.location()) + || task.file().location().equals(FILE_B.location())) + .allMatch(task -> task.spec().specId() == 0); + assertThat(tasks) + .filteredOn(task -> task.file().location().equals(fileWithNewSpec.location())) + .allMatch(task -> task.spec().specId() == 1); + } + } + + // ==================== Endpoint Support Tests ==================== + + /** Helper class to hold catalog and adapter for endpoint support tests. */ + private static class CatalogWithAdapter { + final RESTCatalog catalog; + final RESTCatalogAdapter adapter; + + CatalogWithAdapter(RESTCatalog catalog, RESTCatalogAdapter adapter) { + this.catalog = catalog; + this.adapter = adapter; + } + } + + // Helper: Create base catalog endpoints (namespace and table operations) + private List baseCatalogEndpoints() { + return ImmutableList.of( + Endpoint.V1_CREATE_NAMESPACE, + Endpoint.V1_LOAD_NAMESPACE, + Endpoint.V1_LIST_TABLES, + Endpoint.V1_CREATE_TABLE, + Endpoint.V1_LOAD_TABLE, + Endpoint.V1_UPDATE_TABLE); + } + + // Helper: Create endpoint list with base + specified planning endpoints + private List endpointsWithPlanning(Endpoint... planningEndpoints) { + return ImmutableList.builder() + .addAll(baseCatalogEndpoints()) + .add(planningEndpoints) + .build(); + } + + // Helper: Create catalog with custom endpoint support and optional planning behavior + private CatalogWithAdapter catalogWithEndpoints( + List endpoints, TestPlanningBehavior planningBehavior) { + RESTCatalogAdapter adapter = + Mockito.spy( + new RESTCatalogAdapter(backendCatalog) { + @Override + public T execute( + HTTPRequest request, + Class responseType, + Consumer errorHandler, + Consumer> responseHeaders) { + if (ResourcePaths.config().equals(request.path())) { + return castResponse( + responseType, ConfigResponse.builder().withEndpoints(endpoints).build()); + } + return super.execute(request, responseType, errorHandler, responseHeaders); + } + }); + + if (planningBehavior != null) { + adapter.setPlanningBehavior(planningBehavior); + } + + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + catalog.initialize( + "test", + ImmutableMap.of( + CatalogProperties.FILE_IO_IMPL, + "org.apache.iceberg.inmemory.InMemoryFileIO", + RESTCatalogProperties.REST_SCAN_PLANNING_ENABLED, + "true")); + return new CatalogWithAdapter(catalog, adapter); + } + + @Test + public void serverDoesNotSupportPlanningEndpoint() throws IOException { + // Server doesn't support scan planning at all - should fall back to client-side planning + CatalogWithAdapter catalogWithAdapter = catalogWithEndpoints(baseCatalogEndpoints(), null); + RESTCatalog catalog = catalogWithAdapter.catalog; + Table table = createTableWithScanPlanning(catalog, "no_planning_support"); + assertThat(table).isNotInstanceOf(RESTTable.class); + table.newAppend().appendFile(FILE_A).commit(); + + // Should fall back to client-side planning when endpoint is not supported + assertThat(table.newScan().planFiles()) + .hasSize(1) + .first() + .extracting(ContentScanTask::file) + .extracting(ContentFile::location) + .isEqualTo(FILE_A.location()); + } + + @Test + public void serverSupportsPlanningSyncOnlyNotAsync() { + // Server supports submit (sync) but not fetch (async polling) + // Use ASYNC planning to trigger SUBMITTED status, which will hit the Endpoint.check() + CatalogWithAdapter catalogWithAdapter = + catalogWithEndpoints( + endpointsWithPlanning( + Endpoint.V1_SUBMIT_TABLE_SCAN_PLAN, Endpoint.V1_FETCH_TABLE_SCAN_PLAN_TASKS), + TestPlanningBehavior.builder().asynchronous().build()); + + RESTCatalog catalog = catalogWithAdapter.catalog; + RESTTable table = restTableFor(catalog, "async_not_supported"); + setParserContext(table); + + // Should fail with UnsupportedOperationException when trying to fetch async plan result + // because V1_FETCH_TABLE_SCAN_PLAN endpoint is not supported + assertThatThrownBy(restTableScanFor(table)::planFiles) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Server does not support endpoint: %s", Endpoint.V1_FETCH_TABLE_SCAN_PLAN); + } + + @Test + public void serverSupportsPlanningButNotPagination() { + // Server supports planning but not task pagination endpoint + // Use synchronousWithPagination (tasksPerPage=1) to trigger pagination, which will hit + // Endpoint.check() + CatalogWithAdapter catalogWithAdapter = + catalogWithEndpoints( + endpointsWithPlanning( + Endpoint.V1_SUBMIT_TABLE_SCAN_PLAN, + Endpoint.V1_FETCH_TABLE_SCAN_PLAN, + Endpoint.V1_CANCEL_TABLE_SCAN_PLAN), + TestPlanningBehavior.builder().synchronousWithPagination().build()); + + RESTCatalog catalog = catalogWithAdapter.catalog; + RESTTable table = restTableFor(catalog, "pagination_not_supported"); + table.newAppend().appendFile(FILE_B).commit(); + setParserContext(table); + RESTTableScan scan = restTableScanFor(table); + + // Should fail with UnsupportedOperationException when trying to fetch paginated tasks + // because V1_FETCH_TABLE_SCAN_PLAN_TASKS endpoint is not supported + assertThatThrownBy(scan::planFiles) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage( + "Server does not support endpoint: %s", Endpoint.V1_FETCH_TABLE_SCAN_PLAN_TASKS); + } + + @Test + public void serverSupportsPlanningButNotCancellation() throws IOException { + // Server supports planning but not the cancel endpoint + CatalogWithAdapter catalogWithAdapter = + catalogWithEndpoints( + endpointsWithPlanning( + Endpoint.V1_SUBMIT_TABLE_SCAN_PLAN, + Endpoint.V1_FETCH_TABLE_SCAN_PLAN, + Endpoint.V1_FETCH_TABLE_SCAN_PLAN_TASKS), + TestPlanningBehavior.builder().asynchronous().build()); + + RESTCatalog catalog = catalogWithAdapter.catalog; + RESTTable table = restTableFor(catalog, "cancellation_not_supported"); + setParserContext(table); + RESTTableScan scan = restTableScanFor(table); + + // Get the iterable - this starts async planning + CloseableIterable iterable = scan.planFiles(); + + // Cancellation should not fail even though server doesn't support it + // The client should handle this gracefully by returning false + boolean cancelled = scan.cancelPlan(); + iterable.close(); + + // Verify no exception was thrown - cancelPlan returns false when endpoint not supported + assertThat(cancelled).isFalse(); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java b/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java index ebcced1b3766..fcf97e3abc26 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java @@ -225,4 +225,67 @@ public void viewWithMultipartNamespace() { assertThat(withPrefix.view(ident)).isEqualTo("v1/ws/catalog/namespaces/n%1Fs/views/view-name"); assertThat(withoutPrefix.view(ident)).isEqualTo("v1/namespaces/n%1Fs/views/view-name"); } + + @Test + public void planEndpointPath() { + TableIdentifier tableId = TableIdentifier.of("test_namespace", "test_table"); + + assertThat(withPrefix.planTableScan(tableId)) + .isEqualTo("v1/ws/catalog/namespaces/test_namespace/tables/test_table/plan"); + assertThat(withoutPrefix.planTableScan(tableId)) + .isEqualTo("v1/namespaces/test_namespace/tables/test_table/plan"); + + // Test with different identifiers + TableIdentifier complexId = TableIdentifier.of(Namespace.of("db", "schema"), "my_table"); + assertThat(withPrefix.planTableScan(complexId)) + .isEqualTo("v1/ws/catalog/namespaces/db%1Fschema/tables/my_table/plan"); + assertThat(withoutPrefix.planTableScan(complexId)) + .isEqualTo("v1/namespaces/db%1Fschema/tables/my_table/plan"); + } + + @Test + public void fetchScanTasksPath() { + TableIdentifier tableId = TableIdentifier.of("test_namespace", "test_table"); + + assertThat(withPrefix.fetchScanTasks(tableId)) + .isEqualTo("v1/ws/catalog/namespaces/test_namespace/tables/test_table/tasks"); + assertThat(withoutPrefix.fetchScanTasks(tableId)) + .isEqualTo("v1/namespaces/test_namespace/tables/test_table/tasks"); + + // Test with different identifiers + TableIdentifier complexId = TableIdentifier.of(Namespace.of("db", "schema"), "my_table"); + assertThat(withPrefix.fetchScanTasks(complexId)) + .isEqualTo("v1/ws/catalog/namespaces/db%1Fschema/tables/my_table/tasks"); + assertThat(withoutPrefix.fetchScanTasks(complexId)) + .isEqualTo("v1/namespaces/db%1Fschema/tables/my_table/tasks"); + } + + @Test + public void cancelPlanEndpointPath() { + TableIdentifier tableId = TableIdentifier.of("test_namespace", "test_table"); + String planId = "plan-abc-123"; + + assertThat(withPrefix.plan(tableId, planId)) + .isEqualTo("v1/ws/catalog/namespaces/test_namespace/tables/test_table/plan/plan-abc-123"); + assertThat(withoutPrefix.plan(tableId, planId)) + .isEqualTo("v1/namespaces/test_namespace/tables/test_table/plan/plan-abc-123"); + + // The planId contains a space which needs to be encoded + String spaceSeperatedPlanId = "plan with spaces"; + // The expected encoded version of the planId + String encodedPlanId = "plan+with+spaces"; + + assertThat(withPrefix.plan(tableId, spaceSeperatedPlanId)) + .isEqualTo( + "v1/ws/catalog/namespaces/test_namespace/tables/test_table/plan/" + encodedPlanId); + assertThat(withoutPrefix.plan(tableId, spaceSeperatedPlanId)) + .isEqualTo("v1/namespaces/test_namespace/tables/test_table/plan/" + encodedPlanId); + + // Test with different identifiers + TableIdentifier complexId = TableIdentifier.of(Namespace.of("db", "schema"), "my_table"); + assertThat(withPrefix.plan(complexId, "plan-xyz-789")) + .isEqualTo("v1/ws/catalog/namespaces/db%1Fschema/tables/my_table/plan/plan-xyz-789"); + assertThat(withoutPrefix.plan(complexId, "plan-xyz-789")) + .isEqualTo("v1/namespaces/db%1Fschema/tables/my_table/plan/plan-xyz-789"); + } } From 122c4408b101bcf846440348040caef016e16626 Mon Sep 17 00:00:00 2001 From: Raunaq Morarka Date: Wed, 10 Dec 2025 17:56:27 +0530 Subject: [PATCH 073/201] API: Reduce 'Scanning table' log verbosity for long list of strings (#14757) --- .../iceberg/expressions/ExpressionUtil.java | 17 ++++++++--------- .../iceberg/expressions/TestExpressionUtil.java | 10 ++++++++++ 2 files changed, 18 insertions(+), 9 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java index d3dc00d914c7..9bb2b713439d 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java @@ -24,7 +24,6 @@ import java.time.temporal.ChronoUnit; import java.util.List; import java.util.Locale; -import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.regex.Pattern; @@ -69,7 +68,6 @@ public class ExpressionUtil { "\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{7,9})?)?([-+]\\d{2}:\\d{2}|Z)"); static final int LONG_IN_PREDICATE_ABBREVIATION_THRESHOLD = 10; - private static final int LONG_IN_PREDICATE_ABBREVIATION_MIN_GAIN = 5; private ExpressionUtil() {} @@ -502,19 +500,20 @@ public String predicate(UnboundPredicate pred) { private static List abbreviateValues(List sanitizedValues) { if (sanitizedValues.size() >= LONG_IN_PREDICATE_ABBREVIATION_THRESHOLD) { - Set distinctValues = ImmutableSet.copyOf(sanitizedValues); - if (distinctValues.size() - <= sanitizedValues.size() - LONG_IN_PREDICATE_ABBREVIATION_MIN_GAIN) { - List abbreviatedList = Lists.newArrayListWithCapacity(distinctValues.size() + 1); - abbreviatedList.addAll(distinctValues); + List distinctValues = ImmutableSet.copyOf(sanitizedValues).asList(); + int abbreviatedSize = + Math.min(distinctValues.size(), LONG_IN_PREDICATE_ABBREVIATION_THRESHOLD); + List abbreviatedList = Lists.newArrayListWithCapacity(abbreviatedSize + 1); + abbreviatedList.addAll(distinctValues.subList(0, abbreviatedSize)); + if (abbreviatedSize < sanitizedValues.size()) { abbreviatedList.add( String.format( Locale.ROOT, "... (%d values hidden, %d in total)", - sanitizedValues.size() - distinctValues.size(), + sanitizedValues.size() - abbreviatedSize, sanitizedValues.size())); - return abbreviatedList; } + return abbreviatedList; } return sanitizedValues; } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java index ca08951b1f53..d9fe26eacc6b 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java @@ -115,6 +115,16 @@ public void testSanitizeLongIn() { .as("Sanitized string should be abbreviated") .isEqualTo("test IN ((2-digit-int), (3-digit-int), ... (8 values hidden, 10 in total))"); + Object[] tooLongStringsList = + IntStream.range(0, ExpressionUtil.LONG_IN_PREDICATE_ABBREVIATION_THRESHOLD + 5) + .mapToObj(i -> "string_" + i) + .toArray(); + + assertThat(ExpressionUtil.toSanitizedString(Expressions.in("test", tooLongStringsList))) + .as("Sanitized string should be abbreviated") + .isEqualTo( + "test IN ((hash-14128790), (hash-1056a62b), (hash-22fd6340), (hash-3f9d20e4), (hash-136200f0), (hash-25fc9033), (hash-681d31e2), (hash-6c1796d4), (hash-382d143e), (hash-272f4e5b), ... (5 values hidden, 15 in total))"); + // The sanitization resulting in an expression tree does not abbreviate List expectedValues = Lists.newArrayList(); expectedValues.addAll(Collections.nCopies(5, "(2-digit-int)")); From 11188387fd8e9b8f6944b760d294f262bc7bff32 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Wed, 10 Dec 2025 14:10:08 +0100 Subject: [PATCH 074/201] Flink: Dynamic Sink: Handle NoSuchNamespaceException properly (#14812) --- .../sink/dynamic/TableMetadataCache.java | 5 +- .../sink/dynamic/TestTableMetadataCache.java | 24 ++++++++++ .../flink/sink/dynamic/TestTableUpdater.java | 47 +++++++++++++++++++ 3 files changed, 74 insertions(+), 2 deletions(-) diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java index 8a8362a41996..e790d9a92977 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java @@ -29,6 +29,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.slf4j.Logger; @@ -198,8 +199,8 @@ private Tuple2 refreshTable(TableIdentifier identifier) { Table table = catalog.loadTable(identifier); update(identifier, table); return EXISTS; - } catch (NoSuchTableException e) { - LOG.debug("Table doesn't exist {}", identifier, e); + } catch (NoSuchTableException | NoSuchNamespaceException e) { + LOG.debug("Table or namespace doesn't exist {}", identifier, e); tableCache.put( identifier, new CacheItem(cacheRefreshClock.millis(), false, null, null, null, 1)); return Tuple2.of(false, e); diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java index 7f91d2f8d585..72b420c3452f 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java @@ -130,4 +130,28 @@ void testNoCacheRefreshingBeforeRefreshIntervalElapses() { assertThat(cacheItem).isNotNull(); assertThat(cacheItem.inputSchemas()).containsKeys(SCHEMA, SCHEMA2); } + + @Test + void testNoSuchNamespaceExceptionHandling() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.of("nonexistent_namespace", "myTable"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + + TableMetadataCache.ResolvedSchemaInfo result = cache.schema(tableIdentifier, SCHEMA, false); + + assertThat(result).isEqualTo(TableMetadataCache.NOT_FOUND); + assertThat(cache.getInternalCache().get(tableIdentifier)).isNotNull(); + } + + @Test + void testNoSuchTableExceptionHandling() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.nonexistent_table"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + + TableMetadataCache.ResolvedSchemaInfo result = cache.schema(tableIdentifier, SCHEMA, false); + + assertThat(result).isEqualTo(TableMetadataCache.NOT_FOUND); + assertThat(cache.getInternalCache().get(tableIdentifier)).isNotNull(); + } } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java index bcc5d8064517..a49624a65ebf 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java @@ -28,6 +28,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; import org.apache.iceberg.inmemory.InMemoryCatalog; @@ -211,4 +212,50 @@ void testDropUnusedColumns() { assertThat(tableSchema.findField("data")).isNotNull(); assertThat(tableSchema.findField("extra")).isNull(); } + + @Test + void testNamespaceAndTableCreation() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + SupportsNamespaces namespaceCatalog = (SupportsNamespaces) catalog; + TableIdentifier tableIdentifier = TableIdentifier.of("new_namespace", "myTable"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); + + assertThat(namespaceCatalog.namespaceExists(Namespace.of("new_namespace"))).isFalse(); + assertThat(catalog.tableExists(tableIdentifier)).isFalse(); + + Tuple2 result = + tableUpdater.update( + tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + + assertThat(namespaceCatalog.namespaceExists(Namespace.of("new_namespace"))).isTrue(); + + assertThat(catalog.tableExists(tableIdentifier)).isTrue(); + assertThat(result.f0.resolvedTableSchema().sameSchema(SCHEMA)).isTrue(); + assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testTableCreationWithExistingNamespace() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + SupportsNamespaces namespaceCatalog = (SupportsNamespaces) catalog; + Namespace namespace = Namespace.of("existing_namespace"); + namespaceCatalog.createNamespace(namespace); + + TableIdentifier tableIdentifier = TableIdentifier.of("existing_namespace", "myTable"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); + + assertThat(namespaceCatalog.namespaceExists(namespace)).isTrue(); + assertThat(catalog.tableExists(tableIdentifier)).isFalse(); + + Tuple2 result = + tableUpdater.update( + tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + + assertThat(namespaceCatalog.namespaceExists(namespace)).isTrue(); + assertThat(catalog.tableExists(tableIdentifier)).isTrue(); + assertThat(result.f0.resolvedTableSchema().sameSchema(SCHEMA)).isTrue(); + assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); + } } From 482d850c252c38f507476233227d1f2cc0e69eda Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 10 Dec 2025 18:02:01 +0100 Subject: [PATCH 075/201] Spark: Test all simple types in TestSelect (#14804) --- .../apache/iceberg/spark/sql/TestSelect.java | 36 +++++++++++++++++++ .../apache/iceberg/spark/sql/TestSelect.java | 36 +++++++++++++++++++ 2 files changed, 72 insertions(+) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 8fa8406278ea..2d4d63c1a127 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -22,6 +22,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.sql.Timestamp; import java.text.SimpleDateFormat; import java.util.Arrays; import java.util.Date; @@ -643,4 +644,39 @@ public void testRequiredNestedFieldInOptionalStructFilter() { assertEquals("Should return all expected rows", ImmutableList.of(row(0)), result); sql("DROP TABLE IF EXISTS %s", nestedStructTable); } + + @TestTemplate + public void simpleTypesInFilter() { + String tableName = tableName("simple_types_table"); + sql( + "CREATE TABLE IF NOT EXISTS %s (id bigint, boolean boolean, integer integer, long long, " + + "float float, double double, string string, date date, timestamp timestamp) USING iceberg", + tableName); + sql( + "INSERT INTO %s VALUES (1, true, 1, 1L, 1.1, 1.3, '1.5', to_date('2021-01-01'), to_timestamp('2021-01-01T00:00:00')), " + + "(2, false, 2, 2L, 2.2, 2.4, '2.6', to_date('2022-02-02'), to_timestamp('2022-02-02T00:00:00')), " + + "(3, true, 3, 3L, 3.3, 3.6, '3.9', to_date('2023-03-03'), to_timestamp('2023-03-03T00:00:00'))", + tableName); + assertThat(sql("SELECT id FROM %s where id > 1", tableName)).containsExactly(row(2L), row(3L)); + assertThat(sql("SELECT id, boolean FROM %s where boolean = true", tableName)) + .containsExactly(row(1L, true), row(3L, true)); + assertThat(sql("SELECT long FROM %s where long > 1", tableName)) + .containsExactly(row(2L), row(3L)); + assertThat(sql("SELECT float FROM %s where float > 1.1f", tableName)) + .containsExactly(row(2.2f), row(3.3f)); + assertThat(sql("SELECT double FROM %s where double > 1.3", tableName)) + .containsExactly(row(2.4d), row(3.6d)); + assertThat(sql("SELECT string FROM %s where string > '1.5'", tableName)) + .containsExactly(row("2.6"), row("3.9")); + java.sql.Date dateOne = java.sql.Date.valueOf("2022-02-02"); + java.sql.Date dateTwo = java.sql.Date.valueOf("2023-03-03"); + assertThat(sql("SELECT date FROM %s where date > to_date('2021-01-01')", tableName)) + .containsExactly(row(dateOne), row(dateTwo)); + assertThat( + sql("SELECT timestamp FROM %s where timestamp > to_timestamp('2021-01-01')", tableName)) + .containsExactly( + row(new Timestamp(dateOne.getTime())), row(new Timestamp(dateTwo.getTime()))); + + sql("DROP TABLE IF EXISTS %s", tableName); + } } diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 8fa8406278ea..2d4d63c1a127 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -22,6 +22,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.sql.Timestamp; import java.text.SimpleDateFormat; import java.util.Arrays; import java.util.Date; @@ -643,4 +644,39 @@ public void testRequiredNestedFieldInOptionalStructFilter() { assertEquals("Should return all expected rows", ImmutableList.of(row(0)), result); sql("DROP TABLE IF EXISTS %s", nestedStructTable); } + + @TestTemplate + public void simpleTypesInFilter() { + String tableName = tableName("simple_types_table"); + sql( + "CREATE TABLE IF NOT EXISTS %s (id bigint, boolean boolean, integer integer, long long, " + + "float float, double double, string string, date date, timestamp timestamp) USING iceberg", + tableName); + sql( + "INSERT INTO %s VALUES (1, true, 1, 1L, 1.1, 1.3, '1.5', to_date('2021-01-01'), to_timestamp('2021-01-01T00:00:00')), " + + "(2, false, 2, 2L, 2.2, 2.4, '2.6', to_date('2022-02-02'), to_timestamp('2022-02-02T00:00:00')), " + + "(3, true, 3, 3L, 3.3, 3.6, '3.9', to_date('2023-03-03'), to_timestamp('2023-03-03T00:00:00'))", + tableName); + assertThat(sql("SELECT id FROM %s where id > 1", tableName)).containsExactly(row(2L), row(3L)); + assertThat(sql("SELECT id, boolean FROM %s where boolean = true", tableName)) + .containsExactly(row(1L, true), row(3L, true)); + assertThat(sql("SELECT long FROM %s where long > 1", tableName)) + .containsExactly(row(2L), row(3L)); + assertThat(sql("SELECT float FROM %s where float > 1.1f", tableName)) + .containsExactly(row(2.2f), row(3.3f)); + assertThat(sql("SELECT double FROM %s where double > 1.3", tableName)) + .containsExactly(row(2.4d), row(3.6d)); + assertThat(sql("SELECT string FROM %s where string > '1.5'", tableName)) + .containsExactly(row("2.6"), row("3.9")); + java.sql.Date dateOne = java.sql.Date.valueOf("2022-02-02"); + java.sql.Date dateTwo = java.sql.Date.valueOf("2023-03-03"); + assertThat(sql("SELECT date FROM %s where date > to_date('2021-01-01')", tableName)) + .containsExactly(row(dateOne), row(dateTwo)); + assertThat( + sql("SELECT timestamp FROM %s where timestamp > to_timestamp('2021-01-01')", tableName)) + .containsExactly( + row(new Timestamp(dateOne.getTime())), row(new Timestamp(dateTwo.getTime()))); + + sql("DROP TABLE IF EXISTS %s", tableName); + } } From cbd35799dc70e989528b1c14d640fe91cdafa52d Mon Sep 17 00:00:00 2001 From: Sreesh Maheshwar Date: Wed, 10 Dec 2025 17:16:23 +0000 Subject: [PATCH 076/201] Encryption: Simplify Hive key handling and add transaction tests (#14752) * Encryption: Simplify Hive key handling and add transaction tests * spotless --- .../iceberg/hive/HiveTableOperations.java | 30 +++------ .../spark/sql/TestTableEncryption.java | 64 +++++++++++++++++-- 2 files changed, 67 insertions(+), 27 deletions(-) diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java index 62340fe40138..a97d82bb6f1a 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java @@ -91,12 +91,7 @@ public class HiveTableOperations extends BaseMetastoreTableOperations private String tableKeyId; private int encryptionDekLength; - // keys loaded from the latest metadata - private Optional> encryptedKeysFromMetadata = Optional.empty(); - - // keys added to EM (e.g. as a result of a FileAppend) but not committed into the latest metadata - // yet - private Optional> encryptedKeysPending = Optional.empty(); + private List encryptedKeys = List.of(); protected HiveTableOperations( Configuration conf, @@ -157,12 +152,9 @@ public EncryptionManager encryption() { encryptionProperties.put( TableProperties.ENCRYPTION_DEK_LENGTH, String.valueOf(encryptionDekLength)); - List keys = Lists.newLinkedList(); - encryptedKeysFromMetadata.ifPresent(keys::addAll); - encryptedKeysPending.ifPresent(keys::addAll); - encryptionManager = - EncryptionUtil.createEncryptionManager(keys, encryptionProperties, keyManagementClient); + EncryptionUtil.createEncryptionManager( + encryptedKeys, encryptionProperties, keyManagementClient); } else { return PlaintextEncryptionManager.instance(); } @@ -218,24 +210,20 @@ the table key parameter (along with existing snapshots) in the file, making the ? Integer.parseInt(dekLengthFromHMS) : TableProperties.ENCRYPTION_DEK_LENGTH_DEFAULT; - encryptedKeysFromMetadata = Optional.ofNullable(current().encryptionKeys()); + encryptedKeys = + Optional.ofNullable(current().encryptionKeys()) + .map(Lists::newLinkedList) + .orElseGet(Lists::newLinkedList); if (encryptionManager != null) { - encryptedKeysPending = Optional.of(Lists.newLinkedList()); - Set keyIdsFromMetadata = - encryptedKeysFromMetadata.orElseGet(Lists::newLinkedList).stream() - .map(EncryptedKey::keyId) - .collect(Collectors.toSet()); + encryptedKeys.stream().map(EncryptedKey::keyId).collect(Collectors.toSet()); for (EncryptedKey keyFromEM : EncryptionUtil.encryptionKeys(encryptionManager).values()) { if (!keyIdsFromMetadata.contains(keyFromEM.keyId())) { - encryptedKeysPending.get().add(keyFromEM); + encryptedKeys.add(keyFromEM); } } - - } else { - encryptedKeysPending = Optional.empty(); } // Force re-creation of encryption manager with updated keys diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java index 85e7f48b59d7..905516bff92c 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java @@ -112,8 +112,8 @@ private static List currentDataFiles(Table table) { @TestTemplate public void testRefresh() { - catalog.initialize(catalogName, catalogConfig); - Table table = catalog.loadTable(tableIdent); + validationCatalog.initialize(catalogName, catalogConfig); + Table table = validationCatalog.loadTable(tableIdent); assertThat(currentDataFiles(table)).isNotEmpty(); @@ -124,10 +124,26 @@ public void testRefresh() { } @TestTemplate - public void testTransaction() { - catalog.initialize(catalogName, catalogConfig); + public void testAppendTransaction() { + validationCatalog.initialize(catalogName, catalogConfig); + Table table = validationCatalog.loadTable(tableIdent); - Table table = catalog.loadTable(tableIdent); + List dataFiles = currentDataFiles(table); + Transaction transaction = table.newTransaction(); + AppendFiles append = transaction.newAppend(); + + // add an arbitrary datafile + append.appendFile(dataFiles.get(0)); + append.commit(); + transaction.commitTransaction(); + + assertThat(currentDataFiles(table)).hasSize(dataFiles.size() + 1); + } + + @TestTemplate + public void testConcurrentAppendTransactions() { + validationCatalog.initialize(catalogName, catalogConfig); + Table table = validationCatalog.loadTable(tableIdent); List dataFiles = currentDataFiles(table); Transaction transaction = table.newTransaction(); @@ -135,10 +151,46 @@ public void testTransaction() { // add an arbitrary datafile append.appendFile(dataFiles.get(0)); + + // append to the table in the meantime. use a separate load to avoid shared operations + validationCatalog.loadTable(tableIdent).newFastAppend().appendFile(dataFiles.get(0)).commit(); + append.commit(); transaction.commitTransaction(); - assertThat(currentDataFiles(table).size()).isEqualTo(dataFiles.size() + 1); + assertThat(currentDataFiles(table)).hasSize(dataFiles.size() + 2); + } + + // See CatalogTests#testConcurrentReplaceTransactions + @TestTemplate + public void testConcurrentReplaceTransactions() { + validationCatalog.initialize(catalogName, catalogConfig); + + Table table = validationCatalog.loadTable(tableIdent); + DataFile file = currentDataFiles(table).get(0); + Schema schema = table.schema(); + + // Write data for a replace transaction that will be committed later + Transaction secondReplace = + validationCatalog + .buildTable(tableIdent, schema) + .withProperty("encryption.key-id", UnitestKMS.MASTER_KEY_NAME1) + .replaceTransaction(); + secondReplace.newFastAppend().appendFile(file).commit(); + + // Commit another replace transaction first + Transaction firstReplace = + validationCatalog + .buildTable(tableIdent, schema) + .withProperty("encryption.key-id", UnitestKMS.MASTER_KEY_NAME1) + .replaceTransaction(); + firstReplace.newFastAppend().appendFile(file).commit(); + firstReplace.commitTransaction(); + + secondReplace.commitTransaction(); + + Table afterSecondReplace = validationCatalog.loadTable(tableIdent); + assertThat(currentDataFiles(afterSecondReplace)).hasSize(1); } @TestTemplate From 361131336622223c546187ff5369512343414a03 Mon Sep 17 00:00:00 2001 From: Thomas Powell Date: Wed, 10 Dec 2025 17:47:28 +0000 Subject: [PATCH 077/201] Handle SupportsWithPrefix in EncryptingFileIO (#14727) * Handle SupportsWithPrefix in EncryptingFileIO. * rename * additional tests for delegation --------- Co-authored-by: Thomas Powell --- .../iceberg/encryption/EncryptingFileIO.java | 29 +++++++- .../encryption/TestEncryptingFileIO.java | 68 +++++++++++++++++++ 2 files changed, 96 insertions(+), 1 deletion(-) create mode 100644 api/src/test/java/org/apache/iceberg/encryption/TestEncryptingFileIO.java diff --git a/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java b/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java index 2ab335a4d6c6..b4b99017fa66 100644 --- a/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java +++ b/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java @@ -30,8 +30,10 @@ import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestListFile; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileInfo; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.SupportsPrefixOperations; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -46,7 +48,11 @@ public static EncryptingFileIO combine(FileIO io, EncryptionManager em) { return combine(encryptingIO.io, em); } - return new EncryptingFileIO(io, em); + if (io instanceof SupportsPrefixOperations) { + return new WithSupportsPrefixOperations((SupportsPrefixOperations) io, em); + } else { + return new EncryptingFileIO(io, em); + } } private final FileIO io; @@ -206,4 +212,25 @@ public EncryptionKeyMetadata copy() { return new SimpleKeyMetadata(metadataBuffer.duplicate()); } } + + static class WithSupportsPrefixOperations extends EncryptingFileIO + implements SupportsPrefixOperations { + + private final SupportsPrefixOperations prefixIo; + + WithSupportsPrefixOperations(SupportsPrefixOperations io, EncryptionManager em) { + super(io, em); + this.prefixIo = io; + } + + @Override + public Iterable listPrefix(String prefix) { + return prefixIo.listPrefix(prefix); + } + + @Override + public void deletePrefix(String prefix) { + prefixIo.deletePrefix(prefix); + } + } } diff --git a/api/src/test/java/org/apache/iceberg/encryption/TestEncryptingFileIO.java b/api/src/test/java/org/apache/iceberg/encryption/TestEncryptingFileIO.java new file mode 100644 index 000000000000..944d89489c0a --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/encryption/TestEncryptingFileIO.java @@ -0,0 +1,68 @@ +/* + * 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.encryption; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.withSettings; + +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileInfo; +import org.apache.iceberg.io.SupportsPrefixOperations; +import org.junit.jupiter.api.Test; + +public class TestEncryptingFileIO { + + @Test + public void delegateEncryptingIOWithAndWithoutMixins() { + EncryptionManager em = mock(EncryptionManager.class); + + FileIO fileIONoMixins = mock(FileIO.class); + assertThat(EncryptingFileIO.combine(fileIONoMixins, em)) + .isInstanceOf(EncryptingFileIO.class) + .extracting(EncryptingFileIO::encryptionManager) + .isEqualTo(em); + + FileIO fileIOWithMixins = + mock(FileIO.class, withSettings().extraInterfaces(SupportsPrefixOperations.class)); + assertThat(EncryptingFileIO.combine(fileIOWithMixins, em)) + .isInstanceOf(EncryptingFileIO.WithSupportsPrefixOperations.class) + .extracting(EncryptingFileIO::encryptionManager) + .isEqualTo(em); + } + + @Test + public void prefixOperationsDelegation() { + EncryptionManager em = mock(EncryptionManager.class); + SupportsPrefixOperations delegate = mock(SupportsPrefixOperations.class); + + EncryptingFileIO.WithSupportsPrefixOperations fileIO = + (EncryptingFileIO.WithSupportsPrefixOperations) EncryptingFileIO.combine(delegate, em); + + String prefix = "prefix"; + Iterable fileInfos = mock(Iterable.class); + when(delegate.listPrefix(prefix)).thenReturn(fileInfos); + assertThat(fileIO.listPrefix(prefix)).isEqualTo(fileInfos); + + fileIO.deletePrefix(prefix); + verify(delegate).deletePrefix(prefix); + } +} From d894a0283e137c44d910acfc4a77119d9df2a005 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 10 Dec 2025 20:07:58 +0100 Subject: [PATCH 078/201] Core: Align CharSequenceSet impl with Data/DeleteFileSet (#11322) --- .../apache/iceberg/util/CharSequenceSet.java | 178 +++--------------- .../iceberg/util/CharSequenceWrapper.java | 5 +- .../iceberg/util/TestCharSequenceSet.java | 175 ++++++++++++++++- 3 files changed, 197 insertions(+), 161 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java b/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java index cfdac0104c47..c13cbfd0cc28 100644 --- a/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java +++ b/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java @@ -18,183 +18,53 @@ */ package org.apache.iceberg.util; -import java.io.Serializable; -import java.util.Collection; -import java.util.Iterator; -import java.util.Set; -import java.util.stream.Collectors; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Iterators; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.relocated.com.google.common.collect.Streams; -public class CharSequenceSet implements Set, Serializable { +public class CharSequenceSet extends WrapperSet { private static final ThreadLocal WRAPPERS = ThreadLocal.withInitial(() -> CharSequenceWrapper.wrap(null)); - public static CharSequenceSet of(Iterable charSequences) { - return new CharSequenceSet(charSequences); - } - - public static CharSequenceSet empty() { - return new CharSequenceSet(ImmutableList.of()); - } - - private final Set wrapperSet; - - private CharSequenceSet(Iterable charSequences) { - this.wrapperSet = - Sets.newHashSet(Iterables.transform(charSequences, CharSequenceWrapper::wrap)); + private CharSequenceSet() { + // needed for serialization } - @Override - public int size() { - return wrapperSet.size(); + private CharSequenceSet(Iterable charSequences) { + super( + Iterables.transform( + charSequences, + obj -> { + Preconditions.checkNotNull(obj, "Invalid object: null"); + return CharSequenceWrapper.wrap(obj); + })); } - @Override - public boolean isEmpty() { - return wrapperSet.isEmpty(); + public static CharSequenceSet of(Iterable charSequences) { + return new CharSequenceSet(charSequences); } - @Override - public boolean contains(Object obj) { - if (obj instanceof CharSequence) { - CharSequenceWrapper wrapper = WRAPPERS.get(); - boolean result = wrapperSet.contains(wrapper.set((CharSequence) obj)); - wrapper.set(null); // don't hold a reference to the value - return result; - } - return false; + public static CharSequenceSet empty() { + return new CharSequenceSet(); } @Override - public Iterator iterator() { - return Iterators.transform(wrapperSet.iterator(), CharSequenceWrapper::get); + protected Wrapper wrapper() { + return WRAPPERS.get(); } @Override - public Object[] toArray() { - return Iterators.toArray(iterator(), CharSequence.class); + protected Wrapper wrap(CharSequence file) { + return CharSequenceWrapper.wrap(file); } @Override - @SuppressWarnings("unchecked") - public T[] toArray(T[] destArray) { - int size = wrapperSet.size(); - if (destArray.length < size) { - return (T[]) toArray(); - } - - Iterator iter = iterator(); - int ind = 0; - while (iter.hasNext()) { - destArray[ind] = (T) iter.next(); - ind += 1; - } - - if (destArray.length > size) { - destArray[size] = null; - } - - return destArray; + protected Class elementClass() { + return CharSequence.class; } @Override public boolean add(CharSequence charSequence) { - return wrapperSet.add(CharSequenceWrapper.wrap(charSequence)); - } - - @Override - public boolean remove(Object obj) { - if (obj instanceof CharSequence) { - CharSequenceWrapper wrapper = WRAPPERS.get(); - boolean result = wrapperSet.remove(wrapper.set((CharSequence) obj)); - wrapper.set(null); // don't hold a reference to the value - return result; - } - return false; - } - - @Override - @SuppressWarnings("CollectionUndefinedEquality") - public boolean containsAll(Collection objects) { - if (objects != null) { - return Iterables.all(objects, this::contains); - } - return false; - } - - @Override - public boolean addAll(Collection charSequences) { - if (charSequences != null) { - return Iterables.addAll( - wrapperSet, Iterables.transform(charSequences, CharSequenceWrapper::wrap)); - } - return false; - } - - @Override - public boolean retainAll(Collection objects) { - if (objects != null) { - Set toRetain = - objects.stream() - .filter(CharSequence.class::isInstance) - .map(CharSequence.class::cast) - .map(CharSequenceWrapper::wrap) - .collect(Collectors.toSet()); - - return Iterables.retainAll(wrapperSet, toRetain); - } - - return false; - } - - @Override - @SuppressWarnings("CollectionUndefinedEquality") - public boolean removeAll(Collection objects) { - if (objects != null) { - return objects.stream().filter(this::remove).count() != 0; - } - - return false; - } - - @Override - public void clear() { - wrapperSet.clear(); - } - - @SuppressWarnings("CollectionUndefinedEquality") - @Override - public boolean equals(Object other) { - if (this == other) { - return true; - } else if (!(other instanceof Set)) { - return false; - } - - Set that = (Set) other; - - if (size() != that.size()) { - return false; - } - - try { - return containsAll(that); - } catch (ClassCastException | NullPointerException unused) { - return false; - } - } - - @Override - public int hashCode() { - return wrapperSet.stream().mapToInt(CharSequenceWrapper::hashCode).sum(); - } - - @Override - public String toString() { - return Streams.stream(iterator()).collect(Collectors.joining("CharSequenceSet({", ", ", "})")); + // method is needed to not break API compatibility + return super.add(charSequence); } } diff --git a/api/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java b/api/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java index 854264c1ae21..59e8eb712dc6 100644 --- a/api/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java +++ b/api/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java @@ -18,12 +18,11 @@ */ package org.apache.iceberg.util; -import java.io.Serializable; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.JavaHashes; /** Wrapper class to adapt CharSequence for use in maps and sets. */ -public class CharSequenceWrapper implements CharSequence, Serializable { +public class CharSequenceWrapper implements CharSequence, WrapperSet.Wrapper { public static CharSequenceWrapper wrap(CharSequence seq) { return new CharSequenceWrapper(seq); } @@ -39,6 +38,7 @@ private CharSequenceWrapper(CharSequence wrapped) { this.wrapped = wrapped; } + @Override public CharSequenceWrapper set(CharSequence newWrapped) { this.wrapped = newWrapped; this.hashCode = 0; @@ -46,6 +46,7 @@ public CharSequenceWrapper set(CharSequence newWrapped) { return this; } + @Override public CharSequence get() { return wrapped; } diff --git a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java index 324742c07a2d..093d2a0c6b87 100644 --- a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java +++ b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java @@ -19,10 +19,12 @@ package org.apache.iceberg.util; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Arrays; import java.util.Collections; import java.util.Set; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.junit.jupiter.api.Test; @@ -42,15 +44,115 @@ public void testSearchingInCharSequenceCollection() { @Test public void nullString() { - assertThat(CharSequenceSet.of(Arrays.asList((String) null))).contains((String) null); + assertThatThrownBy(() -> CharSequenceSet.of(Arrays.asList((String) null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); assertThat(CharSequenceSet.empty()).doesNotContain((String) null); } + @Test + public void emptySet() { + assertThat(CharSequenceSet.empty()).isEmpty(); + assertThat(CharSequenceSet.empty()).doesNotContain("a", "b", "c"); + } + + @Test + public void insertionOrderIsMaintained() { + CharSequenceSet set = CharSequenceSet.empty(); + set.addAll(ImmutableList.of("d", "a", "c")); + set.add("b"); + set.add("d"); + + assertThat(set).hasSize(4).containsExactly("d", "a", "c", "b"); + } + + @Test + public void clear() { + CharSequenceSet set = CharSequenceSet.of(ImmutableList.of("a", "b")); + set.clear(); + assertThat(set).isEmpty(); + } + + @Test + public void addAll() { + CharSequenceSet empty = CharSequenceSet.empty(); + assertThatThrownBy(() -> empty.add(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.addAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.addAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.addAll(Arrays.asList("a", null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + CharSequenceSet set = CharSequenceSet.empty(); + set.addAll(ImmutableList.of("b", "a", "c", "a")); + assertThat(set).hasSize(3).containsExactly("b", "a", "c"); + } + + @Test + public void contains() { + CharSequenceSet set = CharSequenceSet.of(ImmutableList.of("b", "a")); + assertThatThrownBy(() -> set.contains(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThat(set).hasSize(2).containsExactly("b", "a").doesNotContain("c").doesNotContain("d"); + + assertThatThrownBy(() -> CharSequenceSet.of(Arrays.asList("c", "b", null, "a"))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + } + + @Test + public void containsAll() { + CharSequenceSet set = CharSequenceSet.of(ImmutableList.of("b", "a")); + assertThatThrownBy(() -> set.containsAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> set.containsAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> set.containsAll(Arrays.asList("a", null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThat(set.containsAll(ImmutableList.of("a", "b"))).isTrue(); + assertThat(set.containsAll(ImmutableList.of("b", "a", "c"))).isFalse(); + assertThat(set.containsAll(ImmutableList.of("b"))).isTrue(); + } + @Test public void testRetainAll() { + CharSequenceSet empty = CharSequenceSet.empty(); + assertThatThrownBy(() -> empty.retainAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.retainAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.retainAll(Arrays.asList("123", null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.retainAll(ImmutableList.of("456", "789", 123))) + .isInstanceOf(ClassCastException.class) + .hasMessage("Cannot cast java.lang.Integer to java.lang.CharSequence"); + CharSequenceSet set = CharSequenceSet.of(ImmutableList.of("123", "456")); - assertThat(set.retainAll(ImmutableList.of("456", "789", 123))) + assertThat(set.retainAll(ImmutableList.of("456", "789", "555"))) .overridingErrorMessage("Set should be changed") .isTrue(); @@ -61,24 +163,74 @@ public void testRetainAll() { .overridingErrorMessage("Set should not be changed") .isFalse(); - assertThat(set.retainAll(ImmutableList.of(123, 456))) + assertThat(set.retainAll(ImmutableList.of("555", "789"))) .overridingErrorMessage("Set should be changed") .isTrue(); assertThat(set).isEmpty(); } + @Test + public void toArray() { + CharSequenceSet set = CharSequenceSet.of(ImmutableList.of("b", "a")); + assertThat(set.toArray()).hasSize(2).containsExactly("b", "a"); + + CharSequence[] array = new CharSequence[1]; + assertThat(set.toArray(array)).hasSize(2).containsExactly("b", "a"); + + array = new CharSequence[0]; + assertThat(set.toArray(array)).hasSize(2).containsExactly("b", "a"); + + array = new CharSequence[5]; + assertThat(set.toArray(array)).hasSize(5).containsExactly("b", "a", null, null, null); + + array = new CharSequence[2]; + assertThat(set.toArray(array)).hasSize(2).containsExactly("b", "a"); + } + + @Test + public void remove() { + CharSequenceSet set = CharSequenceSet.of(ImmutableSet.of("a", "b", "c")); + assertThatThrownBy(() -> set.remove(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + set.remove("a"); + assertThat(set).containsExactly("b", "c"); + set.remove("b"); + assertThat(set).containsExactly("c"); + set.remove("c"); + assertThat(set).isEmpty(); + } + @Test public void testRemoveAll() { + CharSequenceSet empty = CharSequenceSet.empty(); + assertThatThrownBy(() -> empty.removeAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.removeAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.removeAll(Arrays.asList("123", null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.removeAll(ImmutableList.of("123", 456))) + .isInstanceOf(ClassCastException.class) + .hasMessage("Cannot cast java.lang.Integer to java.lang.CharSequence"); + CharSequenceSet set = CharSequenceSet.of(ImmutableList.of("123", "456")); - assertThat(set.removeAll(ImmutableList.of("456", "789", 123))) + assertThat(set.removeAll(ImmutableList.of("456", "789"))) .overridingErrorMessage("Set should be changed") .isTrue(); assertThat(set).hasSize(1).contains("123"); set = CharSequenceSet.of(ImmutableList.of("123", "456")); - assertThat(set.removeAll(ImmutableList.of(123, 456))) + assertThat(set.removeAll(ImmutableList.of("333", "789"))) .overridingErrorMessage("Set should not be changed") .isFalse(); @@ -119,4 +271,17 @@ public void testEqualsAndHashCode() { .isEqualTo(set3.hashCode()) .isEqualTo(set4.hashCode()); } + + @Test + public void kryoSerialization() throws Exception { + CharSequenceSet charSequences = CharSequenceSet.of(ImmutableList.of("c", "b", "a")); + assertThat(TestHelpers.KryoHelpers.roundTripSerialize(charSequences)).isEqualTo(charSequences); + } + + @Test + public void javaSerialization() throws Exception { + CharSequenceSet charSequences = CharSequenceSet.of(ImmutableList.of("c", "b", "a")); + CharSequenceSet deserialize = TestHelpers.deserialize(TestHelpers.serialize(charSequences)); + assertThat(deserialize).isEqualTo(charSequences); + } } From 36940952929af15fc204cc8548d374932cbc70a0 Mon Sep 17 00:00:00 2001 From: Vladislav Sidorovich Date: Thu, 11 Dec 2025 00:42:18 +0100 Subject: [PATCH 079/201] Throw CommitFailedException when BQ returns FAILED_PRECONDITION. (#14801) --- .../gcp/bigquery/BigQueryMetastoreClientImpl.java | 4 ++-- .../gcp/bigquery/BigQueryTableOperations.java | 12 +----------- .../gcp/bigquery/TestBigQueryTableOperations.java | 5 ++--- 3 files changed, 5 insertions(+), 16 deletions(-) diff --git a/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryMetastoreClientImpl.java b/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryMetastoreClientImpl.java index 9af9844d84de..f61c37fb08de 100644 --- a/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryMetastoreClientImpl.java +++ b/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryMetastoreClientImpl.java @@ -61,6 +61,7 @@ import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.BadRequestException; +import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.ForbiddenException; import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchIcebergTableException; @@ -70,7 +71,6 @@ import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ServiceFailureException; import org.apache.iceberg.exceptions.ServiceUnavailableException; -import org.apache.iceberg.exceptions.ValidationException; 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.Maps; @@ -650,7 +650,7 @@ private static HttpResponse convertExceptionIfUnsuccessful(HttpResponse response case HttpStatusCodes.STATUS_CODE_FORBIDDEN: throw new ForbiddenException("%s", errorMessage); case HttpStatusCodes.STATUS_CODE_PRECONDITION_FAILED: - throw new ValidationException("%s", errorMessage); + throw new CommitFailedException("%s", errorMessage); case HttpStatusCodes.STATUS_CODE_NOT_FOUND: throw new IllegalArgumentException(errorMessage); case HttpStatusCodes.STATUS_CODE_SERVER_ERROR: diff --git a/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryTableOperations.java b/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryTableOperations.java index d57aab50530a..e5f0a449574c 100644 --- a/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryTableOperations.java +++ b/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryTableOperations.java @@ -21,7 +21,6 @@ import com.google.api.services.bigquery.model.ExternalCatalogTableOptions; import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableReference; -import java.util.Locale; import java.util.Map; import org.apache.iceberg.BaseMetastoreOperations; import org.apache.iceberg.BaseMetastoreTableOperations; @@ -178,16 +177,7 @@ private void updateTable( } options.setParameters(buildTableParameters(newMetadataLocation, metadata)); - try { - client.update(tableReference, table); - } catch (ValidationException e) { - if (e.getMessage().toLowerCase(Locale.ENGLISH).contains("etag mismatch")) { - throw new CommitFailedException( - "Updating table failed due to conflict updates (etag mismatch). Retry the update"); - } - - throw e; - } + client.update(tableReference, table); } // To make the table queryable from Hive, the user would likely be setting the HIVE_ENGINE_ENABLED diff --git a/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryTableOperations.java b/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryTableOperations.java index 9b8b90e1f83d..340410688295 100644 --- a/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryTableOperations.java +++ b/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryTableOperations.java @@ -177,12 +177,11 @@ public void failWhenEtagMismatch() throws Exception { org.apache.iceberg.Table loadedTable = catalog.loadTable(IDENTIFIER); when(client.update(any(), any())) - .thenThrow(new ValidationException("error message etag mismatch")); + .thenThrow(new CommitFailedException("error message etag mismatch")); assertThatThrownBy( () -> loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit()) .isInstanceOf(CommitFailedException.class) - .hasMessageContaining( - "Updating table failed due to conflict updates (etag mismatch). Retry the update"); + .hasMessage("error message etag mismatch"); } @Test From 5025d581ac292b04cbaa99fbe660277ddbceb9f1 Mon Sep 17 00:00:00 2001 From: Xianyang Liu Date: Thu, 11 Dec 2025 07:51:25 +0800 Subject: [PATCH 080/201] Build: Improvements around applying spotless for Scala (#14798) --- baseline.gradle | 27 +++++++++++++++------------ 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/baseline.gradle b/baseline.gradle index 54e67b84f853..438687f431a1 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -52,6 +52,7 @@ subprojects { apply plugin: 'com.palantir.baseline-exact-dependencies' apply plugin: 'com.diffplug.spotless' + String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") pluginManager.withPlugin('com.diffplug.spotless') { spotless { java { @@ -63,18 +64,21 @@ subprojects { licenseHeaderFile "$rootDir/.baseline/copyright/copyright-header-java.txt" } - // Configure different scalafmt rules for specific Scala version - if (project.name.startsWith("iceberg-spark") && project.name.endsWith("2.13")) { - scala { - target 'src/**/*.scala' - scalafmt("3.9.7").configFile("$rootDir/.baseline/scala/.scala213fmt.conf") - licenseHeaderFile "$rootDir/.baseline/copyright/copyright-header-java.txt", "package" + if (project.name.startsWith("iceberg-spark")) { + String scalafmtConfigFile = null + // Configure different scalafmt rules for specific Scala version + if (scalaVersion?.startsWith("2.12")) { + scalafmtConfigFile = "$rootDir/.baseline/scala/.scala212fmt.conf" + } else if (scalaVersion?.startsWith("2.13")) { + scalafmtConfigFile = "$rootDir/.baseline/scala/.scala213fmt.conf" } - } else if (project.name.startsWith("iceberg-spark") && project.name.endsWith("2.12")) { - scala { - target 'src/**/*.scala' - scalafmt("3.9.7").configFile("$rootDir/.baseline/scala/.scala212fmt.conf") - licenseHeaderFile "$rootDir/.baseline/copyright/copyright-header-java.txt", "package" + + if (scalafmtConfigFile != null) { + scala { + target 'src/**/*.scala' + scalafmt("3.9.7").configFile(scalafmtConfigFile) + licenseHeaderFile "$rootDir/.baseline/copyright/copyright-header-java.txt", "package" + } } } } @@ -182,7 +186,6 @@ subprojects { } pluginManager.withPlugin('scala') { - String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") tasks.withType(ScalaCompile).configureEach { scalaCompile -> if (scalaVersion?.startsWith("2.12")) { scalaCompile.scalaCompileOptions.additionalParameters = [ From 23dc32e5f1130c9d79d71cbf0be5b85162d8976c Mon Sep 17 00:00:00 2001 From: Prashant Singh <35593236+singhpk234@users.noreply.github.com> Date: Wed, 10 Dec 2025 23:20:46 -0800 Subject: [PATCH 081/201] REST: Implement Batch Scan for RESTTableScan (#14776) Co-authored-by: Prashant Singh --- .../org/apache/iceberg/BatchScanAdapter.java | 4 +-- .../org/apache/iceberg/rest/RESTTable.java | 7 ++++ .../iceberg/rest/TestRESTScanPlanning.java | 34 +++++++++++++++---- 3 files changed, 36 insertions(+), 9 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/BatchScanAdapter.java b/api/src/main/java/org/apache/iceberg/BatchScanAdapter.java index d8c5dda88558..f723e04d0c4e 100644 --- a/api/src/main/java/org/apache/iceberg/BatchScanAdapter.java +++ b/api/src/main/java/org/apache/iceberg/BatchScanAdapter.java @@ -25,11 +25,11 @@ import org.apache.iceberg.metrics.MetricsReporter; /** An adapter that allows using {@link TableScan} as {@link BatchScan}. */ -class BatchScanAdapter implements BatchScan { +public class BatchScanAdapter implements BatchScan { private final TableScan scan; - BatchScanAdapter(TableScan scan) { + public BatchScanAdapter(TableScan scan) { this.scan = scan; } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTTable.java b/core/src/main/java/org/apache/iceberg/rest/RESTTable.java index 5565122aaa6f..052e1432703f 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTTable.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTTable.java @@ -22,6 +22,8 @@ import java.util.Set; import java.util.function.Supplier; import org.apache.iceberg.BaseTable; +import org.apache.iceberg.BatchScan; +import org.apache.iceberg.BatchScanAdapter; import org.apache.iceberg.ImmutableTableScanContext; import org.apache.iceberg.TableOperations; import org.apache.iceberg.TableScan; @@ -67,4 +69,9 @@ public TableScan newScan() { resourcePaths, supportedEndpoints); } + + @Override + public BatchScan newBatchScan() { + return new BatchScanAdapter(newScan()); + } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java index ba13d1e3c1c4..f84197b0f16e 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java @@ -30,7 +30,6 @@ import static org.apache.iceberg.TestBase.SPEC; 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 com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; @@ -58,6 +57,7 @@ import org.apache.iceberg.MetadataTableUtils; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Scan; +import org.apache.iceberg.ScanTask; import org.apache.iceberg.Table; import org.apache.iceberg.TableScan; import org.apache.iceberg.catalog.Namespace; @@ -348,6 +348,25 @@ void scanPlanningWithAllTasksInSingleResponse( } } + @ParameterizedTest + @EnumSource(PlanningMode.class) + void scanPlanningWithBatchScan( + Function planMode) + throws IOException { + configurePlanningBehavior(planMode); + Table table = restTableFor(scanPlanningCatalog(), "batch_scan_table"); + setParserContext(table); + + // Verify actual data file is returned with correct count + try (CloseableIterable iterable = table.newBatchScan().planFiles()) { + List tasks = Lists.newArrayList(iterable); + + assertThat(tasks).hasSize(1); + assertThat(tasks.get(0).asFileScanTask().file().location()).isEqualTo(FILE_A.location()); + assertThat(tasks.get(0).asFileScanTask().deletes()).isEmpty(); + } + } + @Test public void nestedPlanTaskPagination() throws IOException { // Configure: synchronous planning with very small pages (creates nested plan task structure) @@ -409,11 +428,7 @@ public void iterableCloseTriggersCancel() throws IOException { @ParameterizedTest @EnumSource(MetadataTableType.class) - public void metadataTablesWithRemotePlanning(MetadataTableType type) throws IOException { - assumeThat(type) - .as("POSITION_DELETES table does not implement newScan() method") - .isNotEqualTo(MetadataTableType.POSITION_DELETES); - + public void metadataTablesWithRemotePlanning(MetadataTableType type) { configurePlanningBehavior(TestPlanningBehavior.Builder::synchronous); RESTTable table = restTableFor(scanPlanningCatalog(), "metadata_tables_test"); table.newAppend().appendFile(FILE_B).commit(); @@ -424,7 +439,12 @@ public void metadataTablesWithRemotePlanning(MetadataTableType type) throws IOEx // tasks, this test just verifies that metadata tables can be scanned with RESTTable. Table metadataTableInstance = MetadataTableUtils.createMetadataTableInstance(table, type); assertThat(metadataTableInstance).isNotNull(); - assertThat(metadataTableInstance.newScan().planFiles()).isNotEmpty(); + if (type.equals(MetadataTableType.POSITION_DELETES)) { + // Position deletes table only uses batch scan + assertThat(metadataTableInstance.newBatchScan().planFiles()).isNotEmpty(); + } else { + assertThat(metadataTableInstance.newScan().planFiles()).isNotEmpty(); + } } @ParameterizedTest From f5317674e2f7ce8eac54fb062073d33f833750ff Mon Sep 17 00:00:00 2001 From: Manikandan R Date: Thu, 11 Dec 2025 23:03:10 +0530 Subject: [PATCH 082/201] Support for TIME, TIMESTAMPNTZ_NANO, UUID types in Inclusive Metrics Evaluator (#13195) * Support for Variant TIME, TIMESTAMPNTZ_NANO, TIMESTAMPTZ_NANO & UUID types in Inclusive Metrics Evaluator * Used conversion maps to replace switch cases * Used suppress warnings to ignore cyclomatic complexity for now. added to do to fix this later using correct approach * Addressed review comments * Parameterized all timestamp related tests * Added test case * Addressed review comments. 1. Simplified conversions for timestamp, timestampnano, date based on the conversions used in literals 2. Simplified the tests * test simplifications and cleanup * Rebased. Variable renaming * Fixed checkstyle warnings * Added more test cases * Combined tests by parameterizing with different params * Use nested arguments * Fixed checkstyle issues * Addressed review comments --------- Co-authored-by: mani --- .../expressions/VariantExpressionUtil.java | 44 ++- ...tInclusiveMetricsEvaluatorWithExtract.java | 350 ++++++++++++++++++ 2 files changed, 390 insertions(+), 4 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/expressions/VariantExpressionUtil.java b/api/src/main/java/org/apache/iceberg/expressions/VariantExpressionUtil.java index dca11d5e4662..daec5216f0ef 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/VariantExpressionUtil.java +++ b/api/src/main/java/org/apache/iceberg/expressions/VariantExpressionUtil.java @@ -24,13 +24,11 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.variants.PhysicalType; import org.apache.iceberg.variants.VariantValue; class VariantExpressionUtil { - // TODO: Implement PhysicalType.TIME - // TODO: Implement PhysicalType.TIMESTAMPNTZ_NANO and PhysicalType.TIMESTAMPTZ_NANO - // TODO: Implement PhysicalType.UUID private static final Map NO_CONVERSION_NEEDED = ImmutableMap.builder() .put(Types.IntegerType.get(), PhysicalType.INT32) @@ -40,6 +38,10 @@ class VariantExpressionUtil { .put(Types.DateType.get(), PhysicalType.DATE) .put(Types.TimestampType.withoutZone(), PhysicalType.TIMESTAMPNTZ) .put(Types.TimestampType.withZone(), PhysicalType.TIMESTAMPTZ) + .put(Types.TimestampNanoType.withoutZone(), PhysicalType.TIMESTAMPNTZ_NANOS) + .put(Types.TimestampNanoType.withZone(), PhysicalType.TIMESTAMPTZ_NANOS) + .put(Types.TimeType.get(), PhysicalType.TIME) + .put(Types.UUIDType.get(), PhysicalType.UUID) .put(Types.StringType.get(), PhysicalType.STRING) .put(Types.BinaryType.get(), PhysicalType.BINARY) .put(Types.UnknownType.get(), PhysicalType.NULL) @@ -47,7 +49,7 @@ class VariantExpressionUtil { private VariantExpressionUtil() {} - @SuppressWarnings("unchecked") + @SuppressWarnings({"unchecked", "CyclomaticComplexity"}) static T castTo(VariantValue value, Type type) { if (value == null) { return null; @@ -111,6 +113,40 @@ static T castTo(VariantValue value, Type type) { } break; + case TIMESTAMP: + if (value.type() == PhysicalType.TIMESTAMPTZ_NANOS + || value.type() == PhysicalType.TIMESTAMPNTZ_NANOS) { + return (T) + (Long) DateTimeUtil.nanosToMicros(((Number) value.asPrimitive().get()).longValue()); + } else if (value.type() == PhysicalType.DATE) { + return (T) + (Long) + DateTimeUtil.microsFromTimestamp( + DateTimeUtil.dateFromDays(((Number) value.asPrimitive().get()).intValue()) + .atStartOfDay()); + } + break; + case TIMESTAMP_NANO: + if (value.type() == PhysicalType.TIMESTAMPTZ || value.type() == PhysicalType.TIMESTAMPNTZ) { + return (T) + (Long) DateTimeUtil.microsToNanos(((Number) value.asPrimitive().get()).longValue()); + } else if (value.type() == PhysicalType.DATE) { + return (T) + (Long) + DateTimeUtil.nanosFromTimestamp( + DateTimeUtil.dateFromDays(((Number) value.asPrimitive().get()).intValue()) + .atStartOfDay()); + } + break; + case DATE: + if (value.type() == PhysicalType.TIMESTAMPTZ || value.type() == PhysicalType.TIMESTAMPNTZ) { + return (T) + (Integer) DateTimeUtil.microsToDays(((Number) value.asPrimitive().get()).longValue()); + } else if (value.type() == PhysicalType.TIMESTAMPTZ_NANOS + || value.type() == PhysicalType.TIMESTAMPNTZ_NANOS) { + return (T) + (Integer) DateTimeUtil.nanosToDays(((Number) value.asPrimitive().get()).longValue()); + } } return null; diff --git a/core/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluatorWithExtract.java b/core/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluatorWithExtract.java index c629270bbb0f..1ee0a7ac635a 100644 --- a/core/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluatorWithExtract.java +++ b/core/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluatorWithExtract.java @@ -41,8 +41,10 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.nio.ByteBuffer; import java.util.List; import java.util.Map; +import java.util.UUID; import org.apache.iceberg.DataFile; import org.apache.iceberg.Schema; import org.apache.iceberg.TestHelpers.Row; @@ -52,10 +54,13 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.variants.PhysicalType; import org.apache.iceberg.variants.VariantTestUtil; +import org.apache.iceberg.variants.VariantValue; import org.apache.iceberg.variants.Variants; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.FieldSource; public class TestInclusiveMetricsEvaluatorWithExtract { @@ -683,4 +688,349 @@ public void testIntegerNotIn() { .as("Should read: id above upper bound (85 > 79, 86 > 79)") .isTrue(); } + + private static final List DATEANDTIMESTAMPTYPESEQPARAMETERS = + List.of( + Arguments.of( + Types.TimestampNanoType.withoutZone().toString(), + Arguments.of( + "1970-01-11T00:00:01.123456789", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456"), + false), + Arguments.of( + "1970-01-31T00:00:01.123456", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456"), + true), + Arguments.of( + "1970-03-21T00:00:01.123456789", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456"), + true), + Arguments.of( + "1970-03-31T00:00:01.123456", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456"), + true), + Arguments.of( + "1970-04-01T00:00:01.123456789", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456"), + false), + Arguments.of( + "1970-01-11T00:00:01.123456789", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31"), + false), + Arguments.of( + "1970-01-31T00:00:01.123456789", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31"), + true), + Arguments.of( + "1970-03-21T00:00:01.123456789", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31"), + true), + Arguments.of( + "1970-03-31T00:00:00.000000000", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31"), + true), + Arguments.of( + "1970-04-01T00:00:01.123456789", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31"), + false)), + Arguments.of( + Types.DateType.get().toString(), + Arguments.of( + "1970-01-11", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456"), + false), + Arguments.of( + "1970-01-31", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456"), + true), + Arguments.of( + "1970-03-21", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456"), + true), + Arguments.of( + "1970-03-31", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456"), + true), + Arguments.of( + "1970-04-01", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456"), + false), + Arguments.of( + "1970-01-11", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456"), + false), + Arguments.of( + "1970-01-31", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456"), + true), + Arguments.of( + "1970-03-21", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456"), + true), + Arguments.of( + "1970-03-31", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456"), + true), + Arguments.of( + "1970-04-01", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456"), + false)), + Arguments.of( + Types.TimestampType.withoutZone().toString(), + Arguments.of( + "1970-01-11T00:00:01.123456", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456"), + false), + Arguments.of( + "1970-01-31T00:00:01.123456", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456"), + true), + Arguments.of( + "1970-03-21T00:00:01.123456", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456"), + true), + Arguments.of( + "1970-03-31T00:00:01.123456", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456"), + true), + Arguments.of( + "1970-04-01T00:00:01.123456", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456"), + false), + Arguments.of( + "1970-01-11T00:00:01.123456", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31"), + false), + Arguments.of( + "1970-01-31T00:00:01.123456", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31"), + true), + Arguments.of( + "1970-03-21T00:00:01.123456", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31"), + true), + Arguments.of( + "1970-03-31T00:00:00.000000", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31"), + true), + Arguments.of( + "1970-04-01T00:00:01.123456", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31"), + false))); + + @ParameterizedTest + @FieldSource("DATEANDTIMESTAMPTYPESEQPARAMETERS") + public void testDateAndTimestampTypesEq(String variantType, Arguments args) { + // lower bounds + Map lowerBounds = + ImmutableMap.of( + 2, + VariantTestUtil.variantBuffer( + Map.of("$['event_timestamp']", (VariantValue) args.get()[1]))); + // upper bounds + Map upperBounds = + ImmutableMap.of( + 2, + VariantTestUtil.variantBuffer( + Map.of("$['event_timestamp']", (VariantValue) args.get()[2]))); + + DataFile file = + new TestDataFile("file.parquet", Row.of(), 50, null, null, null, lowerBounds, upperBounds); + Expression expr = equal(extract("variant", "$.event_timestamp", variantType), args.get()[0]); + assertThat(shouldRead(expr, file)).isEqualTo(args.get()[3]); + } + + private static final List DATEANDTIMESTAMPTYPESNOTEQPARAMETERS = + List.of( + Arguments.of( + Types.TimestampNanoType.withoutZone().toString(), + Arguments.of( + "1970-01-11T00:00:01.123456", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-01-31T00:00:01.123456", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-03-01T00:00:01.123456", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-03-31T00:00:01.123456", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-04-01T00:00:01.123456", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-01-11T00:00:01.123456789", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31")), + Arguments.of( + "1970-01-31T00:00:01.123456789", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31")), + Arguments.of( + "1970-03-01T00:00:01.123456789", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31")), + Arguments.of( + "1970-03-31T00:00:01.123456789", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31")), + Arguments.of( + "1970-04-01T00:00:01.123456789", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31"))), + Arguments.of( + Types.DateType.get().toString(), + Arguments.of( + "1970-01-31", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-03-01", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-03-31", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-04-01", + Variants.ofIsoTimestampntz("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntz("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-01-11", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-03-01", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-01-31", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-03-31", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-04-01", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456"))), + Arguments.of( + Types.TimestampType.withoutZone().toString(), + Arguments.of( + "1970-01-11T00:00:01.123456", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31")), + Arguments.of( + "1970-03-01T00:00:01.123456", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31")), + Arguments.of( + "1970-01-31T00:00:01.123456", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31")), + Arguments.of( + "1970-03-31T00:00:01.123456", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31")), + Arguments.of( + "1970-04-01T00:00:01.123456", + Variants.ofIsoDate("1970-01-31"), + Variants.ofIsoDate("1970-03-31")), + Arguments.of( + "1970-01-11T00:00:01.123456", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-01-31T00:00:01.123456", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-03-01T00:00:01.123456", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-03-31T00:00:01.123456", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456")), + Arguments.of( + "1970-04-01T00:00:01.123456", + Variants.ofIsoTimestampntzNanos("1970-01-31T00:00:01.123456"), + Variants.ofIsoTimestampntzNanos("1970-03-31T00:00:01.123456")))); + + @ParameterizedTest + @FieldSource("DATEANDTIMESTAMPTYPESNOTEQPARAMETERS") + public void testDateAndTimestampTypesNotEq(String variantType, Arguments args) { + // lower bounds + Map lowerBounds = + ImmutableMap.of( + 2, + VariantTestUtil.variantBuffer( + Map.of("$['event_timestamp']", (VariantValue) args.get()[1]))); + // upper bounds + Map upperBounds = + ImmutableMap.of( + 2, + VariantTestUtil.variantBuffer( + Map.of("$['event_timestamp']", (VariantValue) args.get()[2]))); + + DataFile file = + new TestDataFile("file.parquet", Row.of(), 50, null, null, null, lowerBounds, upperBounds); + Expression expr = notEqual(extract("variant", "$.event_timestamp", variantType), args.get()[0]); + assertThat(shouldRead(expr, file)).as("Should read: many possible timestamps" + expr).isTrue(); + } + + @Test + public void testUUIDEq() { + UUID uuid = UUID.randomUUID(); + // lower bounds + Map lowerBounds = + ImmutableMap.of( + 2, VariantTestUtil.variantBuffer(Map.of("$['event_uuid']", Variants.ofUUID(uuid)))); + // upper bounds + Map upperBounds = + ImmutableMap.of( + 2, VariantTestUtil.variantBuffer(Map.of("$['event_uuid']", Variants.ofUUID(uuid)))); + DataFile file = + new TestDataFile("file.parquet", Row.of(), 50, null, null, null, lowerBounds, upperBounds); + Expression expr = equal(extract("variant", "$.event_uuid", PhysicalType.UUID.name()), uuid); + assertThat(shouldRead(expr, file)).as("Should read: many possible UUIDs" + expr).isTrue(); + } } From fc981b499a09cef352f123daa70668d765dfe282 Mon Sep 17 00:00:00 2001 From: aiborodin Date: Fri, 12 Dec 2025 05:33:05 +1100 Subject: [PATCH 083/201] Flink: Log on cache refresh in dynamic sink (#14792) --- .../sink/dynamic/TableMetadataCache.java | 30 ++++++++++++++----- 1 file changed, 22 insertions(+), 8 deletions(-) diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java index e790d9a92977..3be8bbcd9123 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java @@ -79,7 +79,7 @@ Tuple2 exists(TableIdentifier identifier) { CacheItem cached = tableCache.get(identifier); if (cached != null && Boolean.TRUE.equals(cached.tableExists)) { return EXISTS; - } else if (needsRefresh(cached, true)) { + } else if (needsRefresh(identifier, cached, true)) { return refreshTable(identifier); } else { return NOT_EXISTS; @@ -116,7 +116,7 @@ private String branch(TableIdentifier identifier, String branch, boolean allowRe return branch; } - if (needsRefresh(cached, allowRefresh)) { + if (needsRefresh(identifier, cached, allowRefresh)) { refreshTable(identifier); return branch(identifier, branch, false); } else { @@ -156,7 +156,7 @@ private ResolvedSchemaInfo schema( } } - if (needsRefresh(cached, allowRefresh)) { + if (needsRefresh(identifier, cached, allowRefresh)) { refreshTable(identifier); return schema(identifier, input, false, dropUnusedColumns); } else if (compatible != null) { @@ -186,7 +186,7 @@ private PartitionSpec spec(TableIdentifier identifier, PartitionSpec spec, boole } } - if (needsRefresh(cached, allowRefresh)) { + if (needsRefresh(identifier, cached, allowRefresh)) { refreshTable(identifier); return spec(identifier, spec, false); } else { @@ -207,10 +207,24 @@ private Tuple2 refreshTable(TableIdentifier identifier) { } } - private boolean needsRefresh(CacheItem cacheItem, boolean allowRefresh) { - return allowRefresh - && (cacheItem == null - || cacheRefreshClock.millis() - cacheItem.createdTimestampMillis > refreshMs); + private boolean needsRefresh( + TableIdentifier identifier, CacheItem cacheItem, boolean allowRefresh) { + if (!allowRefresh) { + return false; + } + + if (cacheItem == null) { + return true; + } + + long nowMillis = cacheRefreshClock.millis(); + long timeElapsedMillis = nowMillis - cacheItem.createdTimestampMillis; + if (timeElapsedMillis > refreshMs) { + LOG.info("Refreshing table metadata for {} after {} millis", identifier, timeElapsedMillis); + return true; + } + + return false; } public void invalidate(TableIdentifier identifier) { From 7418f4986de820efe04570b28392a0db2f640ec9 Mon Sep 17 00:00:00 2001 From: Prashant Singh <35593236+singhpk234@users.noreply.github.com> Date: Thu, 11 Dec 2025 15:12:14 -0800 Subject: [PATCH 084/201] Core: disable flaky test for batchScan RemoteScanPlanning (#14826) Co-authored-by: Prashant Singh --- .../test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java index f84197b0f16e..6996570b3c56 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java @@ -348,6 +348,7 @@ void scanPlanningWithAllTasksInSingleResponse( } } + @Disabled("Temporarily disabled: Fix tracked via issue-14823") @ParameterizedTest @EnumSource(PlanningMode.class) void scanPlanningWithBatchScan( From e90b06cc5f07b948a56021d129f128dd8b81123c Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 12 Dec 2025 03:41:14 +0100 Subject: [PATCH 085/201] Flink: Backport: Dynamic Sink: Handle NoSuchNamespaceException properly (#14812) (#14819) This is a clean backport of #14812 for Flink 1.20 and 2.0. --- .../sink/dynamic/TableMetadataCache.java | 5 +- .../sink/dynamic/TestTableMetadataCache.java | 24 ++++++++++ .../flink/sink/dynamic/TestTableUpdater.java | 47 +++++++++++++++++++ .../sink/dynamic/TableMetadataCache.java | 5 +- .../sink/dynamic/TestTableMetadataCache.java | 24 ++++++++++ .../flink/sink/dynamic/TestTableUpdater.java | 47 +++++++++++++++++++ 6 files changed, 148 insertions(+), 4 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java index 8a8362a41996..e790d9a92977 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java @@ -29,6 +29,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.slf4j.Logger; @@ -198,8 +199,8 @@ private Tuple2 refreshTable(TableIdentifier identifier) { Table table = catalog.loadTable(identifier); update(identifier, table); return EXISTS; - } catch (NoSuchTableException e) { - LOG.debug("Table doesn't exist {}", identifier, e); + } catch (NoSuchTableException | NoSuchNamespaceException e) { + LOG.debug("Table or namespace doesn't exist {}", identifier, e); tableCache.put( identifier, new CacheItem(cacheRefreshClock.millis(), false, null, null, null, 1)); return Tuple2.of(false, e); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java index 7f91d2f8d585..72b420c3452f 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java @@ -130,4 +130,28 @@ void testNoCacheRefreshingBeforeRefreshIntervalElapses() { assertThat(cacheItem).isNotNull(); assertThat(cacheItem.inputSchemas()).containsKeys(SCHEMA, SCHEMA2); } + + @Test + void testNoSuchNamespaceExceptionHandling() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.of("nonexistent_namespace", "myTable"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + + TableMetadataCache.ResolvedSchemaInfo result = cache.schema(tableIdentifier, SCHEMA, false); + + assertThat(result).isEqualTo(TableMetadataCache.NOT_FOUND); + assertThat(cache.getInternalCache().get(tableIdentifier)).isNotNull(); + } + + @Test + void testNoSuchTableExceptionHandling() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.nonexistent_table"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + + TableMetadataCache.ResolvedSchemaInfo result = cache.schema(tableIdentifier, SCHEMA, false); + + assertThat(result).isEqualTo(TableMetadataCache.NOT_FOUND); + assertThat(cache.getInternalCache().get(tableIdentifier)).isNotNull(); + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java index bcc5d8064517..a49624a65ebf 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java @@ -28,6 +28,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; import org.apache.iceberg.inmemory.InMemoryCatalog; @@ -211,4 +212,50 @@ void testDropUnusedColumns() { assertThat(tableSchema.findField("data")).isNotNull(); assertThat(tableSchema.findField("extra")).isNull(); } + + @Test + void testNamespaceAndTableCreation() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + SupportsNamespaces namespaceCatalog = (SupportsNamespaces) catalog; + TableIdentifier tableIdentifier = TableIdentifier.of("new_namespace", "myTable"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); + + assertThat(namespaceCatalog.namespaceExists(Namespace.of("new_namespace"))).isFalse(); + assertThat(catalog.tableExists(tableIdentifier)).isFalse(); + + Tuple2 result = + tableUpdater.update( + tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + + assertThat(namespaceCatalog.namespaceExists(Namespace.of("new_namespace"))).isTrue(); + + assertThat(catalog.tableExists(tableIdentifier)).isTrue(); + assertThat(result.f0.resolvedTableSchema().sameSchema(SCHEMA)).isTrue(); + assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testTableCreationWithExistingNamespace() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + SupportsNamespaces namespaceCatalog = (SupportsNamespaces) catalog; + Namespace namespace = Namespace.of("existing_namespace"); + namespaceCatalog.createNamespace(namespace); + + TableIdentifier tableIdentifier = TableIdentifier.of("existing_namespace", "myTable"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); + + assertThat(namespaceCatalog.namespaceExists(namespace)).isTrue(); + assertThat(catalog.tableExists(tableIdentifier)).isFalse(); + + Tuple2 result = + tableUpdater.update( + tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + + assertThat(namespaceCatalog.namespaceExists(namespace)).isTrue(); + assertThat(catalog.tableExists(tableIdentifier)).isTrue(); + assertThat(result.f0.resolvedTableSchema().sameSchema(SCHEMA)).isTrue(); + assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); + } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java index 8a8362a41996..e790d9a92977 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java @@ -29,6 +29,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.slf4j.Logger; @@ -198,8 +199,8 @@ private Tuple2 refreshTable(TableIdentifier identifier) { Table table = catalog.loadTable(identifier); update(identifier, table); return EXISTS; - } catch (NoSuchTableException e) { - LOG.debug("Table doesn't exist {}", identifier, e); + } catch (NoSuchTableException | NoSuchNamespaceException e) { + LOG.debug("Table or namespace doesn't exist {}", identifier, e); tableCache.put( identifier, new CacheItem(cacheRefreshClock.millis(), false, null, null, null, 1)); return Tuple2.of(false, e); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java index 7f91d2f8d585..72b420c3452f 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java @@ -130,4 +130,28 @@ void testNoCacheRefreshingBeforeRefreshIntervalElapses() { assertThat(cacheItem).isNotNull(); assertThat(cacheItem.inputSchemas()).containsKeys(SCHEMA, SCHEMA2); } + + @Test + void testNoSuchNamespaceExceptionHandling() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.of("nonexistent_namespace", "myTable"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + + TableMetadataCache.ResolvedSchemaInfo result = cache.schema(tableIdentifier, SCHEMA, false); + + assertThat(result).isEqualTo(TableMetadataCache.NOT_FOUND); + assertThat(cache.getInternalCache().get(tableIdentifier)).isNotNull(); + } + + @Test + void testNoSuchTableExceptionHandling() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + TableIdentifier tableIdentifier = TableIdentifier.parse("default.nonexistent_table"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + + TableMetadataCache.ResolvedSchemaInfo result = cache.schema(tableIdentifier, SCHEMA, false); + + assertThat(result).isEqualTo(TableMetadataCache.NOT_FOUND); + assertThat(cache.getInternalCache().get(tableIdentifier)).isNotNull(); + } } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java index bcc5d8064517..a49624a65ebf 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java @@ -28,6 +28,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.sink.TestFlinkIcebergSinkBase; import org.apache.iceberg.inmemory.InMemoryCatalog; @@ -211,4 +212,50 @@ void testDropUnusedColumns() { assertThat(tableSchema.findField("data")).isNotNull(); assertThat(tableSchema.findField("extra")).isNull(); } + + @Test + void testNamespaceAndTableCreation() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + SupportsNamespaces namespaceCatalog = (SupportsNamespaces) catalog; + TableIdentifier tableIdentifier = TableIdentifier.of("new_namespace", "myTable"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); + + assertThat(namespaceCatalog.namespaceExists(Namespace.of("new_namespace"))).isFalse(); + assertThat(catalog.tableExists(tableIdentifier)).isFalse(); + + Tuple2 result = + tableUpdater.update( + tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + + assertThat(namespaceCatalog.namespaceExists(Namespace.of("new_namespace"))).isTrue(); + + assertThat(catalog.tableExists(tableIdentifier)).isTrue(); + assertThat(result.f0.resolvedTableSchema().sameSchema(SCHEMA)).isTrue(); + assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); + } + + @Test + void testTableCreationWithExistingNamespace() { + Catalog catalog = CATALOG_EXTENSION.catalog(); + SupportsNamespaces namespaceCatalog = (SupportsNamespaces) catalog; + Namespace namespace = Namespace.of("existing_namespace"); + namespaceCatalog.createNamespace(namespace); + + TableIdentifier tableIdentifier = TableIdentifier.of("existing_namespace", "myTable"); + TableMetadataCache cache = new TableMetadataCache(catalog, 10, Long.MAX_VALUE, 10); + TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); + + assertThat(namespaceCatalog.namespaceExists(namespace)).isTrue(); + assertThat(catalog.tableExists(tableIdentifier)).isFalse(); + + Tuple2 result = + tableUpdater.update( + tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + + assertThat(namespaceCatalog.namespaceExists(namespace)).isTrue(); + assertThat(catalog.tableExists(tableIdentifier)).isTrue(); + assertThat(result.f0.resolvedTableSchema().sameSchema(SCHEMA)).isTrue(); + assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); + } } From cc02655c7f70f1ac04273070835c4616bbb56d5f Mon Sep 17 00:00:00 2001 From: aiborodin Date: Fri, 12 Dec 2025 18:54:40 +1100 Subject: [PATCH 086/201] Flink: Backport: Log on cache refresh in dynamic sink (#14828)a Backports #14792 --- .../sink/dynamic/TableMetadataCache.java | 30 ++++++++++++++----- .../sink/dynamic/TableMetadataCache.java | 30 ++++++++++++++----- 2 files changed, 44 insertions(+), 16 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java index e790d9a92977..3be8bbcd9123 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java @@ -79,7 +79,7 @@ Tuple2 exists(TableIdentifier identifier) { CacheItem cached = tableCache.get(identifier); if (cached != null && Boolean.TRUE.equals(cached.tableExists)) { return EXISTS; - } else if (needsRefresh(cached, true)) { + } else if (needsRefresh(identifier, cached, true)) { return refreshTable(identifier); } else { return NOT_EXISTS; @@ -116,7 +116,7 @@ private String branch(TableIdentifier identifier, String branch, boolean allowRe return branch; } - if (needsRefresh(cached, allowRefresh)) { + if (needsRefresh(identifier, cached, allowRefresh)) { refreshTable(identifier); return branch(identifier, branch, false); } else { @@ -156,7 +156,7 @@ private ResolvedSchemaInfo schema( } } - if (needsRefresh(cached, allowRefresh)) { + if (needsRefresh(identifier, cached, allowRefresh)) { refreshTable(identifier); return schema(identifier, input, false, dropUnusedColumns); } else if (compatible != null) { @@ -186,7 +186,7 @@ private PartitionSpec spec(TableIdentifier identifier, PartitionSpec spec, boole } } - if (needsRefresh(cached, allowRefresh)) { + if (needsRefresh(identifier, cached, allowRefresh)) { refreshTable(identifier); return spec(identifier, spec, false); } else { @@ -207,10 +207,24 @@ private Tuple2 refreshTable(TableIdentifier identifier) { } } - private boolean needsRefresh(CacheItem cacheItem, boolean allowRefresh) { - return allowRefresh - && (cacheItem == null - || cacheRefreshClock.millis() - cacheItem.createdTimestampMillis > refreshMs); + private boolean needsRefresh( + TableIdentifier identifier, CacheItem cacheItem, boolean allowRefresh) { + if (!allowRefresh) { + return false; + } + + if (cacheItem == null) { + return true; + } + + long nowMillis = cacheRefreshClock.millis(); + long timeElapsedMillis = nowMillis - cacheItem.createdTimestampMillis; + if (timeElapsedMillis > refreshMs) { + LOG.info("Refreshing table metadata for {} after {} millis", identifier, timeElapsedMillis); + return true; + } + + return false; } public void invalidate(TableIdentifier identifier) { diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java index e790d9a92977..3be8bbcd9123 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableMetadataCache.java @@ -79,7 +79,7 @@ Tuple2 exists(TableIdentifier identifier) { CacheItem cached = tableCache.get(identifier); if (cached != null && Boolean.TRUE.equals(cached.tableExists)) { return EXISTS; - } else if (needsRefresh(cached, true)) { + } else if (needsRefresh(identifier, cached, true)) { return refreshTable(identifier); } else { return NOT_EXISTS; @@ -116,7 +116,7 @@ private String branch(TableIdentifier identifier, String branch, boolean allowRe return branch; } - if (needsRefresh(cached, allowRefresh)) { + if (needsRefresh(identifier, cached, allowRefresh)) { refreshTable(identifier); return branch(identifier, branch, false); } else { @@ -156,7 +156,7 @@ private ResolvedSchemaInfo schema( } } - if (needsRefresh(cached, allowRefresh)) { + if (needsRefresh(identifier, cached, allowRefresh)) { refreshTable(identifier); return schema(identifier, input, false, dropUnusedColumns); } else if (compatible != null) { @@ -186,7 +186,7 @@ private PartitionSpec spec(TableIdentifier identifier, PartitionSpec spec, boole } } - if (needsRefresh(cached, allowRefresh)) { + if (needsRefresh(identifier, cached, allowRefresh)) { refreshTable(identifier); return spec(identifier, spec, false); } else { @@ -207,10 +207,24 @@ private Tuple2 refreshTable(TableIdentifier identifier) { } } - private boolean needsRefresh(CacheItem cacheItem, boolean allowRefresh) { - return allowRefresh - && (cacheItem == null - || cacheRefreshClock.millis() - cacheItem.createdTimestampMillis > refreshMs); + private boolean needsRefresh( + TableIdentifier identifier, CacheItem cacheItem, boolean allowRefresh) { + if (!allowRefresh) { + return false; + } + + if (cacheItem == null) { + return true; + } + + long nowMillis = cacheRefreshClock.millis(); + long timeElapsedMillis = nowMillis - cacheItem.createdTimestampMillis; + if (timeElapsedMillis > refreshMs) { + LOG.info("Refreshing table metadata for {} after {} millis", identifier, timeElapsedMillis); + return true; + } + + return false; } public void invalidate(TableIdentifier identifier) { From c68f0418774782f73abca8d9e9c366198910c101 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Fri, 12 Dec 2025 12:40:26 +0100 Subject: [PATCH 087/201] Core: Expose the stats of the manifest file content cache (#13560) --- .../org/apache/iceberg/ManifestFiles.java | 6 ++ .../iceberg/metrics/CacheMetricsReport.java | 39 ++++++++++++ .../apache/iceberg/TestManifestCaching.java | 14 +++-- .../metrics/TestCacheMetricsReport.java | 62 +++++++++++++++++++ 4 files changed, 116 insertions(+), 5 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/metrics/CacheMetricsReport.java create mode 100644 core/src/test/java/org/apache/iceberg/metrics/TestCacheMetricsReport.java diff --git a/core/src/main/java/org/apache/iceberg/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java index 739f0be251df..86c4e680673b 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java @@ -33,6 +33,7 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.metrics.CacheMetricsReport; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -86,6 +87,11 @@ public static void dropCache(FileIO fileIO) { CONTENT_CACHES.cleanUp(); } + /** Get statistics of the manifest file content cache for a FileIO. */ + public static CacheMetricsReport contentCacheStats(FileIO io) { + return CacheMetricsReport.of(contentCache(io).stats()); + } + /** * Returns a {@link CloseableIterable} of file paths in the {@link ManifestFile}. * diff --git a/core/src/main/java/org/apache/iceberg/metrics/CacheMetricsReport.java b/core/src/main/java/org/apache/iceberg/metrics/CacheMetricsReport.java new file mode 100644 index 000000000000..7ea6961857f4 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/metrics/CacheMetricsReport.java @@ -0,0 +1,39 @@ +/* + * 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.metrics; + +import com.github.benmanes.caffeine.cache.stats.CacheStats; +import org.immutables.value.Value; + +@Value.Immutable +public abstract class CacheMetricsReport implements MetricsReport { + public abstract long hitCount(); + + public abstract long missCount(); + + public abstract long evictionCount(); + + public static CacheMetricsReport of(CacheStats stats) { + return ImmutableCacheMetricsReport.builder() + .hitCount(stats.hitCount()) + .missCount(stats.missCount()) + .evictionCount(stats.evictionCount()) + .build(); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestManifestCaching.java b/core/src/test/java/org/apache/iceberg/TestManifestCaching.java index 9a944c1583d0..57acb7d92d05 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestCaching.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestCaching.java @@ -78,16 +78,16 @@ public void testPlanWithCache() throws Exception { assertThat(cache.estimatedCacheSize()) .as("All manifest files should be cached") .isEqualTo(numFiles); - assertThat(cache.stats().loadCount()) + assertThat(cache.stats().loadSuccessCount()) .as("All manifest files should be recently loaded") .isEqualTo(numFiles); - long missCount = cache.stats().missCount(); + long missCount = ManifestFiles.contentCacheStats(table.io()).missCount(); // planFiles and verify that cache size still the same TableScan scan2 = table.newScan(); assertThat(scan2.planFiles()).hasSize(numFiles); assertThat(cache.estimatedCacheSize()).isEqualTo(numFiles); - assertThat(cache.stats().missCount()) + assertThat(ManifestFiles.contentCacheStats(table.io()).missCount()) .as("All manifest file reads should hit cache") .isEqualTo(missCount); @@ -115,10 +115,14 @@ public void testPlanWithSmallCache() throws Exception { assertThat(cache.maxTotalBytes()).isEqualTo(1); assertThat(scan.planFiles()).hasSize(numFiles); assertThat(cache.estimatedCacheSize()).isEqualTo(0); - assertThat(cache.stats().loadCount()) + assertThat(cache.stats().loadSuccessCount()) .as("File should not be loaded through cache") .isEqualTo(0); - assertThat(cache.stats().requestCount()).as("Cache should not serve file").isEqualTo(0); + assertThat( + ManifestFiles.contentCacheStats(table.io()).hitCount() + + ManifestFiles.contentCacheStats(table.io()).missCount()) + .as("Cache should not serve file") + .isEqualTo(0); ManifestFiles.dropCache(scan.table().io()); } diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestCacheMetricsReport.java b/core/src/test/java/org/apache/iceberg/metrics/TestCacheMetricsReport.java new file mode 100644 index 000000000000..bbaca531d68a --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/metrics/TestCacheMetricsReport.java @@ -0,0 +1,62 @@ +/* + * 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.metrics; + +import static org.assertj.core.api.Assertions.assertThat; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.Weigher; +import org.junit.jupiter.api.Test; + +public class TestCacheMetricsReport { + @Test + public void testNoInputStats() { + CacheMetricsReport cacheMetrics = CacheMetricsReport.of(Caffeine.newBuilder().build().stats()); + + assertThat(cacheMetrics.hitCount()).isZero(); + assertThat(cacheMetrics.missCount()).isZero(); + assertThat(cacheMetrics.evictionCount()).isZero(); + } + + @Test + public void testCacheMetricsFromCaffeineCache() { + int maxTotalWeight = 300; + + Cache inputCache = + Caffeine.newBuilder() + .maximumWeight(maxTotalWeight) + .weigher((Weigher) (key, value) -> value * 100) + .recordStats() + .build(); + + inputCache.get(1, key -> key); + inputCache.get(1, key -> key); + inputCache.get(2, key -> key); + inputCache.get(3, key -> key); // This evicts the other entries due to max weight + + inputCache.cleanUp(); + + CacheMetricsReport cacheMetrics = CacheMetricsReport.of(inputCache.stats()); + + assertThat(cacheMetrics.hitCount()).isOne(); + assertThat(cacheMetrics.missCount()).isEqualTo(3); + assertThat(cacheMetrics.evictionCount()).isEqualTo(2); + } +} From 849f218d8d9a6783ba04d2d4eafc07754ec3885c Mon Sep 17 00:00:00 2001 From: MehulBatra <66407733+MehulBatra@users.noreply.github.com> Date: Fri, 12 Dec 2025 17:14:52 +0530 Subject: [PATCH 088/201] Docs: Add Apache Fluss integration link (#14829) --- site/nav.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/site/nav.yml b/site/nav.yml index df8073e55cca..58f7dc4d2aa4 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -64,6 +64,7 @@ nav: - Apache Amoro: integrations/amoro.md - Apache Doris: https://doris.apache.org/docs/dev/lakehouse/catalogs/iceberg-catalog - Apache Druid: https://druid.apache.org/docs/latest/development/extensions-contrib/iceberg/ + - Apache Fluss: https://fluss.apache.org/docs/next/streaming-lakehouse/integrate-data-lakes/iceberg/ - BladePipe: https://www.bladepipe.com/docs/dataMigrationAndSync/datasource_func/Iceberg/props_for_iceberg_ds - ClickHouse: https://clickhouse.com/docs/en/engines/table-engines/integrations/iceberg - Daft: integrations/daft.md From 41b5af3e8a9ffaa709f08171113a8abcd201d765 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?N=C3=A1ndor=20Koll=C3=A1r?= Date: Fri, 12 Dec 2025 16:51:16 +0100 Subject: [PATCH 089/201] Azure: KeyManagementClient implementation for Azure Key Vault (#13186) --- azure-bundle/LICENSE | 7 ++ azure-bundle/build.gradle | 1 + .../TestAzureKeyManagementClient.java | 84 +++++++++++++++++++ .../apache/iceberg/azure/AzureProperties.java | 22 +++++ .../AzureKeyManagementClient.java | 68 +++++++++++++++ .../iceberg/azure/TestAzureProperties.java | 7 ++ build.gradle | 1 + 7 files changed, 190 insertions(+) create mode 100644 azure/src/integration/java/org/apache/iceberg/azure/keymanagement/TestAzureKeyManagementClient.java create mode 100644 azure/src/main/java/org/apache/iceberg/azure/keymanagement/AzureKeyManagementClient.java diff --git a/azure-bundle/LICENSE b/azure-bundle/LICENSE index 62069e51acb6..eae11b8f4023 100644 --- a/azure-bundle/LICENSE +++ b/azure-bundle/LICENSE @@ -243,6 +243,13 @@ License: The MIT License (MIT) - http://opensource.org/licenses/MIT -------------------------------------------------------------------------------- + +Group: com.azure Name: azure-security-keyvault-keys Version: 4.10.2 +Project URL: https://github.com/Azure/azure-sdk-for-java +License: The MIT License (MIT) - http://opensource.org/licenses/MIT + +-------------------------------------------------------------------------------- + Group: com.azure Name: azure-storage-internal-avro Version: 12.16.1 Project URL: https://github.com/Azure/azure-sdk-for-java License: The MIT License (MIT) - http://opensource.org/licenses/MIT diff --git a/azure-bundle/build.gradle b/azure-bundle/build.gradle index 46d1b3daea5b..0bdc30fdaa7e 100644 --- a/azure-bundle/build.gradle +++ b/azure-bundle/build.gradle @@ -26,6 +26,7 @@ project(":iceberg-azure-bundle") { dependencies { implementation platform(libs.azuresdk.bom) implementation "com.azure:azure-storage-file-datalake" + implementation "com.azure:azure-security-keyvault-keys" implementation "com.azure:azure-identity" } diff --git a/azure/src/integration/java/org/apache/iceberg/azure/keymanagement/TestAzureKeyManagementClient.java b/azure/src/integration/java/org/apache/iceberg/azure/keymanagement/TestAzureKeyManagementClient.java new file mode 100644 index 000000000000..32adcd46b702 --- /dev/null +++ b/azure/src/integration/java/org/apache/iceberg/azure/keymanagement/TestAzureKeyManagementClient.java @@ -0,0 +1,84 @@ +/* + * 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.azure.keymanagement; + +import static org.apache.iceberg.azure.AzureProperties.AZURE_KEYVAULT_URL; +import static org.assertj.core.api.Assertions.assertThat; + +import com.azure.identity.DefaultAzureCredentialBuilder; +import com.azure.security.keyvault.keys.KeyClient; +import com.azure.security.keyvault.keys.KeyClientBuilder; +import com.azure.security.keyvault.keys.models.KeyType; +import java.nio.ByteBuffer; +import java.time.Duration; +import org.apache.iceberg.encryption.KeyManagementClient; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.condition.EnabledIfEnvironmentVariable; +import org.junit.jupiter.api.condition.EnabledIfEnvironmentVariables; + +@EnabledIfEnvironmentVariables({ + @EnabledIfEnvironmentVariable(named = "AZURE_KEYVAULT_URL", matches = ".*") +}) +public class TestAzureKeyManagementClient { + private static final String ICEBERG_TEST_KEY_NAME = "iceberg-test-key"; + + private static KeyClient keyClient; + + private static KeyManagementClient azureKeyManagementClient; + + @BeforeAll + public static void beforeClass() { + String keyVaultUri = System.getenv("AZURE_KEYVAULT_URL"); + keyClient = + new KeyClientBuilder() + .vaultUrl(keyVaultUri) + .credential(new DefaultAzureCredentialBuilder().build()) + .buildClient(); + keyClient.createKey(ICEBERG_TEST_KEY_NAME, KeyType.RSA); + azureKeyManagementClient = new AzureKeyManagementClient(); + azureKeyManagementClient.initialize(ImmutableMap.of(AZURE_KEYVAULT_URL, keyVaultUri)); + } + + @AfterAll + public static void afterClass() { + if (keyClient != null) { + keyClient.beginDeleteKey(ICEBERG_TEST_KEY_NAME).waitForCompletion(Duration.ofMinutes(3)); + keyClient.purgeDeletedKey(ICEBERG_TEST_KEY_NAME); + } + } + + @Test + public void keyWrapping() { + ByteBuffer key = ByteBuffer.wrap("table-master-key".getBytes()); + + ByteBuffer encryptedKey = azureKeyManagementClient.wrapKey(key, ICEBERG_TEST_KEY_NAME); + ByteBuffer decryptedKey = + azureKeyManagementClient.unwrapKey(encryptedKey, ICEBERG_TEST_KEY_NAME); + + assertThat(decryptedKey).isEqualTo(key); + } + + @Test + public void keyGenerationNotSupported() { + assertThat(azureKeyManagementClient.supportsKeyGeneration()).isFalse(); + } +} diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index 38ac573b59fa..73e99e029221 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -21,6 +21,7 @@ import com.azure.core.credential.AccessToken; import com.azure.core.credential.TokenCredential; import com.azure.core.credential.TokenRequestContext; +import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm; import com.azure.storage.common.StorageSharedKeyCredential; import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; import java.io.Serializable; @@ -48,6 +49,9 @@ public class AzureProperties implements Serializable { public static final String ADLS_SHARED_KEY_ACCOUNT_NAME = "adls.auth.shared-key.account.name"; public static final String ADLS_SHARED_KEY_ACCOUNT_KEY = "adls.auth.shared-key.account.key"; public static final String ADLS_TOKEN = "adls.token"; + public static final String AZURE_KEYVAULT_URL = "azure.keyvault.url"; + public static final String AZURE_KEYVAULT_KEY_WRAP_ALGORITHM = + "azure.keyvault.key-wrap-algorithm"; /** * Configure the ADLS token credential provider used to get {@link TokenCredential}. A fully @@ -91,6 +95,8 @@ public class AzureProperties implements Serializable { private boolean adlsRefreshCredentialsEnabled; private String token; private Map allProperties = Collections.emptyMap(); + private String keyWrapAlgorithm; + private String keyVaultUrl; public AzureProperties() {} @@ -124,6 +130,14 @@ public AzureProperties(Map properties) { PropertyUtil.propertyAsBoolean(properties, ADLS_REFRESH_CREDENTIALS_ENABLED, true); this.token = properties.get(ADLS_TOKEN); this.allProperties = SerializableMap.copyOf(properties); + if (properties.containsKey(AZURE_KEYVAULT_URL)) { + this.keyVaultUrl = properties.get(AZURE_KEYVAULT_URL); + } + + this.keyWrapAlgorithm = + properties.getOrDefault( + AzureProperties.AZURE_KEYVAULT_KEY_WRAP_ALGORITHM, + KeyWrapAlgorithm.RSA_OAEP_256.getValue()); } public Optional adlsReadBlockSize() { @@ -189,4 +203,12 @@ public Mono getToken(TokenRequestContext request) { builder.endpoint("https://" + account); } } + + public KeyWrapAlgorithm keyWrapAlgorithm() { + return KeyWrapAlgorithm.fromString(this.keyWrapAlgorithm); + } + + public Optional keyVaultUrl() { + return Optional.ofNullable(this.keyVaultUrl); + } } diff --git a/azure/src/main/java/org/apache/iceberg/azure/keymanagement/AzureKeyManagementClient.java b/azure/src/main/java/org/apache/iceberg/azure/keymanagement/AzureKeyManagementClient.java new file mode 100644 index 000000000000..4732d3d410c4 --- /dev/null +++ b/azure/src/main/java/org/apache/iceberg/azure/keymanagement/AzureKeyManagementClient.java @@ -0,0 +1,68 @@ +/* + * 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.azure.keymanagement; + +import com.azure.security.keyvault.keys.KeyClient; +import com.azure.security.keyvault.keys.KeyClientBuilder; +import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm; +import com.azure.security.keyvault.keys.cryptography.models.UnwrapResult; +import com.azure.security.keyvault.keys.cryptography.models.WrapResult; +import java.nio.ByteBuffer; +import java.util.Map; +import org.apache.iceberg.azure.AdlsTokenCredentialProviders; +import org.apache.iceberg.azure.AzureProperties; +import org.apache.iceberg.encryption.KeyManagementClient; +import org.apache.iceberg.util.ByteBuffers; + +/** Azure key management client which connects to Azure Key Vault. */ +public class AzureKeyManagementClient implements KeyManagementClient { + private KeyClient keyClient; + private KeyWrapAlgorithm keyWrapAlgorithm; + + @Override + public void initialize(Map properties) { + AzureProperties azureProperties = new AzureProperties(properties); + + this.keyWrapAlgorithm = azureProperties.keyWrapAlgorithm(); + KeyClientBuilder keyClientBuilder = new KeyClientBuilder(); + azureProperties.keyVaultUrl().ifPresent(keyClientBuilder::vaultUrl); + this.keyClient = + keyClientBuilder + .credential(AdlsTokenCredentialProviders.from(properties).credential()) + .buildClient(); + } + + @Override + public ByteBuffer wrapKey(ByteBuffer key, String wrappingKeyId) { + WrapResult wrapResult = + keyClient + .getCryptographyClient(wrappingKeyId) + .wrapKey(keyWrapAlgorithm, ByteBuffers.toByteArray(key)); + return ByteBuffer.wrap(wrapResult.getEncryptedKey()); + } + + @Override + public ByteBuffer unwrapKey(ByteBuffer wrappedKey, String wrappingKeyId) { + UnwrapResult unwrapResult = + keyClient + .getCryptographyClient(wrappingKeyId) + .unwrapKey(keyWrapAlgorithm, ByteBuffers.toByteArray(wrappedKey)); + return ByteBuffer.wrap(unwrapResult.getKey()); + } +} diff --git a/azure/src/test/java/org/apache/iceberg/azure/TestAzureProperties.java b/azure/src/test/java/org/apache/iceberg/azure/TestAzureProperties.java index 514e7faad435..c301d4de4741 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/TestAzureProperties.java +++ b/azure/src/test/java/org/apache/iceberg/azure/TestAzureProperties.java @@ -26,6 +26,8 @@ import static org.apache.iceberg.azure.AzureProperties.ADLS_SHARED_KEY_ACCOUNT_KEY; import static org.apache.iceberg.azure.AzureProperties.ADLS_SHARED_KEY_ACCOUNT_NAME; import static org.apache.iceberg.azure.AzureProperties.ADLS_WRITE_BLOCK_SIZE; +import static org.apache.iceberg.azure.AzureProperties.AZURE_KEYVAULT_KEY_WRAP_ALGORITHM; +import static org.apache.iceberg.azure.AzureProperties.AZURE_KEYVAULT_URL; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import static org.mockito.ArgumentMatchers.any; @@ -39,6 +41,7 @@ import com.azure.core.credential.TokenCredential; import com.azure.core.credential.TokenRequestContext; import com.azure.identity.DefaultAzureCredential; +import com.azure.security.keyvault.keys.cryptography.models.KeyWrapAlgorithm; import com.azure.storage.common.StorageSharedKeyCredential; import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; import java.io.IOException; @@ -73,11 +76,15 @@ public void testSerializable(TestHelpers.RoundTripSerializer ro .put(ADLS_SHARED_KEY_ACCOUNT_KEY, "secret") .put(AzureProperties.ADLS_TOKEN_CREDENTIAL_PROVIDER, "provider") .put(AzureProperties.ADLS_TOKEN_PROVIDER_PREFIX + "client-id", "clientId") + .put(AZURE_KEYVAULT_URL, "https://test-key-vault.vault.azure.net") + .put(AZURE_KEYVAULT_KEY_WRAP_ALGORITHM, KeyWrapAlgorithm.RSA1_5.getValue()) .build()); AzureProperties serdedProps = roundTripSerializer.apply(props); assertThat(serdedProps.adlsReadBlockSize()).isEqualTo(props.adlsReadBlockSize()); assertThat(serdedProps.adlsWriteBlockSize()).isEqualTo(props.adlsWriteBlockSize()); + assertThat(serdedProps.keyVaultUrl()).isEqualTo(props.keyVaultUrl()); + assertThat(serdedProps.keyWrapAlgorithm()).isEqualTo(props.keyWrapAlgorithm()); } @Test diff --git a/build.gradle b/build.gradle index 1a18887b21c7..bd8062193421 100644 --- a/build.gradle +++ b/build.gradle @@ -577,6 +577,7 @@ project(':iceberg-azure') { compileOnly platform(libs.azuresdk.bom) compileOnly "com.azure:azure-storage-file-datalake" + compileOnly "com.azure:azure-security-keyvault-keys" compileOnly "com.azure:azure-identity" testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') From baff19fa94b562402d9119683c42585c9ac66619 Mon Sep 17 00:00:00 2001 From: Danica Fine Date: Fri, 12 Dec 2025 14:02:58 -0800 Subject: [PATCH 090/201] Docs: Update community meetup guidelines (#14770) --- site/docs/community.md | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/site/docs/community.md b/site/docs/community.md index d8611fc6b231..3aa5a1c69e93 100644 --- a/site/docs/community.md +++ b/site/docs/community.md @@ -88,6 +88,15 @@ Hosts are required to ensure that: 4. Planned meetups ought to be brought to the attention of the [dev list](#mailing-lists) 5. All [Community Guidelines](#community-guidelines) must be respected +Meetups *must* be small events under the [ASF branding guidelines](https://www.apache.org/foundation/marks/events.html#dates) and are typically small, informal gatherings. If you're unsure whether an event is a meetup, meetups usually: + +* Rely on a curated selection of talks, where organizers work with the community to include a diverse representation of speakers, topics, and companies. Due to the small number of submissions, meetups might use, but don't require, a CfP. +* Are single-tracked and accommodate only a handful of sessions. +* Have at most 2-3 hours of content, plus networking time. +* Are sponsored by 1-3 companies providing food, drinks, or meeting space; not by selling booth space or marketing opportunities. They don't require substantial financial support. + +If you don't know whether an event qualifies as a meetup, please ask the PMC through the private mailing list! (Be sure to do this *before* using the Apache Iceberg brand or trademark.) + ## Community Guidelines ### Apache Iceberg Community Guidelines From 9a048825fc8d83982ca6d4ea27c07c1c573cd8a6 Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Fri, 12 Dec 2025 15:07:03 -0800 Subject: [PATCH 091/201] Spark, Flink: replace deprecated cleanExpiredFiles in expireSnapshots (#14832) --- .../src/test/java/org/apache/iceberg/catalog/CatalogTests.java | 3 ++- .../flink/maintenance/operator/ExpireSnapshotsProcessor.java | 2 +- .../flink/maintenance/operator/ExpireSnapshotsProcessor.java | 2 +- .../flink/maintenance/operator/ExpireSnapshotsProcessor.java | 2 +- .../iceberg/spark/actions/ExpireSnapshotsSparkAction.java | 3 ++- .../iceberg/spark/actions/ExpireSnapshotsSparkAction.java | 3 ++- 6 files changed, 9 insertions(+), 6 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java index c2fd24856fb2..5d20bc15a9c1 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java @@ -38,6 +38,7 @@ import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; +import org.apache.iceberg.ExpireSnapshots; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.FilesTable; @@ -1650,7 +1651,7 @@ public void testRemoveUnusedSpec(boolean withBranch) { table.newAppend().appendFile(anotherFile).commit(); table .expireSnapshots() - .cleanExpiredFiles(false) + .cleanupLevel(ExpireSnapshots.CleanupLevel.NONE) .expireOlderThan(table.currentSnapshot().timestampMillis()) .cleanExpiredMetadata(true) .commit(); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java index 154512e27ba7..36c8215755a2 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java @@ -107,7 +107,7 @@ public void processElement(Trigger trigger, Context ctx, Collector o ctx.output(DELETE_STREAM, file); deleteFileCounter.incrementAndGet(); }) - .cleanExpiredFiles(true) + .cleanupLevel(ExpireSnapshots.CleanupLevel.ALL) .commit(); LOG.info( diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java index 2db9585ebd8a..8cbd8e269a11 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java @@ -107,7 +107,7 @@ public void processElement(Trigger trigger, Context ctx, Collector o ctx.output(DELETE_STREAM, file); deleteFileCounter.incrementAndGet(); }) - .cleanExpiredFiles(true) + .cleanupLevel(ExpireSnapshots.CleanupLevel.ALL) .commit(); LOG.info( diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java index 2db9585ebd8a..8cbd8e269a11 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/ExpireSnapshotsProcessor.java @@ -107,7 +107,7 @@ public void processElement(Trigger trigger, Context ctx, Collector o ctx.output(DELETE_STREAM, file); deleteFileCounter.incrementAndGet(); }) - .cleanExpiredFiles(true) + .cleanupLevel(ExpireSnapshots.CleanupLevel.ALL) .commit(); LOG.info( diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java index 5f4d4ec15184..e49e7326736f 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java @@ -27,6 +27,7 @@ import java.util.concurrent.ExecutorService; import java.util.function.Consumer; import java.util.stream.Collectors; +import org.apache.iceberg.ExpireSnapshots.CleanupLevel; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; @@ -169,7 +170,7 @@ public Dataset expireFiles() { expireSnapshots.cleanExpiredMetadata(cleanExpiredMetadata); } - expireSnapshots.cleanExpiredFiles(false).commit(); + expireSnapshots.cleanupLevel(CleanupLevel.NONE).commit(); // fetch valid files after expiration TableMetadata updatedMetadata = ops.refresh(); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java index 5f4d4ec15184..e49e7326736f 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java @@ -27,6 +27,7 @@ import java.util.concurrent.ExecutorService; import java.util.function.Consumer; import java.util.stream.Collectors; +import org.apache.iceberg.ExpireSnapshots.CleanupLevel; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; @@ -169,7 +170,7 @@ public Dataset expireFiles() { expireSnapshots.cleanExpiredMetadata(cleanExpiredMetadata); } - expireSnapshots.cleanExpiredFiles(false).commit(); + expireSnapshots.cleanupLevel(CleanupLevel.NONE).commit(); // fetch valid files after expiration TableMetadata updatedMetadata = ops.refresh(); From bc23a774a3cb3b06a63289e445dbf91beb8b5254 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Sat, 13 Dec 2025 09:09:08 +0100 Subject: [PATCH 092/201] Core: Adjust namespace separator in TestRESTCatalog (#14808) Co-authored-by: Eduard Tudenhoefner --- .../iceberg/rest/RESTCatalogAdapter.java | 3 +- .../apache/iceberg/rest/TestRESTCatalog.java | 136 ++++++++++++++++-- .../iceberg/rest/TestResourcePaths.java | 41 ++++++ 3 files changed, 164 insertions(+), 16 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java index 8d59ee0393fa..524b3e760ca6 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -56,6 +56,7 @@ import org.apache.iceberg.exceptions.RESTException; import org.apache.iceberg.exceptions.UnprocessableEntityException; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.rest.HTTPRequest.HTTPMethod; @@ -85,7 +86,7 @@ public class RESTCatalogAdapter extends BaseHTTPClient { @SuppressWarnings("AvoidEscapedUnicodeCharacters") private static final String NAMESPACE_SEPARATOR_UNICODE = "\u002e"; - private static final String NAMESPACE_SEPARATOR_URLENCODED_UTF_8 = "%2E"; + @VisibleForTesting static final String NAMESPACE_SEPARATOR_URLENCODED_UTF_8 = "%2E"; private static final Map, Integer> EXCEPTION_ERROR_CODES = ImmutableMap., Integer>builder() diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index 753d8cb247c2..df4ba3214aea 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -118,7 +118,10 @@ public class TestRESTCatalog extends CatalogTests { private static final ObjectMapper MAPPER = RESTObjectMapper.mapper(); private static final ResourcePaths RESOURCE_PATHS = - ResourcePaths.forCatalogProperties(Maps.newHashMap()); + ResourcePaths.forCatalogProperties( + ImmutableMap.of( + RESTCatalogProperties.NAMESPACE_SEPARATOR, + RESTCatalogAdapter.NAMESPACE_SEPARATOR_URLENCODED_UTF_8)); @TempDir public Path temp; @@ -944,8 +947,6 @@ public void testTableSnapshotLoading() { .build()) .commit(); - ResourcePaths paths = ResourcePaths.forCatalogProperties(Maps.newHashMap()); - Table refsTable = catalog.loadTable(TABLE); // don't call snapshots() directly as that would cause to load all snapshots. Instead, @@ -960,7 +961,8 @@ public void testTableSnapshotLoading() { // verify that the table was loaded with the refs argument verify(adapter, times(1)) .execute( - reqMatcher(HTTPMethod.GET, paths.table(TABLE), Map.of(), Map.of("snapshots", "refs")), + reqMatcher( + HTTPMethod.GET, RESOURCE_PATHS.table(TABLE), Map.of(), Map.of("snapshots", "refs")), eq(LoadTableResponse.class), any(), any()); @@ -969,7 +971,8 @@ public void testTableSnapshotLoading() { assertThat(refsTable.snapshots()).containsExactlyInAnyOrderElementsOf(table.snapshots()); verify(adapter, times(1)) .execute( - reqMatcher(HTTPMethod.GET, paths.table(TABLE), Map.of(), Map.of("snapshots", "all")), + reqMatcher( + HTTPMethod.GET, RESOURCE_PATHS.table(TABLE), Map.of(), Map.of("snapshots", "all")), eq(LoadTableResponse.class), any(), any()); @@ -1038,8 +1041,6 @@ public void testTableSnapshotLoadingWithDivergedBranches(String formatVersion) { .toBranch(branch) .commit(); - ResourcePaths paths = ResourcePaths.forCatalogProperties(Maps.newHashMap()); - Table refsTable = catalog.loadTable(TABLE); // don't call snapshots() directly as that would cause to load all snapshots. Instead, @@ -1054,7 +1055,8 @@ public void testTableSnapshotLoadingWithDivergedBranches(String formatVersion) { // verify that the table was loaded with the refs argument verify(adapter, times(1)) .execute( - reqMatcher(HTTPMethod.GET, paths.table(TABLE), Map.of(), Map.of("snapshots", "refs")), + reqMatcher( + HTTPMethod.GET, RESOURCE_PATHS.table(TABLE), Map.of(), Map.of("snapshots", "refs")), eq(LoadTableResponse.class), any(), any()); @@ -1064,7 +1066,8 @@ public void testTableSnapshotLoadingWithDivergedBranches(String formatVersion) { .containsExactlyInAnyOrderElementsOf(table.snapshots()); verify(adapter, times(1)) .execute( - reqMatcher(HTTPMethod.GET, paths.table(TABLE), Map.of(), Map.of("snapshots", "all")), + reqMatcher( + HTTPMethod.GET, RESOURCE_PATHS.table(TABLE), Map.of(), Map.of("snapshots", "all")), eq(LoadTableResponse.class), any(), any()); @@ -2956,14 +2959,12 @@ public void testNotModified() { any(), any()); - // RESTCatalogAdapter uses %2E as a namespace separator, and we're verifying here which - // server-side path was called - ResourcePaths paths = - ResourcePaths.forCatalogProperties( - ImmutableMap.of(RESTCatalogProperties.NAMESPACE_SEPARATOR, "%2E")); verify(adapterForRESTServer) .execute( - reqMatcher(HTTPMethod.GET, paths.table(metadataTableIdentifier)), any(), any(), any()); + reqMatcher(HTTPMethod.GET, RESOURCE_PATHS.table(metadataTableIdentifier)), + any(), + any(), + any()); } @Test @@ -3311,6 +3312,111 @@ public void testClientDoesNotSendIdempotencyWhenServerNotAdvertising() { local.dropTable(ident); } + @Test + public void nestedNamespaceWithLegacySeparator() { + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); + + // Simulate that the server doesn't send the namespace separator in the overrides + Mockito.doAnswer( + invocation -> { + ConfigResponse configResponse = (ConfigResponse) invocation.callRealMethod(); + + Map overridesWithoutNamespaceSeparator = configResponse.overrides(); + overridesWithoutNamespaceSeparator.remove(RESTCatalogProperties.NAMESPACE_SEPARATOR); + + return ConfigResponse.builder() + .withDefaults(configResponse.defaults()) + .withOverrides(overridesWithoutNamespaceSeparator) + .withEndpoints(configResponse.endpoints()) + .withIdempotencyKeyLifetime(configResponse.idempotencyKeyLifetime()) + .build(); + }) + .when(adapter) + .execute( + reqMatcher(HTTPMethod.GET, ResourcePaths.config()), + eq(ConfigResponse.class), + any(), + any()); + + RESTCatalog catalog = catalog(adapter); + + ResourcePaths pathsWithLegacySeparator = ResourcePaths.forCatalogProperties(ImmutableMap.of()); + + runConfigurableNamespaceSeparatorTest( + catalog, adapter, pathsWithLegacySeparator, RESTUtil.NAMESPACE_SEPARATOR_URLENCODED_UTF_8); + } + + @Test + public void nestedNamespaceWithOverriddenSeparator() { + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); + + // When initializing the catalog, the adapter always sends an override for the namespace + // separator. + Mockito.doAnswer( + invocation -> { + ConfigResponse configResponse = (ConfigResponse) invocation.callRealMethod(); + + assertThat(configResponse.overrides()) + .containsEntry( + RESTCatalogProperties.NAMESPACE_SEPARATOR, + RESTCatalogAdapter.NAMESPACE_SEPARATOR_URLENCODED_UTF_8); + + return configResponse; + }) + .when(adapter) + .execute( + reqMatcher(HTTPMethod.GET, ResourcePaths.config()), + eq(ConfigResponse.class), + any(), + any()); + + RESTCatalog catalog = catalog(adapter); + + runConfigurableNamespaceSeparatorTest( + catalog, adapter, RESOURCE_PATHS, RESTCatalogAdapter.NAMESPACE_SEPARATOR_URLENCODED_UTF_8); + } + + private void runConfigurableNamespaceSeparatorTest( + RESTCatalog catalog, + RESTCatalogAdapter adapter, + ResourcePaths expectedPaths, + String expectedSeparator) { + Namespace nestedNamespace = Namespace.of("ns1", "ns2", "ns3"); + Namespace parentNamespace = Namespace.of("ns1", "ns2"); + TableIdentifier table = TableIdentifier.of(nestedNamespace, "tbl"); + + catalog.createNamespace(nestedNamespace); + + catalog.createTable(table, SCHEMA); + + assertThat(catalog.listNamespaces(parentNamespace)).containsExactly(nestedNamespace); + + // Verify the namespace separator in the path + Mockito.verify(adapter) + .execute( + reqMatcher(HTTPMethod.POST, expectedPaths.tables(nestedNamespace)), + eq(LoadTableResponse.class), + any(), + any()); + + // Verify the namespace separator in query parameters + Mockito.verify(adapter) + .execute( + reqMatcher( + HTTPMethod.GET, + expectedPaths.namespaces(), + Map.of(), + Map.of( + "parent", + RESTUtil.namespaceToQueryParam(parentNamespace, expectedSeparator), + "pageToken", + ""), + null), + eq(ListNamespacesResponse.class), + any(), + any()); + } + private RESTCatalog createCatalogWithIdempAdapter(ConfigResponse cfg, boolean expectOnMutations) { RESTCatalogAdapter adapter = Mockito.spy( diff --git a/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java b/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java index fcf97e3abc26..1f6306eab0a2 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java @@ -110,6 +110,47 @@ public void testNamespaceWithDot(String namespaceSeparator) { .isEqualTo("v1/namespaces/" + namespace); } + @Test + public void nestedNamespaceWithLegacySeparator() { + Namespace namespace = Namespace.of("first", "second", "third"); + String legacySeparator = RESTUtil.NAMESPACE_SEPARATOR_URLENCODED_UTF_8; + String newSeparator = RESTCatalogAdapter.NAMESPACE_SEPARATOR_URLENCODED_UTF_8; + + // legacy separator is always used by default, so no need to configure it + ResourcePaths pathsWithLegacySeparator = ResourcePaths.forCatalogProperties(ImmutableMap.of()); + + // Encode namespace using legacy separator. No need to provide the separator to encodeNamespace + String legacyEncodedNamespace = RESTUtil.encodeNamespace(namespace); + assertThat(pathsWithLegacySeparator.namespace(namespace)) + .contains(legacyEncodedNamespace) + .contains(legacySeparator); + + // Decode the namespace containing legacy separator without providing the separator + assertThat(RESTUtil.decodeNamespace(legacyEncodedNamespace)).isEqualTo(namespace); + + // Decode the namespace containing legacy separator with providing the new separator + assertThat(RESTUtil.decodeNamespace(legacyEncodedNamespace, newSeparator)).isEqualTo(namespace); + } + + @Test + public void nestedNamespaceWithNewSeparator() { + Namespace namespace = Namespace.of("first", "second", "third"); + String newSeparator = RESTCatalogAdapter.NAMESPACE_SEPARATOR_URLENCODED_UTF_8; + + ResourcePaths pathsWithNewSeparator = + ResourcePaths.forCatalogProperties( + ImmutableMap.of(RESTCatalogProperties.NAMESPACE_SEPARATOR, newSeparator)); + + // Encode namespace using new separator + String newEncodedSeparator = RESTUtil.encodeNamespace(namespace, newSeparator); + assertThat(pathsWithNewSeparator.namespace(namespace)) + .contains(newEncodedSeparator) + .contains(newSeparator); + + // Decode the namespace containing new separator with explicitly providing the separator + assertThat(RESTUtil.decodeNamespace(newEncodedSeparator, newSeparator)).isEqualTo(namespace); + } + @Test public void testNamespaceProperties() { Namespace ns = Namespace.of("ns"); From 9daef17156613ba9a0e5c23f0047cbdd072ee7c7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 13 Dec 2025 23:38:45 -0800 Subject: [PATCH 093/201] Build: Bump software.amazon.awssdk:bom from 2.40.3 to 2.40.8 (#14843) Bumps software.amazon.awssdk:bom from 2.40.3 to 2.40.8. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-version: 2.40.8 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 314a1e0471a1..225ee6943e8c 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -31,7 +31,7 @@ arrow = "15.0.2" avro = "1.12.1" assertj-core = "3.27.6" awaitility = "4.3.0" -awssdk-bom = "2.40.3" +awssdk-bom = "2.40.8" azuresdk-bom = "1.3.3" awssdk-s3accessgrants = "2.3.0" bson-ver = "4.11.5" From 83b8afa31a426af40317e4d685af99ad075ea1be Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Sun, 14 Dec 2025 17:02:58 +0900 Subject: [PATCH 094/201] Build: Bump datamodel-code-generator from 0.41.0 to 0.43.1 (#14845) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.41.0 to 0.43.1. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.41.0...0.43.1) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-version: 0.43.1 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- open-api/requirements.txt | 2 +- open-api/rest-catalog-open-api.py | 246 +++++++++++------------------- 2 files changed, 90 insertions(+), 158 deletions(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index 7f8a7438115e..cee1d4c6aa00 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,5 +16,5 @@ # under the License. openapi-spec-validator==0.7.2 -datamodel-code-generator==0.41.0 +datamodel-code-generator==0.43.1 yamllint==1.37.1 diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 9b4fd0c6a85a..fd8667e0f059 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -15,7 +15,6 @@ # specific language governing permissions and limitations # under the License. - from __future__ import annotations from datetime import date, timedelta @@ -139,15 +138,11 @@ class ExpressionType(BaseModel): class TrueExpression(BaseModel): - type: ExpressionType = Field( - default_factory=lambda: ExpressionType.parse_obj('true'), const=True - ) + type: str = Field('true', const=True) class FalseExpression(BaseModel): - type: ExpressionType = Field( - default_factory=lambda: ExpressionType.parse_obj('false'), const=True - ) + type: str = Field('false', const=True) class Reference(BaseModel): @@ -299,22 +294,22 @@ class BaseUpdate(BaseModel): action: str -class AssignUUIDUpdate(BaseModel): +class AssignUUIDUpdate(BaseUpdate): """ Assigning a UUID to a table/view should only be done when creating the table/view. It is not safe to re-assign the UUID if a table/view already has a UUID assigned """ - action: str = Field(..., const=True) + action: str = Field('assign-uuid', const=True) uuid: str -class UpgradeFormatVersionUpdate(BaseModel): - action: str = Field(..., const=True) +class UpgradeFormatVersionUpdate(BaseUpdate): + action: str = Field('upgrade-format-version', const=True) format_version: int = Field(..., alias='format-version') -class SetCurrentSchemaUpdate(BaseModel): - action: str = Field(..., const=True) +class SetCurrentSchemaUpdate(BaseUpdate): + action: str = Field('set-current-schema', const=True) schema_id: int = Field( ..., alias='schema-id', @@ -322,13 +317,13 @@ class SetCurrentSchemaUpdate(BaseModel): ) -class AddPartitionSpecUpdate(BaseModel): - action: str = Field(..., const=True) +class AddPartitionSpecUpdate(BaseUpdate): + action: str = Field('add-spec', const=True) spec: PartitionSpec -class SetDefaultSpecUpdate(BaseModel): - action: str = Field(..., const=True) +class SetDefaultSpecUpdate(BaseUpdate): + action: str = Field('set-default-spec', const=True) spec_id: int = Field( ..., alias='spec-id', @@ -336,13 +331,13 @@ class SetDefaultSpecUpdate(BaseModel): ) -class AddSortOrderUpdate(BaseModel): - action: str = Field(..., const=True) +class AddSortOrderUpdate(BaseUpdate): + action: str = Field('add-sort-order', const=True) sort_order: SortOrder = Field(..., alias='sort-order') -class SetDefaultSortOrderUpdate(BaseModel): - action: str = Field(..., const=True) +class SetDefaultSortOrderUpdate(BaseUpdate): + action: str = Field('set-default-sort-order', const=True) sort_order_id: int = Field( ..., alias='sort-order-id', @@ -350,8 +345,8 @@ class SetDefaultSortOrderUpdate(BaseModel): ) -class AddSnapshotUpdate(BaseModel): - action: str = Field(..., const=True) +class AddSnapshotUpdate(BaseUpdate): + action: str = Field('add-snapshot', const=True) snapshot: Snapshot @@ -365,38 +360,38 @@ class SetSnapshotRefUpdate(BaseModel): min_snapshots_to_keep: Optional[int] = Field(None, alias='min-snapshots-to-keep') -class RemoveSnapshotsUpdate(BaseModel): - action: str = Field(..., const=True) +class RemoveSnapshotsUpdate(BaseUpdate): + action: str = Field('remove-snapshots', const=True) snapshot_ids: List[int] = Field(..., alias='snapshot-ids') -class RemoveSnapshotRefUpdate(BaseModel): - action: str = Field(..., const=True) +class RemoveSnapshotRefUpdate(BaseUpdate): + action: str = Field('remove-snapshot-ref', const=True) ref_name: str = Field(..., alias='ref-name') -class SetLocationUpdate(BaseModel): - action: str = Field(..., const=True) +class SetLocationUpdate(BaseUpdate): + action: str = Field('set-location', const=True) location: str -class SetPropertiesUpdate(BaseModel): - action: str = Field(..., const=True) +class SetPropertiesUpdate(BaseUpdate): + action: str = Field('set-properties', const=True) updates: Dict[str, str] -class RemovePropertiesUpdate(BaseModel): - action: str = Field(..., const=True) +class RemovePropertiesUpdate(BaseUpdate): + action: str = Field('remove-properties', const=True) removals: List[str] -class AddViewVersionUpdate(BaseModel): - action: str = Field(..., const=True) +class AddViewVersionUpdate(BaseUpdate): + action: str = Field('add-view-version', const=True) view_version: ViewVersion = Field(..., alias='view-version') -class SetCurrentViewVersionUpdate(BaseModel): - action: str = Field(..., const=True) +class SetCurrentViewVersionUpdate(BaseUpdate): + action: str = Field('set-current-view-version', const=True) view_version_id: int = Field( ..., alias='view-version-id', @@ -404,33 +399,33 @@ class SetCurrentViewVersionUpdate(BaseModel): ) -class RemoveStatisticsUpdate(BaseModel): - action: str = Field(..., const=True) +class RemoveStatisticsUpdate(BaseUpdate): + action: str = Field('remove-statistics', const=True) snapshot_id: int = Field(..., alias='snapshot-id') -class RemovePartitionStatisticsUpdate(BaseModel): - action: str = Field(..., const=True) +class RemovePartitionStatisticsUpdate(BaseUpdate): + action: str = Field('remove-partition-statistics', const=True) snapshot_id: int = Field(..., alias='snapshot-id') -class RemovePartitionSpecsUpdate(BaseModel): - action: str = Field(..., const=True) +class RemovePartitionSpecsUpdate(BaseUpdate): + action: str = Field('remove-partition-specs', const=True) spec_ids: List[int] = Field(..., alias='spec-ids') -class RemoveSchemasUpdate(BaseModel): - action: str = Field(..., const=True) +class RemoveSchemasUpdate(BaseUpdate): + action: str = Field('remove-schemas', const=True) schema_ids: List[int] = Field(..., alias='schema-ids') -class AddEncryptionKeyUpdate(BaseModel): - action: str = Field(..., const=True) +class AddEncryptionKeyUpdate(BaseUpdate): + action: str = Field('add-encryption-key', const=True) encryption_key: EncryptedKey = Field(..., alias='encryption-key') -class RemoveEncryptionKeyUpdate(BaseModel): - action: str = Field(..., const=True) +class RemoveEncryptionKeyUpdate(BaseUpdate): + action: str = Field('remove-encryption-key', const=True) key_id: str = Field(..., alias='key-id') @@ -438,7 +433,7 @@ class TableRequirement(BaseModel): type: str -class AssertCreate(BaseModel): +class AssertCreate(TableRequirement): """ The table must not already exist; used for create transactions """ @@ -446,7 +441,7 @@ class AssertCreate(BaseModel): type: str = Field(..., const=True) -class AssertTableUUID(BaseModel): +class AssertTableUUID(TableRequirement): """ The table UUID must match the requirement's `uuid` """ @@ -455,7 +450,7 @@ class AssertTableUUID(BaseModel): uuid: str -class AssertRefSnapshotId(BaseModel): +class AssertRefSnapshotId(TableRequirement): """ The table branch or tag identified by the requirement's `ref` must reference the requirement's `snapshot-id`. The `snapshot-id` field is required in this object, but in the case of a `null` @@ -463,53 +458,53 @@ class AssertRefSnapshotId(BaseModel): """ - type: str = Field(..., const=True) + type: str = Field('assert-ref-snapshot-id', const=True) ref: str snapshot_id: int = Field(..., alias='snapshot-id') -class AssertLastAssignedFieldId(BaseModel): +class AssertLastAssignedFieldId(TableRequirement): """ The table's last assigned column id must match the requirement's `last-assigned-field-id` """ - type: str = Field(..., const=True) + type: str = Field('assert-last-assigned-field-id', const=True) last_assigned_field_id: int = Field(..., alias='last-assigned-field-id') -class AssertCurrentSchemaId(BaseModel): +class AssertCurrentSchemaId(TableRequirement): """ The table's current schema id must match the requirement's `current-schema-id` """ - type: str = Field(..., const=True) + type: str = Field('assert-current-schema-id', const=True) current_schema_id: int = Field(..., alias='current-schema-id') -class AssertLastAssignedPartitionId(BaseModel): +class AssertLastAssignedPartitionId(TableRequirement): """ The table's last assigned partition id must match the requirement's `last-assigned-partition-id` """ - type: str = Field(..., const=True) + type: str = Field('assert-last-assigned-partition-id', const=True) last_assigned_partition_id: int = Field(..., alias='last-assigned-partition-id') -class AssertDefaultSpecId(BaseModel): +class AssertDefaultSpecId(TableRequirement): """ The table's default spec id must match the requirement's `default-spec-id` """ - type: str = Field(..., const=True) + type: str = Field('assert-default-spec-id', const=True) default_spec_id: int = Field(..., alias='default-spec-id') -class AssertDefaultSortOrderId(BaseModel): +class AssertDefaultSortOrderId(TableRequirement): """ The table's default sort order id must match the requirement's `default-sort-order-id` """ - type: str = Field(..., const=True) + type: str = Field('assert-default-sort-order-id', const=True) default_sort_order_id: int = Field(..., alias='default-sort-order-id') @@ -536,6 +531,21 @@ class LoadCredentialsResponse(BaseModel): ) +class AsyncPlanningResult(BaseModel): + status: Literal['submitted'] = Field(..., const=True) + plan_id: str = Field( + ..., alias='plan-id', description='ID used to track a planning request' + ) + + +class EmptyPlanningResult(BaseModel): + """ + Empty server-side planning result + """ + + status: Literal['cancelled'] + + class PlanStatus(BaseModel): __root__: Literal['completed', 'submitted', 'cancelled', 'failed'] = Field( ..., description='Status of a server-side planning operation' @@ -917,7 +927,7 @@ class ContentFile(BaseModel): sort_order_id: Optional[int] = Field(None, alias='sort-order-id') -class PositionDeleteFile(BaseModel): +class PositionDeleteFile(ContentFile): content: Literal['position-deletes'] = Field(..., const=True) content_offset: Optional[int] = Field( None, @@ -929,55 +939,13 @@ class PositionDeleteFile(BaseModel): alias='content-size-in-bytes', description='Length, in bytes, of the delete content; required if content-offset is present', ) - file_path: str = Field(..., alias='file-path') - file_format: FileFormat = Field(..., alias='file-format') - spec_id: int = Field(..., alias='spec-id') - partition: List[PrimitiveTypeValue] = Field( - ..., - description='A list of partition field values ordered based on the fields of the partition spec specified by the `spec-id`', - example=[1, 'bar'], - ) - file_size_in_bytes: int = Field( - ..., alias='file-size-in-bytes', description='Total file size in bytes' - ) - record_count: int = Field( - ..., alias='record-count', description='Number of records in the file' - ) - key_metadata: Optional[BinaryTypeValue] = Field( - None, alias='key-metadata', description='Encryption key metadata blob' - ) - split_offsets: Optional[List[int]] = Field( - None, alias='split-offsets', description='List of splittable offsets' - ) - sort_order_id: Optional[int] = Field(None, alias='sort-order-id') -class EqualityDeleteFile(BaseModel): +class EqualityDeleteFile(ContentFile): content: Literal['equality-deletes'] = Field(..., const=True) equality_ids: Optional[List[int]] = Field( None, alias='equality-ids', description='List of equality field IDs' ) - file_path: str = Field(..., alias='file-path') - file_format: FileFormat = Field(..., alias='file-format') - spec_id: int = Field(..., alias='spec-id') - partition: List[PrimitiveTypeValue] = Field( - ..., - description='A list of partition field values ordered based on the fields of the partition spec specified by the `spec-id`', - example=[1, 'bar'], - ) - file_size_in_bytes: int = Field( - ..., alias='file-size-in-bytes', description='Total file size in bytes' - ) - record_count: int = Field( - ..., alias='record-count', description='Number of records in the file' - ) - key_metadata: Optional[BinaryTypeValue] = Field( - None, alias='key-metadata', description='Encryption key metadata blob' - ) - split_offsets: Optional[List[int]] = Field( - None, alias='split-offsets', description='List of splittable offsets' - ) - sort_order_id: Optional[int] = Field(None, alias='sort-order-id') class FieldName(BaseModel): @@ -1035,8 +1003,8 @@ class TransformTerm(BaseModel): term: Reference -class SetPartitionStatisticsUpdate(BaseModel): - action: str = Field(..., const=True) +class SetPartitionStatisticsUpdate(BaseUpdate): + action: str = Field('set-partition-statistics', const=True) partition_statistics: PartitionStatisticsFile = Field( ..., alias='partition-statistics' ) @@ -1054,21 +1022,6 @@ class FailedPlanningResult(IcebergErrorResponse): status: Literal['failed'] = Field(..., const=True) -class AsyncPlanningResult(BaseModel): - status: Literal['submitted'] = Field(..., const=True) - plan_id: str = Field( - ..., alias='plan-id', description='ID used to track a planning request' - ) - - -class EmptyPlanningResult(BaseModel): - """ - Empty server-side planning result - """ - - status: Literal['cancelled'] - - class ReportMetricsRequest2(CommitReport): report_type: str = Field(..., alias='report-type') @@ -1090,7 +1043,7 @@ class ValueMap(BaseModel): ) -class DataFile(BaseModel): +class DataFile(ContentFile): content: str = Field(..., const=True) first_row_id: Optional[int] = Field( None, @@ -1125,27 +1078,6 @@ class DataFile(BaseModel): alias='upper-bounds', description='Map of column id to upper bound primitive type values', ) - file_path: str = Field(..., alias='file-path') - file_format: FileFormat = Field(..., alias='file-format') - spec_id: int = Field(..., alias='spec-id') - partition: List[PrimitiveTypeValue] = Field( - ..., - description='A list of partition field values ordered based on the fields of the partition spec specified by the `spec-id`', - example=[1, 'bar'], - ) - file_size_in_bytes: int = Field( - ..., alias='file-size-in-bytes', description='Total file size in bytes' - ) - record_count: int = Field( - ..., alias='record-count', description='Number of records in the file' - ) - key_metadata: Optional[BinaryTypeValue] = Field( - None, alias='key-metadata', description='Encryption key metadata blob' - ) - split_offsets: Optional[List[int]] = Field( - None, alias='split-offsets', description='List of splittable offsets' - ) - sort_order_id: Optional[int] = Field(None, alias='sort-order-id') class DeleteFile(BaseModel): @@ -1162,8 +1094,8 @@ class Term(BaseModel): __root__: Union[Reference, TransformTerm] -class SetStatisticsUpdate(BaseModel): - action: str = Field(..., const=True) +class SetStatisticsUpdate(BaseUpdate): + action: str = Field('set-statistics', const=True) snapshot_id: Optional[int] = Field( None, alias='snapshot-id', @@ -1173,18 +1105,20 @@ class SetStatisticsUpdate(BaseModel): class UnaryExpression(BaseModel): - type: ExpressionType + type: Literal['is-null', 'not-null', 'is-nan', 'not-nan'] term: Term class LiteralExpression(BaseModel): - type: ExpressionType + type: Literal[ + 'lt', 'lt-eq', 'gt', 'gt-eq', 'eq', 'not-eq', 'starts-with', 'not-starts-with' + ] term: Term value: PrimitiveTypeValue class SetExpression(BaseModel): - type: ExpressionType + type: Literal['in', 'not-in'] term: Term values: List[PrimitiveTypeValue] @@ -1258,15 +1192,13 @@ class Expression(BaseModel): class AndOrExpression(BaseModel): - type: ExpressionType + type: Literal['and', 'or'] left: Expression right: Expression class NotExpression(BaseModel): - type: ExpressionType = Field( - default_factory=lambda: ExpressionType.parse_obj('not'), const=True - ) + type: str = Field('not', const=True) child: Expression @@ -1315,8 +1247,8 @@ class ViewMetadata(BaseModel): properties: Optional[Dict[str, str]] = None -class AddSchemaUpdate(BaseModel): - action: str = Field(..., const=True) +class AddSchemaUpdate(BaseUpdate): + action: str = Field('add-schema', const=True) schema_: Schema = Field(..., alias='schema') last_column_id: Optional[int] = Field( None, From 837df74d6b88561a5a8d761e7883e081fab44b72 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 14 Dec 2025 06:57:52 -0800 Subject: [PATCH 095/201] Build: Bump org.immutables:value from 2.11.7 to 2.12.0 (#14844) Bumps [org.immutables:value](https://github.com/immutables/immutables) from 2.11.7 to 2.12.0. - [Release notes](https://github.com/immutables/immutables/releases) - [Commits](https://github.com/immutables/immutables/compare/2.11.7...2.12.0) --- updated-dependencies: - dependency-name: org.immutables:value dependency-version: 2.12.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 225ee6943e8c..955633379d9e 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -55,7 +55,7 @@ guava = "33.5.0-jre" hadoop3 = "3.4.2" httpcomponents-httpclient5 = "5.5.1" hive2 = { strictly = "2.3.10"} # see rich version usage explanation above -immutables-value = "2.11.7" +immutables-value = "2.12.0" jackson-annotations = "2.20" jackson-bom = "2.20.1" jackson211 = { strictly = "2.11.4"} # see rich version usage explanation above From abd1d7741aca1688477270a3eafad2413d60ace9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 14 Dec 2025 06:58:24 -0800 Subject: [PATCH 096/201] Build: Bump io.netty:netty-buffer from 4.2.7.Final to 4.2.8.Final (#14841) Bumps [io.netty:netty-buffer](https://github.com/netty/netty) from 4.2.7.Final to 4.2.8.Final. - [Commits](https://github.com/netty/netty/compare/netty-4.2.7.Final...netty-4.2.8.Final) --- updated-dependencies: - dependency-name: io.netty:netty-buffer dependency-version: 4.2.8.Final dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 955633379d9e..2bae69505450 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -76,7 +76,7 @@ microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.106.0" -netty-buffer = "4.2.7.Final" +netty-buffer = "4.2.8.Final" object-client-bundle = "3.3.2" orc = "1.9.7" parquet = "1.16.0" From b6e262d144ea3a76f41d48bc5b7a9aaa3b348eb0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 14 Dec 2025 06:58:39 -0800 Subject: [PATCH 097/201] Build: Bump actions/upload-artifact from 5 to 6 (#14840) Bumps [actions/upload-artifact](https://github.com/actions/upload-artifact) from 5 to 6. - [Release notes](https://github.com/actions/upload-artifact/releases) - [Commits](https://github.com/actions/upload-artifact/compare/v5...v6) --- updated-dependencies: - dependency-name: actions/upload-artifact dependency-version: '6' dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/api-binary-compatibility.yml | 2 +- .github/workflows/delta-conversion-ci.yml | 4 ++-- .github/workflows/flink-ci.yml | 2 +- .github/workflows/hive-ci.yml | 2 +- .github/workflows/java-ci.yml | 2 +- .github/workflows/jmh-benchmarks.yml | 2 +- .github/workflows/kafka-connect-ci.yml | 2 +- .github/workflows/recurring-jmh-benchmarks.yml | 2 +- .github/workflows/spark-ci.yml | 2 +- 9 files changed, 10 insertions(+), 10 deletions(-) diff --git a/.github/workflows/api-binary-compatibility.yml b/.github/workflows/api-binary-compatibility.yml index 2b6f8ec323e4..9f7daee250b8 100644 --- a/.github/workflows/api-binary-compatibility.yml +++ b/.github/workflows/api-binary-compatibility.yml @@ -58,7 +58,7 @@ jobs: - run: | echo "Using the old version tag, as per git describe, of $(git describe)"; - run: ./gradlew revapi --rerun-tasks - - uses: actions/upload-artifact@v5 + - uses: actions/upload-artifact@v6 if: failure() with: name: test logs diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index cc6b2d89331e..b55b85ee63ce 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -89,7 +89,7 @@ jobs: restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.12 -DkafkaVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - - uses: actions/upload-artifact@v5 + - uses: actions/upload-artifact@v6 if: failure() with: name: test logs @@ -118,7 +118,7 @@ jobs: restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.13 -DkafkaVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - - uses: actions/upload-artifact@v5 + - uses: actions/upload-artifact@v6 if: failure() with: name: test logs diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 8b8c9426e1ca..bade4a1590a7 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -92,7 +92,7 @@ jobs: restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - run: ./gradlew -DsparkVersions= -DkafkaVersions= -DflinkVersions=${{ matrix.flink }} :iceberg-flink:iceberg-flink-${{ matrix.flink }}:check :iceberg-flink:iceberg-flink-runtime-${{ matrix.flink }}:check -Pquick=true -x javadoc -DtestParallelism=auto - - uses: actions/upload-artifact@v5 + - uses: actions/upload-artifact@v6 if: failure() with: name: test logs diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index 282cf9e8454d..b7ffe48d2154 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -90,7 +90,7 @@ jobs: restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - run: ./gradlew -DsparkVersions= -DflinkVersions= -DkafkaVersions= -Pquick=true :iceberg-mr:check -x javadoc - - uses: actions/upload-artifact@v5 + - uses: actions/upload-artifact@v6 if: failure() with: name: test logs diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 5505bb25db83..19be82b04f28 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -85,7 +85,7 @@ jobs: restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - run: ./gradlew check -DsparkVersions= -DflinkVersions= -DkafkaVersions= -Pquick=true -x javadoc - - uses: actions/upload-artifact@v5 + - uses: actions/upload-artifact@v6 if: failure() with: name: test logs diff --git a/.github/workflows/jmh-benchmarks.yml b/.github/workflows/jmh-benchmarks.yml index 0536fd7c88c7..f7cf88efbe87 100644 --- a/.github/workflows/jmh-benchmarks.yml +++ b/.github/workflows/jmh-benchmarks.yml @@ -95,7 +95,7 @@ jobs: - name: Run Benchmark run: ./gradlew :iceberg-spark:${{ github.event.inputs.spark_version }}:jmh -PjmhIncludeRegex=${{ matrix.benchmark }} -PjmhOutputPath=benchmark/${{ matrix.benchmark }}.txt - - uses: actions/upload-artifact@v5 + - uses: actions/upload-artifact@v6 if: ${{ always() }} with: name: benchmark-results diff --git a/.github/workflows/kafka-connect-ci.yml b/.github/workflows/kafka-connect-ci.yml index 168d64a6c1d5..a75d122ee732 100644 --- a/.github/workflows/kafka-connect-ci.yml +++ b/.github/workflows/kafka-connect-ci.yml @@ -96,7 +96,7 @@ jobs: :iceberg-kafka-connect:iceberg-kafka-connect:check \ :iceberg-kafka-connect:iceberg-kafka-connect-runtime:check \ -Pquick=true -x javadoc - - uses: actions/upload-artifact@v5 + - uses: actions/upload-artifact@v6 if: failure() with: name: test logs diff --git a/.github/workflows/recurring-jmh-benchmarks.yml b/.github/workflows/recurring-jmh-benchmarks.yml index fa5a2e79e1e6..f21da75e937a 100644 --- a/.github/workflows/recurring-jmh-benchmarks.yml +++ b/.github/workflows/recurring-jmh-benchmarks.yml @@ -65,7 +65,7 @@ jobs: - name: Run Benchmark run: ./gradlew :iceberg-spark:${{ matrix.spark_version }}:jmh -PjmhIncludeRegex=${{ matrix.benchmark }} -PjmhOutputPath=benchmark/${{ matrix.benchmark }}.txt -PjmhJsonOutputPath=benchmark/${{ matrix.benchmark }}.json - - uses: actions/upload-artifact@v5 + - uses: actions/upload-artifact@v6 if: ${{ always() }} with: name: benchmark-results diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index 3488dfdfea52..be3a23e3db12 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -108,7 +108,7 @@ jobs: :iceberg-spark:iceberg-spark-extensions-${{ matrix.spark }}_${{ matrix.scala }}:check \ :iceberg-spark:iceberg-spark-runtime-${{ matrix.spark }}_${{ matrix.scala }}:check \ -Pquick=true -x javadoc - - uses: actions/upload-artifact@v5 + - uses: actions/upload-artifact@v6 if: failure() with: name: test logs From 69b4191eb32969afdb41a4df020e863db96d9fff Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 14 Dec 2025 07:12:48 -0800 Subject: [PATCH 098/201] Build: Bump actions/cache from 4 to 5 (#14839) Bumps [actions/cache](https://github.com/actions/cache) from 4 to 5. - [Release notes](https://github.com/actions/cache/releases) - [Changelog](https://github.com/actions/cache/blob/main/RELEASES.md) - [Commits](https://github.com/actions/cache/compare/v4...v5) --- updated-dependencies: - dependency-name: actions/cache dependency-version: '5' dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/delta-conversion-ci.yml | 4 ++-- .github/workflows/flink-ci.yml | 2 +- .github/workflows/hive-ci.yml | 2 +- .github/workflows/java-ci.yml | 2 +- .github/workflows/jmh-benchmarks.yml | 2 +- .github/workflows/kafka-connect-ci.yml | 2 +- .github/workflows/recurring-jmh-benchmarks.yml | 2 +- .github/workflows/spark-ci.yml | 2 +- 8 files changed, 9 insertions(+), 9 deletions(-) diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index b55b85ee63ce..fd3ac0a14cb2 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -80,7 +80,7 @@ jobs: with: distribution: zulu java-version: ${{ matrix.jvm }} - - uses: actions/cache@v4 + - uses: actions/cache@v5 with: path: | ~/.gradle/caches @@ -109,7 +109,7 @@ jobs: with: distribution: zulu java-version: ${{ matrix.jvm }} - - uses: actions/cache@v4 + - uses: actions/cache@v5 with: path: | ~/.gradle/caches diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index bade4a1590a7..4f7a1e438b21 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -83,7 +83,7 @@ jobs: with: distribution: zulu java-version: ${{ matrix.jvm }} - - uses: actions/cache@v4 + - uses: actions/cache@v5 with: path: | ~/.gradle/caches diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index b7ffe48d2154..448d13b23381 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -81,7 +81,7 @@ jobs: with: distribution: zulu java-version: ${{ matrix.jvm }} - - uses: actions/cache@v4 + - uses: actions/cache@v5 with: path: | ~/.gradle/caches diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 19be82b04f28..dedb8fc42765 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -76,7 +76,7 @@ jobs: with: distribution: zulu java-version: ${{ matrix.jvm }} - - uses: actions/cache@v4 + - uses: actions/cache@v5 with: path: | ~/.gradle/caches diff --git a/.github/workflows/jmh-benchmarks.yml b/.github/workflows/jmh-benchmarks.yml index f7cf88efbe87..878b1d3e20a8 100644 --- a/.github/workflows/jmh-benchmarks.yml +++ b/.github/workflows/jmh-benchmarks.yml @@ -83,7 +83,7 @@ jobs: with: distribution: zulu java-version: 17 - - uses: actions/cache@v4 + - uses: actions/cache@v5 with: path: | ~/.gradle/caches diff --git a/.github/workflows/kafka-connect-ci.yml b/.github/workflows/kafka-connect-ci.yml index a75d122ee732..1effed7c2e56 100644 --- a/.github/workflows/kafka-connect-ci.yml +++ b/.github/workflows/kafka-connect-ci.yml @@ -81,7 +81,7 @@ jobs: with: distribution: zulu java-version: ${{ matrix.jvm }} - - uses: actions/cache@v4 + - uses: actions/cache@v5 with: path: | ~/.gradle/caches diff --git a/.github/workflows/recurring-jmh-benchmarks.yml b/.github/workflows/recurring-jmh-benchmarks.yml index f21da75e937a..3424d7f152dc 100644 --- a/.github/workflows/recurring-jmh-benchmarks.yml +++ b/.github/workflows/recurring-jmh-benchmarks.yml @@ -53,7 +53,7 @@ jobs: with: distribution: zulu java-version: 17 - - uses: actions/cache@v4 + - uses: actions/cache@v5 with: path: | ~/.gradle/caches diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index be3a23e3db12..be4083714d9c 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -91,7 +91,7 @@ jobs: with: distribution: zulu java-version: ${{ matrix.jvm }} - - uses: actions/cache@v4 + - uses: actions/cache@v5 with: path: | ~/.gradle/caches From 831b4ea108b9def6ae92258a7382aab096d87ed2 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Mon, 15 Dec 2025 14:02:24 +0100 Subject: [PATCH 099/201] Core: Change removal of deprecations to 1.12.0 (#14392) --- .../main/java/org/apache/iceberg/LocationProviders.java | 2 +- core/src/main/java/org/apache/iceberg/MetricsConfig.java | 2 +- core/src/main/java/org/apache/iceberg/SystemConfigs.java | 4 ++-- .../main/java/org/apache/iceberg/SystemProperties.java | 2 +- .../main/java/org/apache/iceberg/avro/AvroSchemaUtil.java | 2 +- .../java/org/apache/iceberg/data/avro/RawDecoder.java | 2 +- .../iceberg/encryption/StandardEncryptionManager.java | 6 +++--- .../src/main/java/org/apache/iceberg/io/ContentCache.java | 2 +- .../java/org/apache/iceberg/rest/RESTSessionCatalog.java | 2 +- .../main/java/org/apache/iceberg/util/SnapshotUtil.java | 2 +- .../main/java/org/apache/iceberg/util/ThreadPools.java | 8 ++++---- .../java/org/apache/iceberg/TestLocationProvider.java | 6 +++--- .../org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java | 3 ++- 13 files changed, 22 insertions(+), 21 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/LocationProviders.java b/core/src/main/java/org/apache/iceberg/LocationProviders.java index 6a75529ddb91..083e8b15f1d6 100644 --- a/core/src/main/java/org/apache/iceberg/LocationProviders.java +++ b/core/src/main/java/org/apache/iceberg/LocationProviders.java @@ -87,7 +87,7 @@ private static String getAndCheckLegacyLocation(Map properties, if (value != null && DEPRECATED_PROPERTIES.contains(key)) { throw new IllegalArgumentException( String.format( - "Property '%s' has been deprecated and will be removed in 2.0, use '%s' instead.", + "Property '%s' has been deprecated and will be removed in 2.0.0, use '%s' instead.", key, TableProperties.WRITE_DATA_LOCATION)); } diff --git a/core/src/main/java/org/apache/iceberg/MetricsConfig.java b/core/src/main/java/org/apache/iceberg/MetricsConfig.java index 9d1022f5c354..593dbc570b8a 100644 --- a/core/src/main/java/org/apache/iceberg/MetricsConfig.java +++ b/core/src/main/java/org/apache/iceberg/MetricsConfig.java @@ -84,7 +84,7 @@ public static MetricsConfig forPositionDelete() { * Creates a metrics config from table configuration. * * @param props table configuration - * @deprecated use {@link MetricsConfig#forTable(Table)} + * @deprecated use {@link MetricsConfig#forTable(Table)}. Will be removed in 2.0.0 */ @Deprecated public static MetricsConfig fromProperties(Map props) { diff --git a/core/src/main/java/org/apache/iceberg/SystemConfigs.java b/core/src/main/java/org/apache/iceberg/SystemConfigs.java index ad40c17e3076..be59424992ee 100644 --- a/core/src/main/java/org/apache/iceberg/SystemConfigs.java +++ b/core/src/main/java/org/apache/iceberg/SystemConfigs.java @@ -81,7 +81,7 @@ private SystemConfigs() {} Integer::parseUnsignedInt); /** - * @deprecated will be removed in 2.0.0; use name mapping instead + * @deprecated will be removed in 1.12.0; use name mapping instead */ @Deprecated public static final ConfigEntry NETFLIX_UNSAFE_PARQUET_ID_FALLBACK_ENABLED = @@ -91,7 +91,7 @@ private SystemConfigs() {} true, s -> { LOG.warn( - "Fallback ID assignment in Parquet is UNSAFE and will be removed in 2.0.0. Use name mapping instead."); + "Fallback ID assignment in Parquet is UNSAFE and will be removed in 1.12.0. Use name mapping instead."); return Boolean.parseBoolean(s); }); diff --git a/core/src/main/java/org/apache/iceberg/SystemProperties.java b/core/src/main/java/org/apache/iceberg/SystemProperties.java index 7a83d530e2f6..484879bb21b1 100644 --- a/core/src/main/java/org/apache/iceberg/SystemProperties.java +++ b/core/src/main/java/org/apache/iceberg/SystemProperties.java @@ -21,7 +21,7 @@ /** * Configuration properties that are controlled by Java system properties. * - * @deprecated Use {@link SystemConfigs} instead; will be removed in 2.0.0 + * @deprecated Use {@link SystemConfigs} instead; will be removed in 1.12.0 */ @Deprecated public class SystemProperties { diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java index e5aef3605ea5..c67a3089a6bf 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java @@ -129,7 +129,7 @@ public static Schema pruneColumns(Schema schema, Set selectedIds) { } /** - * @deprecated will be removed in 2.0.0; use applyNameMapping and pruneColumns(Schema, Set) + * @deprecated will be removed in 1.12.0; use applyNameMapping and pruneColumns(Schema, Set) * instead. */ @Deprecated diff --git a/core/src/main/java/org/apache/iceberg/data/avro/RawDecoder.java b/core/src/main/java/org/apache/iceberg/data/avro/RawDecoder.java index 436cba05c73a..9f3cdfdad5f4 100644 --- a/core/src/main/java/org/apache/iceberg/data/avro/RawDecoder.java +++ b/core/src/main/java/org/apache/iceberg/data/avro/RawDecoder.java @@ -64,7 +64,7 @@ public static RawDecoder create( * schema used to decode buffers. The {@code writeSchema} must be the schema that was used to * encode all buffers decoded by this class. * - * @deprecated will be removed in 2.0.0; use {@link #create(org.apache.iceberg.Schema, Function, + * @deprecated will be removed in 1.12.0; use {@link #create(org.apache.iceberg.Schema, Function, * Schema)} instead */ @Deprecated diff --git a/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java b/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java index 8edb6a136dc7..043f21728b30 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java +++ b/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java @@ -81,7 +81,7 @@ private TransientEncryptionState(KeyManagementClient kmsClient, List toIds(Iterable snapshots) { } /** - * @deprecated will be removed in 2.0.0, use {@link #newFilesBetween(Long, long, Function, + * @deprecated will be removed in 1.12.0, use {@link #newFilesBetween(Long, long, Function, * FileIO)} instead. */ @Deprecated diff --git a/core/src/main/java/org/apache/iceberg/util/ThreadPools.java b/core/src/main/java/org/apache/iceberg/util/ThreadPools.java index a96f020fe8aa..bb508295ecfa 100644 --- a/core/src/main/java/org/apache/iceberg/util/ThreadPools.java +++ b/core/src/main/java/org/apache/iceberg/util/ThreadPools.java @@ -35,8 +35,8 @@ public class ThreadPools { private ThreadPools() {} /** - * @deprecated Use {@link SystemConfigs#WORKER_THREAD_POOL_SIZE WORKER_THREAD_POOL_SIZE} instead; - * will be removed in 2.0.0 + * @deprecated Use {@link SystemConfigs#WORKER_THREAD_POOL_SIZE} instead. will be removed in + * 1.12.0 */ @Deprecated public static final String WORKER_THREAD_POOL_SIZE_PROP = @@ -113,7 +113,7 @@ private static class AuthRefreshPoolHolder { * either {@link #newExitingWorkerPool(String, int)} or {@link #newFixedThreadPool(String, int)}, * depending on the intended lifecycle of the thread pool. * - * @deprecated will be removed in 2.0.0. Use {@link #newExitingWorkerPool(String, int)} for + * @deprecated will be removed in 1.12.0. Use {@link #newExitingWorkerPool(String, int)} for * long-lived thread pools that require a shutdown hook, or {@link #newFixedThreadPool(String, * int)} for short-lived thread pools where you manage the lifecycle. */ @@ -134,7 +134,7 @@ public static ExecutorService newWorkerPool(String namePrefix) { * either {@link #newExitingWorkerPool(String, int)} or {@link #newFixedThreadPool(String, int)}, * depending on the intended lifecycle of the thread pool. * - * @deprecated will be removed in 2.0.0. Use {@link #newExitingWorkerPool(String, int)} for + * @deprecated will be removed in 1.12.0. Use {@link #newExitingWorkerPool(String, int)} for * long-lived thread pools that require a shutdown hook, or {@link #newFixedThreadPool(String, * int)} for short-lived thread pools where you manage the lifecycle. */ diff --git a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java index 93b4d582942e..146f2c8da5e7 100644 --- a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java +++ b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java @@ -219,7 +219,7 @@ public void testObjectStorageLocationProviderThrowOnDeprecatedProperties() { assertThatThrownBy(() -> table.locationProvider().newDataLocation("file")) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Property 'write.folder-storage.path' has been deprecated and will be removed in 2.0, use 'write.data.path' instead."); + "Property 'write.folder-storage.path' has been deprecated and will be removed in 2.0.0, use 'write.data.path' instead."); table .updateProperties() @@ -230,7 +230,7 @@ public void testObjectStorageLocationProviderThrowOnDeprecatedProperties() { assertThatThrownBy(() -> table.locationProvider().newDataLocation("file")) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Property 'write.object-storage.path' has been deprecated and will be removed in 2.0, use 'write.data.path' instead."); + "Property 'write.object-storage.path' has been deprecated and will be removed in 2.0.0, use 'write.data.path' instead."); } @TestTemplate @@ -245,7 +245,7 @@ public void testDefaultStorageLocationProviderThrowOnDeprecatedProperties() { assertThatThrownBy(() -> table.locationProvider().newDataLocation("file")) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Property 'write.folder-storage.path' has been deprecated and will be removed in 2.0, use 'write.data.path' instead."); + "Property 'write.folder-storage.path' has been deprecated and will be removed in 2.0.0, use 'write.data.path' instead."); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java index db264e4ffc18..06fdc562a8ca 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcTableConcurrency.java @@ -646,7 +646,8 @@ public void setAsciiStream(int parameterIndex, InputStream x, int length) throws delegate.setAsciiStream(parameterIndex, x, length); } - @Deprecated(since = "1.2") + // This is deprecated in JDK, we have to remove it once removed there. + @SuppressWarnings("deprecation") @Override public void setUnicodeStream(int parameterIndex, InputStream inputStream, int length) throws SQLException { From ba28a3365a1a0e3cd64875c5d76191080799e8bf Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Mon, 15 Dec 2025 10:30:26 -0700 Subject: [PATCH 100/201] Core: Deprecate scan response builder deleteFiles API (#14838) Users should not be able to build responses by passing an explicit list of delete files. They already have to pass through a list of file scan tasks which contain the delete files. It's also a bit brittle of an API just because someone could pass through file scan tasks and unrelated delete files. This change deprecates the exposing of the deleteFiles builder API, and will just use the file scan tasks as a source of truth. --- .../apache/iceberg/rest/CatalogHandlers.java | 15 ----------- .../iceberg/rest/TableScanResponseParser.java | 10 +++++++ .../rest/responses/BaseScanTaskResponse.java | 27 ++++++++++++++----- .../FetchPlanningResultResponseParser.java | 1 - .../FetchScanTasksResponseParser.java | 1 - .../PlanTableScanResponseParser.java | 1 - ...TestFetchPlanningResultResponseParser.java | 2 -- .../TestFetchScanTasksResponseParser.java | 2 -- .../TestPlanTableScanResponseParser.java | 5 ---- 9 files changed, 31 insertions(+), 33 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java index 82695eaf7875..e3eff1228af5 100644 --- a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java @@ -700,11 +700,6 @@ public static PlanTableScanResponse planTableScan( .withPlanStatus(PlanStatus.COMPLETED) .withPlanId(planId) .withFileScanTasks(initial.first()) - .withDeleteFiles( - initial.first().stream() - .flatMap(task -> task.deletes().stream()) - .distinct() - .collect(Collectors.toList())) .withSpecsById(table.specs()); if (!nextPlanTasks.isEmpty()) { @@ -733,11 +728,6 @@ public static FetchPlanningResultResponse fetchPlanningResult( Pair, String> initial = IN_MEMORY_PLANNING_STATE.initialScanTasksFor(planId); return FetchPlanningResultResponse.builder() .withPlanStatus(PlanStatus.COMPLETED) - .withDeleteFiles( - initial.first().stream() - .flatMap(task -> task.deletes().stream()) - .distinct() - .collect(Collectors.toList())) .withFileScanTasks(initial.first()) .withPlanTasks(IN_MEMORY_PLANNING_STATE.nextPlanTask(initial.second())) .withSpecsById(table.specs()) @@ -762,11 +752,6 @@ public static FetchScanTasksResponse fetchScanTasks( .withFileScanTasks(fileScanTasks) .withPlanTasks(IN_MEMORY_PLANNING_STATE.nextPlanTask(planTask)) .withSpecsById(table.specs()) - .withDeleteFiles( - fileScanTasks.stream() - .flatMap(task -> task.deletes().stream()) - .distinct() - .collect(Collectors.toList())) .build(); } diff --git a/core/src/main/java/org/apache/iceberg/rest/TableScanResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/TableScanResponseParser.java index 084e74651a0c..02d98904942d 100644 --- a/core/src/main/java/org/apache/iceberg/rest/TableScanResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/TableScanResponseParser.java @@ -77,7 +77,17 @@ public static List parseFileScanTasks( fileScanTaskList.add(fileScanTask); } + if (fileScanTaskList.isEmpty()) { + Preconditions.checkArgument( + deleteFiles == null || deleteFiles.isEmpty(), + "Invalid response: deleteFiles should only be returned with fileScanTasks that reference them"); + } + return fileScanTaskList; + } else { + Preconditions.checkArgument( + deleteFiles == null || deleteFiles.isEmpty(), + "Invalid response: deleteFiles should only be returned with fileScanTasks that reference them"); } return null; diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/BaseScanTaskResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/BaseScanTaskResponse.java index f98a24cf7970..e83cacc48459 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/BaseScanTaskResponse.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/BaseScanTaskResponse.java @@ -23,13 +23,15 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.rest.RESTResponse; +import org.apache.iceberg.util.DeleteFileSet; public abstract class BaseScanTaskResponse implements RESTResponse { private final List planTasks; private final List fileScanTasks; - private final List deleteFiles; + private final DeleteFileSet deleteFiles; private final Map specsById; protected BaseScanTaskResponse( @@ -39,7 +41,7 @@ protected BaseScanTaskResponse( Map specsById) { this.planTasks = planTasks; this.fileScanTasks = fileScanTasks; - this.deleteFiles = deleteFiles; + this.deleteFiles = deleteFiles == null ? null : DeleteFileSet.of(deleteFiles); this.specsById = specsById; } @@ -52,7 +54,7 @@ public List fileScanTasks() { } public List deleteFiles() { - return deleteFiles; + return deleteFiles == null ? null : Lists.newArrayList(deleteFiles.iterator()); } /** @@ -66,7 +68,7 @@ public Map specsById() { public abstract static class Builder, R extends BaseScanTaskResponse> { private List planTasks; private List fileScanTasks; - private List deleteFiles; + private DeleteFileSet deleteFiles; private Map specsById; protected Builder() {} @@ -83,11 +85,20 @@ public B withPlanTasks(List tasks) { public B withFileScanTasks(List tasks) { this.fileScanTasks = tasks; + if (fileScanTasks != null) { + this.deleteFiles = + DeleteFileSet.of( + () -> tasks.stream().flatMap(task -> task.deletes().stream()).iterator()); + } return self(); } + /** + * @deprecated since 1.11.0, will be removed in 1.12.0. + */ + @Deprecated public B withDeleteFiles(List deleteFilesList) { - this.deleteFiles = deleteFilesList; + this.deleteFiles = DeleteFileSet.of(deleteFilesList); return self(); } @@ -108,8 +119,12 @@ public List fileScanTasks() { return fileScanTasks; } + /** + * @deprecated since 1.11.0, visibility will be reduced in 1.12.0. + */ + @Deprecated public List deleteFiles() { - return deleteFiles; + return deleteFiles == null ? null : Lists.newArrayList(deleteFiles.iterator()); } /** diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/FetchPlanningResultResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/FetchPlanningResultResponseParser.java index 2ce416a0c39c..5400ef1dd13d 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/FetchPlanningResultResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/FetchPlanningResultResponseParser.java @@ -86,7 +86,6 @@ public static FetchPlanningResultResponse fromJson( .withPlanStatus(planStatus) .withPlanTasks(planTasks) .withFileScanTasks(fileScanTasks) - .withDeleteFiles(deleteFiles) .withSpecsById(specsById) .build(); } diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/FetchScanTasksResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/FetchScanTasksResponseParser.java index 14ef77b39f68..e1a4d8117caf 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/FetchScanTasksResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/FetchScanTasksResponseParser.java @@ -79,7 +79,6 @@ public static FetchScanTasksResponse fromJson( .withPlanTasks(planTasks) .withSpecsById(specsById) .withFileScanTasks(fileScanTasks) - .withDeleteFiles(deleteFiles) .build(); } } diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/PlanTableScanResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/PlanTableScanResponseParser.java index df2ad94a3238..c2f47b86d3f0 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/PlanTableScanResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/PlanTableScanResponseParser.java @@ -110,7 +110,6 @@ public static PlanTableScanResponse fromJson( .withPlanStatus(planStatus) .withPlanTasks(planTasks) .withFileScanTasks(fileScanTasks) - .withDeleteFiles(deleteFiles) .withSpecsById(specsById); if (json.hasNonNull(STORAGE_CREDENTIALS)) { diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java index 7c1021d4afab..5d9d2cad828c 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java @@ -186,7 +186,6 @@ public void roundTripSerdeWithValidStatusAndFileScanTasks() throws JsonProcessin FetchPlanningResultResponse.builder() .withPlanStatus(planStatus) .withFileScanTasks(List.of(fileScanTask)) - .withDeleteFiles(List.of(FILE_A_DELETES)) // assume this has been set .withSpecsById(PARTITION_SPECS_BY_ID) .build(); @@ -219,7 +218,6 @@ public void roundTripSerdeWithValidStatusAndFileScanTasks() throws JsonProcessin FetchPlanningResultResponse.builder() .withPlanStatus(fromResponse.planStatus()) .withPlanTasks(fromResponse.planTasks()) - .withDeleteFiles(fromResponse.deleteFiles()) .withFileScanTasks(fromResponse.fileScanTasks()) .withSpecsById(PARTITION_SPECS_BY_ID) .build(); diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchScanTasksResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchScanTasksResponseParser.java index d7824bc6a673..7b44b3533a5b 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchScanTasksResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchScanTasksResponseParser.java @@ -122,7 +122,6 @@ public void roundTripSerdeWithFileScanTasks() { FetchScanTasksResponse response = FetchScanTasksResponse.builder() .withFileScanTasks(List.of(fileScanTask)) - .withDeleteFiles(List.of(FILE_A_DELETES)) // assume you have set this already .withSpecsById(PARTITION_SPECS_BY_ID) .build(); @@ -149,7 +148,6 @@ public void roundTripSerdeWithFileScanTasks() { FetchScanTasksResponse copyResponse = FetchScanTasksResponse.builder() .withPlanTasks(fromResponse.planTasks()) - .withDeleteFiles(fromResponse.deleteFiles()) .withFileScanTasks(fromResponse.fileScanTasks()) .withSpecsById(PARTITION_SPECS_BY_ID) .build(); diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestPlanTableScanResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestPlanTableScanResponseParser.java index e2c9f21dabba..454e838bcca2 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestPlanTableScanResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestPlanTableScanResponseParser.java @@ -238,7 +238,6 @@ public void roundTripSerdeWithValidStatusAndFileScanTasks() { PlanTableScanResponse.builder() .withPlanStatus(PlanStatus.COMPLETED) .withFileScanTasks(List.of(fileScanTask)) - .withDeleteFiles(List.of(FILE_A_DELETES)) .withSpecsById(PARTITION_SPECS_BY_ID) .build(); @@ -307,7 +306,6 @@ public void multipleTasksWithDifferentDeleteFilesDontAccumulateReferences() { PlanTableScanResponse.builder() .withPlanStatus(PlanStatus.COMPLETED) .withFileScanTasks(List.of(taskA, taskB, taskC)) - .withDeleteFiles(List.of(FILE_A_DELETES, FILE_B_DELETES, FILE_C2_DELETES)) .withSpecsById(PARTITION_SPECS_BY_ID) .build(); @@ -425,7 +423,6 @@ public void roundTripSerdeWithoutDeleteFiles() { .withPlanStatus(fromResponse.planStatus()) .withPlanId(fromResponse.planId()) .withPlanTasks(fromResponse.planTasks()) - .withDeleteFiles(fromResponse.deleteFiles()) .withFileScanTasks(fromResponse.fileScanTasks()) .withSpecsById(PARTITION_SPECS_BY_ID) .build(); @@ -578,7 +575,6 @@ public void roundTripSerdeWithValidStatusAndFileScanTasksAndCredentials() { PlanTableScanResponse.builder() .withPlanStatus(PlanStatus.COMPLETED) .withFileScanTasks(List.of(fileScanTask)) - .withDeleteFiles(List.of(FILE_A_DELETES)) .withSpecsById(PARTITION_SPECS_BY_ID) .withCredentials(credentials) .build(); @@ -645,7 +641,6 @@ public void roundTripSerdeWithValidStatusAndFileScanTasksAndCredentials() { .withPlanStatus(fromResponse.planStatus()) .withPlanId(fromResponse.planId()) .withPlanTasks(fromResponse.planTasks()) - .withDeleteFiles(fromResponse.deleteFiles()) .withFileScanTasks(fromResponse.fileScanTasks()) .withSpecsById(PARTITION_SPECS_BY_ID) .withCredentials(credentials) From 90bfc3dff5f40f9ba886832ba9ccaa7b42298e9a Mon Sep 17 00:00:00 2001 From: Prashant Singh <35593236+singhpk234@users.noreply.github.com> Date: Mon, 15 Dec 2025 14:07:19 -0800 Subject: [PATCH 101/201] SPEC: Add NoSuchPlanId to cancel endpoint (#14796) Co-authored-by: Prashant Singh --- open-api/rest-catalog-open-api.yaml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index a438b8ae3b5c..4d009071f2e2 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -801,6 +801,7 @@ paths: 404: description: Not Found + - NoSuchPlanIdException, the plan-id does not exist - NoSuchTableException, the table does not exist - NoSuchNamespaceException, the namespace does not exist content: @@ -808,6 +809,8 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' examples: + PlanIdDoesNotExist: + $ref: '#/components/examples/NoSuchPlanIdError' TableDoesNotExist: $ref: '#/components/examples/NoSuchTableError' NamespaceDoesNotExist: From f2449550c72e8d744b43438ef332716121af2c6b Mon Sep 17 00:00:00 2001 From: GuoYu <511955993@qq.com> Date: Tue, 16 Dec 2025 17:01:26 +0800 Subject: [PATCH 102/201] Flink, Core: RewriteDataFiles add max file group count (#14837) --- .../actions/SizeBasedFileRewritePlanner.java | 23 ++++++++- .../org/apache/iceberg/util/BinPacking.java | 46 ++++++++++++++--- .../apache/iceberg/util/TestBinPacking.java | 50 ++++++++++++++++++- docs/docs/flink-maintenance.md | 1 + .../maintenance/api/RewriteDataFiles.java | 13 +++++ .../maintenance/operator/RewriteUtil.java | 8 ++- .../operator/TestDataFileRewritePlanner.java | 20 ++++++++ 7 files changed, 150 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewritePlanner.java b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewritePlanner.java index fad139078b5d..bcd00541308f 100644 --- a/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewritePlanner.java +++ b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewritePlanner.java @@ -107,6 +107,15 @@ public abstract class SizeBasedFileRewritePlanner< public static final long MAX_FILE_GROUP_SIZE_BYTES_DEFAULT = 100L * 1024 * 1024 * 1024; // 100 GB + /** + * This option controls the largest count of data that should be rewritten in a single file group. + * It helps with breaking down the rewriting of very large partitions which may not be rewritable + * otherwise due to the resource constraints of the cluster. + */ + public static final String MAX_FILE_GROUP_INPUT_FILES = "max-file-group-input-files"; + + public static final long MAX_FILE_GROUP_INPUT_FILES_DEFAULT = Long.MAX_VALUE; + private static final long SPLIT_OVERHEAD = 5L * 1024; private final Table table; @@ -116,6 +125,7 @@ public abstract class SizeBasedFileRewritePlanner< private int minInputFiles; private boolean rewriteAll; private long maxGroupSize; + private long maxGroupCount; private int outputSpecId; protected SizeBasedFileRewritePlanner(Table table) { @@ -151,6 +161,7 @@ public void init(Map options) { this.minInputFiles = minInputFiles(options); this.rewriteAll = rewriteAll(options); this.maxGroupSize = maxGroupSize(options); + this.maxGroupCount = maxGroupCount(options); this.outputSpecId = outputSpecId(options); if (rewriteAll) { @@ -168,7 +179,8 @@ protected boolean outsideDesiredFileSizeRange(T task) { protected Iterable> planFileGroups(Iterable tasks) { Iterable filteredTasks = rewriteAll ? tasks : filterFiles(tasks); - BinPacking.ListPacker packer = new BinPacking.ListPacker<>(maxGroupSize, 1, false); + BinPacking.ListPacker packer = + new BinPacking.ListPacker<>(maxGroupSize, 1, false, maxGroupCount); List> groups = packer.pack(filteredTasks, ContentScanTask::length); return rewriteAll ? groups : filterFileGroups(groups); } @@ -337,6 +349,15 @@ private long maxGroupSize(Map options) { return value; } + private long maxGroupCount(Map options) { + long value = + PropertyUtil.propertyAsLong( + options, MAX_FILE_GROUP_INPUT_FILES, MAX_FILE_GROUP_INPUT_FILES_DEFAULT); + Preconditions.checkArgument( + value > 0, "'%s' is set to %s but must be > 0", MAX_FILE_GROUP_INPUT_FILES, value); + return value; + } + private boolean rewriteAll(Map options) { return PropertyUtil.propertyAsBoolean(options, REWRITE_ALL, REWRITE_ALL_DEFAULT); } diff --git a/core/src/main/java/org/apache/iceberg/util/BinPacking.java b/core/src/main/java/org/apache/iceberg/util/BinPacking.java index f3160389ca6c..db31b63974fe 100644 --- a/core/src/main/java/org/apache/iceberg/util/BinPacking.java +++ b/core/src/main/java/org/apache/iceberg/util/BinPacking.java @@ -36,11 +36,18 @@ public static class ListPacker { private final long targetWeight; private final int lookback; private final boolean largestBinFirst; + private final long maxItemsPerBin; public ListPacker(long targetWeight, int lookback, boolean largestBinFirst) { + this(targetWeight, lookback, largestBinFirst, Long.MAX_VALUE); + } + + public ListPacker( + long targetWeight, int lookback, boolean largestBinFirst, long maxItemsPerBin) { this.targetWeight = targetWeight; this.lookback = lookback; this.largestBinFirst = largestBinFirst; + this.maxItemsPerBin = maxItemsPerBin; } public List> packEnd(List items, Function weightFunc) { @@ -48,13 +55,19 @@ public List> packEnd(List items, Function weightFunc) { ImmutableList.copyOf( Iterables.transform( new PackingIterable<>( - Lists.reverse(items), targetWeight, lookback, weightFunc, largestBinFirst), + Lists.reverse(items), + targetWeight, + lookback, + weightFunc, + largestBinFirst, + maxItemsPerBin), Lists::reverse))); } public List> pack(Iterable items, Function weightFunc) { return ImmutableList.copyOf( - new PackingIterable<>(items, targetWeight, lookback, weightFunc, largestBinFirst)); + new PackingIterable<>( + items, targetWeight, lookback, weightFunc, largestBinFirst, maxItemsPerBin)); } } @@ -62,12 +75,13 @@ public static class PackingIterable implements Iterable> { private final Iterable iterable; private final long targetWeight; private final int lookback; + private final long maxSize; private final Function weightFunc; private final boolean largestBinFirst; public PackingIterable( Iterable iterable, long targetWeight, int lookback, Function weightFunc) { - this(iterable, targetWeight, lookback, weightFunc, false); + this(iterable, targetWeight, lookback, weightFunc, false, Long.MAX_VALUE); } public PackingIterable( @@ -76,11 +90,22 @@ public PackingIterable( int lookback, Function weightFunc, boolean largestBinFirst) { + this(iterable, targetWeight, lookback, weightFunc, largestBinFirst, Long.MAX_VALUE); + } + + public PackingIterable( + Iterable iterable, + long targetWeight, + int lookback, + Function weightFunc, + boolean largestBinFirst, + long maxSize) { Preconditions.checkArgument( lookback > 0, "Bin look-back size must be greater than 0: %s", lookback); this.iterable = iterable; this.targetWeight = targetWeight; this.lookback = lookback; + this.maxSize = maxSize; this.weightFunc = weightFunc; this.largestBinFirst = largestBinFirst; } @@ -88,7 +113,7 @@ public PackingIterable( @Override public Iterator> iterator() { return new PackingIterator<>( - iterable.iterator(), targetWeight, lookback, weightFunc, largestBinFirst); + iterable.iterator(), targetWeight, lookback, maxSize, weightFunc, largestBinFirst); } } @@ -97,6 +122,7 @@ private static class PackingIterator implements Iterator> { private final Iterator items; private final long targetWeight; private final int lookback; + private final long maxSize; private final Function weightFunc; private final boolean largestBinFirst; @@ -104,11 +130,13 @@ private PackingIterator( Iterator items, long targetWeight, int lookback, + long maxSize, Function weightFunc, boolean largestBinFirst) { this.items = items; this.targetWeight = targetWeight; this.lookback = lookback; + this.maxSize = maxSize; this.weightFunc = weightFunc; this.largestBinFirst = largestBinFirst; } @@ -163,7 +191,7 @@ private Bin findBin(long weight) { } private Bin newBin() { - return new Bin<>(targetWeight); + return new Bin<>(targetWeight, maxSize); } private static Bin removeLargestBin(Collection> bins) { @@ -181,11 +209,14 @@ private static Bin removeLargestBin(Collection> bins) { private static class Bin { private final long targetWeight; + private final long maxSize; private final List items = Lists.newArrayList(); private long binWeight = 0L; + private int binSize = 0; - Bin(long targetWeight) { + Bin(long targetWeight, long maxSize) { this.targetWeight = targetWeight; + this.maxSize = maxSize; } List items() { @@ -193,11 +224,12 @@ List items() { } boolean canAdd(long weight) { - return binWeight + weight <= targetWeight; + return binWeight + weight <= targetWeight && binSize < maxSize; } void add(T item, long weight) { this.binWeight += weight; + this.binSize++; items.add(item); } diff --git a/core/src/test/java/org/apache/iceberg/util/TestBinPacking.java b/core/src/test/java/org/apache/iceberg/util/TestBinPacking.java index f0cc6db1ad6e..213f099c8ad2 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestBinPacking.java +++ b/core/src/test/java/org/apache/iceberg/util/TestBinPacking.java @@ -61,6 +61,37 @@ public void testBasicBinPacking() { .isEqualTo(list(list(1, 2, 3, 4, 5))); } + @Test + public void testBasicBinPackingTargetSize() { + assertThat(pack(list(1, 2, 3, 4, 5), 3, Integer.MAX_VALUE, 2)) + .as("Should pack the first 2 values") + .isEqualTo(list(list(1, 2), list(3), list(4), list(5))); + + assertThat(pack(list(1, 2, 3, 4, 5), 5, Integer.MAX_VALUE, 2)) + .as("Should pack the first 2 values") + .isEqualTo(list(list(1, 2), list(3), list(4), list(5))); + + assertThat(pack(list(1, 2, 3, 4, 5), 6, Integer.MAX_VALUE, 2)) + .as("Should pack the first 3 values") + .isEqualTo(list(list(1, 2), list(3), list(4), list(5))); + + assertThat(pack(list(1, 2, 3, 4, 5), 8, Integer.MAX_VALUE, 3)) + .as("Should pack the first 3 values") + .isEqualTo(list(list(1, 2, 3), list(4), list(5))); + + assertThat(pack(list(1, 2, 3, 4, 5), 9, Integer.MAX_VALUE, 3)) + .as("Should pack the first 3 values, last 2 values") + .isEqualTo(list(list(1, 2, 3), list(4, 5))); + + assertThat(pack(list(1, 2, 3, 4, 5), 10, Integer.MAX_VALUE, 3)) + .as("Should pack the first 3 values, last 2 values") + .isEqualTo(list(list(1, 2, 3), list(4, 5))); + + assertThat(pack(list(1, 2, 3, 4, 5), 14, Integer.MAX_VALUE, 3)) + .as("Should pack the first 3 values, last 2 values") + .isEqualTo(list(list(1, 2, 3), list(4, 5))); + } + @Test public void testReverseBinPackingSingleLookback() { assertThat(packEnd(list(1, 2, 3, 4, 5), 3, 1)) @@ -212,12 +243,27 @@ private List> pack(List items, long targetWeight) { } private List> pack(List items, long targetWeight, int lookback) { - return pack(items, targetWeight, lookback, false); + return pack(items, targetWeight, lookback, Long.MAX_VALUE); } private List> pack( List items, long targetWeight, int lookback, boolean largestBinFirst) { - ListPacker packer = new ListPacker<>(targetWeight, lookback, largestBinFirst); + return pack(items, targetWeight, lookback, largestBinFirst, Long.MAX_VALUE); + } + + private List> pack( + List items, long targetWeight, int lookback, long targetSize) { + return pack(items, targetWeight, lookback, false, targetSize); + } + + private List> pack( + List items, + long targetWeight, + int lookback, + boolean largestBinFirst, + long targetSize) { + ListPacker packer = + new ListPacker<>(targetWeight, lookback, largestBinFirst, targetSize); return packer.pack(items, Integer::longValue); } diff --git a/docs/docs/flink-maintenance.md b/docs/docs/flink-maintenance.md index a3d3ff1e4b43..37fb9a7a84c4 100644 --- a/docs/docs/flink-maintenance.md +++ b/docs/docs/flink-maintenance.md @@ -218,6 +218,7 @@ env.execute("Table Maintenance Job"); | `partialProgressMaxCommits(int)` | Maximum commits allowed for partial progress when partialProgressEnabled is true | 10 | int | | `maxRewriteBytes(long)` | Maximum bytes to rewrite per execution | Long.MAX_VALUE | long | | `filter(Expression)` | Filter expression for selecting files to rewrite | Expressions.alwaysTrue() | Expression | +| `maxFileGroupInputFiles(long)` | Maximum allowed number of input files within a file group | Long.MAX_VALUE | long | #### DeleteOrphanFiles Configuration diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java index bedf70725a63..3b64a79eee89 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java @@ -181,6 +181,19 @@ public Builder maxFileGroupSizeBytes(long maxFileGroupSizeBytes) { return this; } + /** + * Configures the max file count for rewriting. See {@link + * SizeBasedFileRewritePlanner#MAX_FILE_GROUP_INPUT_FILES} for more details. + * + * @param maxFileGroupInputFiles file count for rewrite + */ + public Builder maxFileGroupInputFiles(long maxFileGroupInputFiles) { + this.rewriteOptions.put( + SizeBasedFileRewritePlanner.MAX_FILE_GROUP_INPUT_FILES, + String.valueOf(maxFileGroupInputFiles)); + return this; + } + /** * Configures max files to rewrite. See {@link BinPackRewriteFilePlanner#MAX_FILES_TO_REWRITE} * for more details. diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java index 68aaf29ac0d1..95992ccd979a 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java @@ -22,6 +22,7 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.List; +import java.util.Map; import java.util.Set; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.ProcessFunctionTestHarnesses; @@ -38,6 +39,11 @@ private RewriteUtil() {} static List planDataFileRewrite(TableLoader tableLoader) throws Exception { + return planDataFileRewrite(tableLoader, ImmutableMap.of(MIN_INPUT_FILES, "2")); + } + + static List planDataFileRewrite( + TableLoader tableLoader, Map rewriterOptions) throws Exception { try (OneInputStreamOperatorTestHarness testHarness = ProcessFunctionTestHarnesses.forProcessFunction( @@ -48,7 +54,7 @@ static List planDataFileRewrite(TableLoader tableLoader, 11, 10_000_000L, - ImmutableMap.of(MIN_INPUT_FILES, "2"), + rewriterOptions, Expressions.alwaysTrue()))) { testHarness.open(); diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java index 9f4f96e1065b..cb1a41bb4353 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.maintenance.operator; +import static org.apache.iceberg.actions.SizeBasedFileRewritePlanner.MAX_FILE_GROUP_INPUT_FILES; import static org.apache.iceberg.actions.SizeBasedFileRewritePlanner.MIN_INPUT_FILES; import static org.apache.iceberg.flink.maintenance.operator.RewriteUtil.newDataFiles; import static org.apache.iceberg.flink.maintenance.operator.RewriteUtil.planDataFileRewrite; @@ -182,6 +183,25 @@ void testMaxRewriteBytes() throws Exception { } } + @Test + void testMaxFileGroupCount() throws Exception { + Table table = createPartitionedTable(); + insertPartitioned(table, 1, "p1"); + insertPartitioned(table, 2, "p1"); + insertPartitioned(table, 3, "p2"); + insertPartitioned(table, 4, "p2"); + insertPartitioned(table, 5, "p2"); + insertPartitioned(table, 6, "p2"); + + List planWithNoLimit = planDataFileRewrite(tableLoader()); + assertThat(planWithNoLimit).hasSize(2); + + List planWithMaxFileGroupCount = + planDataFileRewrite( + tableLoader(), ImmutableMap.of(MIN_INPUT_FILES, "2", MAX_FILE_GROUP_INPUT_FILES, "2")); + assertThat(planWithMaxFileGroupCount).hasSize(3); + } + void assertRewriteFileGroup( DataFileRewritePlanner.PlannedGroup plannedGroup, Table table, Set files) { assertThat(plannedGroup.table().currentSnapshot().snapshotId()) From d5bfcaf7ca518d70a4cb47167c948a4a367df9e3 Mon Sep 17 00:00:00 2001 From: Vamsi Krishna <49806665+pallevam@users.noreply.github.com> Date: Tue, 16 Dec 2025 22:03:38 +0530 Subject: [PATCH 103/201] Docs: Add schema selection example for time travel queries (#14825) --- docs/docs/spark-queries.md | 74 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 74 insertions(+) diff --git a/docs/docs/spark-queries.md b/docs/docs/spark-queries.md index 41189d05ffb4..ce626f42b14b 100644 --- a/docs/docs/spark-queries.md +++ b/docs/docs/spark-queries.md @@ -116,6 +116,80 @@ SELECT * FROM prod.db.table VERSION AS OF 'historical-snapshot'; SELECT * FROM prod.db.table.`tag_historical-snapshot`; ``` +For example, consider a table that evolves its schema over time, and see how each type of time travel query selects its schema: + +```sql +-- snapshot S1: initial schema (id, status) +CREATE TABLE prod.db.orders ( + id BIGINT, + status STRING +) USING iceberg; + +INSERT INTO prod.db.orders VALUES (1, 'NEW'), (2, 'PAID'); + +-- record snapshot S1's snapshot_id and committed_at timestamp +-- e.g. snapshot_id = 101, committed_at = '2025-01-01 10:00:00' + +-- snapshot S2: add a new column "total" and write new data +ALTER TABLE prod.db.orders ADD COLUMN total DOUBLE; + +INSERT INTO prod.db.orders VALUES (3, 'PAID', 100.0); + +-- now S2 is the current snapshot with schema (id, status, total) +``` + +Time travel queries that select a specific snapshot or timestamp use the +snapshot's schema: + +```sql +-- uses the snapshot schema of S1: columns (id, status) +SELECT * FROM prod.db.orders VERSION AS OF 101; + +SELECT * FROM prod.db.orders TIMESTAMP AS OF '2025-01-01 10:00:00'; +``` + +In both queries above, the result only has `id` and `status`. The `total` +column does not exist in the S1 schema and is not visible, even though the +current table schema includes `total`. + +Now create a branch and a tag that both reference S1: + +```sql +-- branch "audit_branch" points to snapshot S1 +ALTER TABLE prod.db.orders CREATE BRANCH audit_branch AS OF VERSION 101; + +-- tag "first_load" also points to snapshot S1 +ALTER TABLE prod.db.orders CREATE TAG first_load AS OF VERSION 101; +``` + +When you query a branch, Spark uses the table's current schema: + +```sql +-- uses the table schema: columns (id, status, total) +SELECT * FROM prod.db.orders VERSION AS OF 'audit_branch'; + +-- equivalent identifier form +SELECT * FROM prod.db.orders.`branch_audit_branch`; +``` + +In these queries, the result has columns `(id, status, total)`. For the rows +from S1, `total` is returned as `NULL` because that column did not exist when +those rows were written. + +When you query a tag, Spark uses the snapshot's schema referenced by the tag: + +```sql +-- uses the snapshot schema of S1: columns (id, status) +SELECT * FROM prod.db.orders VERSION AS OF 'first_load'; + +-- equivalent identifier form +SELECT * FROM prod.db.orders.`tag_first_load`; +``` + +These queries only return `id` and `status`, because tags are bound to a +specific snapshot and use that snapshot's schema, even if the table's current +schema has evolved. + ## Querying with DataFrames To load a table as a DataFrame, use `table`: From baee8877117b10b761c1c9c261512b255ac7e739 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 16 Dec 2025 10:57:13 -0800 Subject: [PATCH 104/201] fix typo in assert message (#14855) --- .../expressions/TestInclusiveManifestEvaluator.java | 6 +++--- .../expressions/TestInclusiveMetricsEvaluator.java | 6 +++--- .../TestInclusiveMetricsEvaluatorWithExtract.java | 6 +++--- .../TestInclusiveMetricsEvaluatorWithTransforms.java | 6 +++--- .../iceberg/data/TestMetricsRowGroupFilter.java | 12 ++++++------ .../parquet/TestDictionaryRowGroupFilter.java | 12 ++++++------ 6 files changed, 24 insertions(+), 24 deletions(-) diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java index 068c862e2bda..6c4944e9cd3a 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java @@ -353,7 +353,7 @@ public void testIntegerLt() { shouldRead = ManifestEvaluator.forRowFilter(lessThan("id", INT_MAX_VALUE), SPEC, true).eval(FILE); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @Test @@ -395,7 +395,7 @@ public void testIntegerGt() { shouldRead = ManifestEvaluator.forRowFilter(greaterThan("id", INT_MAX_VALUE - 4), SPEC, true).eval(FILE); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @Test @@ -418,7 +418,7 @@ public void testIntegerGtEq() { shouldRead = ManifestEvaluator.forRowFilter(greaterThanOrEqual("id", INT_MAX_VALUE - 4), SPEC, true) .eval(FILE); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @Test diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java index 2f4fbf395739..8ae34c4c16ce 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java @@ -474,7 +474,7 @@ public void testIntegerLt() { assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = new InclusiveMetricsEvaluator(SCHEMA, lessThan("id", INT_MAX_VALUE)).eval(FILE); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @Test @@ -513,7 +513,7 @@ public void testIntegerGt() { shouldRead = new InclusiveMetricsEvaluator(SCHEMA, greaterThan("id", INT_MAX_VALUE - 4)).eval(FILE); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @Test @@ -535,7 +535,7 @@ public void testIntegerGtEq() { shouldRead = new InclusiveMetricsEvaluator(SCHEMA, greaterThanOrEqual("id", INT_MAX_VALUE - 4)) .eval(FILE); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @Test diff --git a/core/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluatorWithExtract.java b/core/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluatorWithExtract.java index 1ee0a7ac635a..03c1c12f4fba 100644 --- a/core/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluatorWithExtract.java +++ b/core/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluatorWithExtract.java @@ -309,7 +309,7 @@ public void testIntegerLt() { .isTrue(); assertThat(shouldRead(lessThan(extract("variant", "$.event_id", "long"), INT_MAX_VALUE))) - .as("Should read: may possible ids") + .as("Should read: many possible ids") .isTrue(); } @@ -351,7 +351,7 @@ public void testIntegerGt() { .isTrue(); assertThat(shouldRead(greaterThan(extract("variant", "$.event_id", "long"), INT_MAX_VALUE - 4))) - .as("Should read: may possible ids") + .as("Should read: many possible ids") .isTrue(); } @@ -377,7 +377,7 @@ public void testIntegerGtEq() { assertThat( shouldRead( greaterThanOrEqual(extract("variant", "$.event_id", "long"), INT_MAX_VALUE - 4))) - .as("Should read: may possible ids") + .as("Should read: many possible ids") .isTrue(); } diff --git a/core/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluatorWithTransforms.java b/core/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluatorWithTransforms.java index 5632830064bc..41a14667eae1 100644 --- a/core/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluatorWithTransforms.java +++ b/core/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluatorWithTransforms.java @@ -368,7 +368,7 @@ public void testIntegerLt() { .isTrue(); assertThat(shouldRead(lessThan(day("ts"), INT_MAX_VALUE))) - .as("Should read: may possible ids") + .as("Should read: many possible ids") .isTrue(); } @@ -406,7 +406,7 @@ public void testIntegerGt() { .isTrue(); assertThat(shouldRead(greaterThan(day("ts"), INT_MAX_VALUE - 4))) - .as("Should read: may possible ids") + .as("Should read: many possible ids") .isTrue(); } @@ -425,7 +425,7 @@ public void testIntegerGtEq() { .isTrue(); assertThat(shouldRead(greaterThanOrEqual(day("ts"), INT_MAX_VALUE - 4))) - .as("Should read: may possible ids") + .as("Should read: many possible ids") .isTrue(); } diff --git a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java index e1f9f07e7a37..3cb46b309d82 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java +++ b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java @@ -577,7 +577,7 @@ public void testIntegerLt() { assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(lessThan("id", INT_MAX_VALUE)); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @TestTemplate @@ -609,7 +609,7 @@ public void testIntegerGt() { assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(greaterThan("id", INT_MAX_VALUE - 4)); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @TestTemplate @@ -624,7 +624,7 @@ public void testIntegerGtEq() { assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(greaterThanOrEqual("id", INT_MAX_VALUE - 4)); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @TestTemplate @@ -713,7 +713,7 @@ public void testStructFieldLt() { assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(lessThan("struct_not_null.int_field", INT_MAX_VALUE)); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @TestTemplate @@ -746,7 +746,7 @@ public void testStructFieldGt() { assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(greaterThan("struct_not_null.int_field", INT_MAX_VALUE - 4)); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @TestTemplate @@ -762,7 +762,7 @@ public void testStructFieldGtEq() { assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(greaterThanOrEqual("struct_not_null.int_field", INT_MAX_VALUE - 4)); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @TestTemplate diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java index 48cc2b0992a5..7a15f8609823 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java @@ -608,7 +608,7 @@ public void testIntegerLt() { shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, lessThan("id", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @TestTemplate @@ -656,7 +656,7 @@ public void testIntegerGt() { shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, greaterThan("id", INT_MAX_VALUE - 4)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @TestTemplate @@ -679,7 +679,7 @@ public void testIntegerGtEq() { shouldRead = new ParquetDictionaryRowGroupFilter(SCHEMA, greaterThanOrEqual("id", INT_MAX_VALUE - 4)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @TestTemplate @@ -835,7 +835,7 @@ SCHEMA, lessThan("struct_not_null.int_field", INT_MIN_VALUE + 1)) new ParquetDictionaryRowGroupFilter( SCHEMA, lessThan("struct_not_null.int_field", INT_MAX_VALUE)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @TestTemplate @@ -891,7 +891,7 @@ SCHEMA, greaterThan("struct_not_null.int_field", INT_MAX_VALUE - 1)) new ParquetDictionaryRowGroupFilter( SCHEMA, greaterThan("struct_not_null.int_field", INT_MAX_VALUE - 4)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @TestTemplate @@ -918,7 +918,7 @@ SCHEMA, greaterThanOrEqual("struct_not_null.int_field", INT_MAX_VALUE)) new ParquetDictionaryRowGroupFilter( SCHEMA, greaterThanOrEqual("struct_not_null.int_field", INT_MAX_VALUE - 4)) .shouldRead(parquetSchema, rowGroupMetadata, dictionaryStore); - assertThat(shouldRead).as("Should read: may possible ids").isTrue(); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } @TestTemplate From b23f13f71e7aa39b728ba0af5aa7e157f20e498b Mon Sep 17 00:00:00 2001 From: Prashant Singh <35593236+singhpk234@users.noreply.github.com> Date: Tue, 16 Dec 2025 11:24:11 -0800 Subject: [PATCH 105/201] Core: Address Race Condition in ScanTaskIterable (#14824) - Implements poison pill termination i.e produce a dummy task which would mark all the producers have completed production. this will not require us to always check the taskQueue.empty() before all the workers have terminated - Add proper exception propagtion to the iterator consumer - Add termination on exceptions so that no data is lost and capture the exception ot propagate via RTE --------- Co-authored-by: Prashant Singh Co-authored-by: Amogh Jahagirdar --- .../apache/iceberg/rest/ScanTaskIterable.java | 135 ++-- .../iceberg/rest/TestRESTScanPlanning.java | 1 - .../iceberg/rest/TestScanTaskIterable.java | 605 ++++++++++++++++++ 3 files changed, 693 insertions(+), 48 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/rest/TestScanTaskIterable.java diff --git a/core/src/main/java/org/apache/iceberg/rest/ScanTaskIterable.java b/core/src/main/java/org/apache/iceberg/rest/ScanTaskIterable.java index a03be784da93..4500ecf4dfdb 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ScanTaskIterable.java +++ b/core/src/main/java/org/apache/iceberg/rest/ScanTaskIterable.java @@ -29,6 +29,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.iceberg.BaseFileScanTask; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.io.CloseableIterable; @@ -44,7 +46,9 @@ class ScanTaskIterable implements CloseableIterable { private static final Logger LOG = LoggerFactory.getLogger(ScanTaskIterable.class); private static final int DEFAULT_TASK_QUEUE_CAPACITY = 1000; private static final long QUEUE_POLL_TIMEOUT_MS = 100; - private static final int WORKER_POOL_SIZE = Math.max(1, ThreadPools.WORKER_THREAD_POOL_SIZE / 4); + // Dummy task acts as a poison pill to indicate that there will be no more tasks + private static final FileScanTask DUMMY_TASK = new BaseFileScanTask(null, null, null, null, null); + private final AtomicReference failure = new AtomicReference<>(null); private final BlockingQueue taskQueue; private final ConcurrentLinkedQueue initialFileScanTasks; private final ConcurrentLinkedQueue planTasks; @@ -82,7 +86,9 @@ class ScanTaskIterable implements CloseableIterable { if (initialPlanTasks != null && !initialPlanTasks.isEmpty()) { planTasks.addAll(initialPlanTasks); } else if (initialFileScanTasks.isEmpty()) { - // nothing to do, no need to spawn workers. + // Add dummy task to indicate there is no work to be done. + // Queue is empty at this point, so add() will never fail. + taskQueue.add(DUMMY_TASK); return; } @@ -90,15 +96,7 @@ class ScanTaskIterable implements CloseableIterable { } private void submitFixedWorkers() { - if (planTasks.isEmpty() && initialFileScanTasks.isEmpty()) { - // nothing to do - return; - } - - // need to spawn at least one worker to enqueue initial file scan tasks - int numWorkers = Math.min(WORKER_POOL_SIZE, Math.max(planTasks.size(), 1)); - - for (int i = 0; i < numWorkers; i++) { + for (int i = 0; i < ThreadPools.WORKER_THREAD_POOL_SIZE; i++) { executorService.execute(new PlanTaskWorker()); } } @@ -118,17 +116,12 @@ public void run() { activeWorkers.incrementAndGet(); try { - while (!shutdown.get()) { + while (!shutdown.get() && !Thread.currentThread().isInterrupted()) { String planTask = planTasks.poll(); if (planTask == null) { // if there are no more plan tasks, see if we can just add any remaining initial // file scan tasks before exiting. - while (!initialFileScanTasks.isEmpty()) { - FileScanTask initialFileScanTask = initialFileScanTasks.poll(); - if (initialFileScanTask != null) { - taskQueue.put(initialFileScanTask); - } - } + offerInitialFileScanTasks(); return; } @@ -137,16 +130,65 @@ public void run() { } catch (InterruptedException e) { Thread.currentThread().interrupt(); + failure.compareAndSet(null, new RuntimeException("PlanWorker was interrupted", e)); + shutdown.set(true); } catch (Exception e) { - throw new RuntimeException("Worker failed processing planTask", e); + failure.compareAndSet(null, new RuntimeException("Worker failed processing planTask", e)); + shutdown.set(true); } finally { - int remaining = activeWorkers.decrementAndGet(); + handleWorkerExit(); + } + } + + /** + * Offers a task to the queue with timeout, periodically checking for shutdown. Returns true if + * the task was successfully added, false if shutdown was requested. Throws InterruptedException + * if the thread is interrupted while waiting. + */ + private boolean offerWithTimeout(FileScanTask task) throws InterruptedException { + while (!shutdown.get()) { + if (taskQueue.offer(task, QUEUE_POLL_TIMEOUT_MS, TimeUnit.MILLISECONDS)) { + return true; + } + } + return false; + } + + private void handleWorkerExit() { + boolean isLastWorker = activeWorkers.decrementAndGet() == 0; + boolean hasWorkLeft = !planTasks.isEmpty() || !initialFileScanTasks.isEmpty(); + boolean isShuttingDown = shutdown.get(); + + if (isLastWorker && (!hasWorkLeft || isShuttingDown)) { + signalCompletion(); + } else if (isLastWorker && hasWorkLeft) { + failure.compareAndSet( + null, + new IllegalStateException("Workers have exited but there is still work to be done")); + shutdown.set(true); + } + } + + private void signalCompletion() { + try { + // Use offer with timeout to avoid blocking indefinitely if queue is full and consumer + // stopped draining. If shutdown is already set, consumer will exit via its shutdown check. + offerWithTimeout(DUMMY_TASK); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + // we should just shut down and not rethrow since we are trying to signal completion + // its fine if we fail to put the dummy task in this case. + shutdown.set(true); + } + } - if (remaining == 0 - && !planTasks.isEmpty() - && !shutdown.get() - && !initialFileScanTasks.isEmpty()) { - executorService.execute(new PlanTaskWorker()); + private void offerInitialFileScanTasks() throws InterruptedException { + while (!initialFileScanTasks.isEmpty() && !Thread.currentThread().isInterrupted()) { + FileScanTask initialFileScanTask = initialFileScanTasks.poll(); + if (initialFileScanTask != null) { + if (!offerWithTimeout(initialFileScanTask)) { + return; + } } } } @@ -164,16 +206,13 @@ private void processPlanTask(String planTask) throws InterruptedException { } // Now since the network IO is done, first add any initial file scan tasks - while (!initialFileScanTasks.isEmpty()) { - FileScanTask initialFileScanTask = initialFileScanTasks.poll(); - if (initialFileScanTask != null) { - taskQueue.put(initialFileScanTask); - } - } + offerInitialFileScanTasks(); if (response.fileScanTasks() != null) { for (FileScanTask task : response.fileScanTasks()) { - taskQueue.put(task); + if (!offerWithTimeout(task)) { + return; + } } } } @@ -201,21 +240,30 @@ public boolean hasNext() { return true; } - while (true) { - if (isDone()) { - return false; - } - + boolean hasNext = false; + while (!shutdown.get()) { try { nextTask = taskQueue.poll(QUEUE_POLL_TIMEOUT_MS, TimeUnit.MILLISECONDS); - if (nextTask != null) { - return true; + if (nextTask == DUMMY_TASK) { + nextTask = null; + shutdown.set(true); // Mark as done so while loop exits on subsequent calls + break; + } else if (nextTask != null) { + hasNext = true; + break; } } catch (InterruptedException e) { Thread.currentThread().interrupt(); - return false; + shutdown.set(true); } } + + RuntimeException workerFailure = failure.get(); + if (workerFailure != null) { + throw workerFailure; + } + + return hasNext; } @Override @@ -238,12 +286,5 @@ public void close() { taskQueue.clear(); planTasks.clear(); } - - private boolean isDone() { - return taskQueue.isEmpty() - && planTasks.isEmpty() - && activeWorkers.get() == 0 - && initialFileScanTasks.isEmpty(); - } } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java index 6996570b3c56..f84197b0f16e 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTScanPlanning.java @@ -348,7 +348,6 @@ void scanPlanningWithAllTasksInSingleResponse( } } - @Disabled("Temporarily disabled: Fix tracked via issue-14823") @ParameterizedTest @EnumSource(PlanningMode.class) void scanPlanningWithBatchScan( diff --git a/core/src/test/java/org/apache/iceberg/rest/TestScanTaskIterable.java b/core/src/test/java/org/apache/iceberg/rest/TestScanTaskIterable.java new file mode 100644 index 000000000000..cf4d8030b35f --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/TestScanTaskIterable.java @@ -0,0 +1,605 @@ +/* + * 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.rest; + +import static java.util.stream.Collectors.toList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.IntStream; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MockFileScanTask; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.CloseableIterator; +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.Lists; +import org.apache.iceberg.rest.requests.FetchScanTasksRequest; +import org.apache.iceberg.rest.responses.FetchScanTasksResponse; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class TestScanTaskIterable { + + private static final TableIdentifier TABLE_IDENTIFIER = + TableIdentifier.of(Namespace.of("ns"), "table"); + private static final String FETCH_TASKS_PATH = "v1/namespaces/ns/tables/table/tasks"; + private static final Map HEADERS = + ImmutableMap.of("Authorization", "Bearer token"); + + private RESTClient mockClient; + private ResourcePaths resourcePaths; + private ExecutorService executorService; + private ParserContext parserContext; + + @BeforeEach + public void before() { + mockClient = mock(RESTClient.class); + resourcePaths = ResourcePaths.forCatalogProperties(ImmutableMap.of()); + executorService = Executors.newFixedThreadPool(4); + parserContext = ParserContext.builder().build(); + } + + @AfterEach + public void after() { + if (executorService != null) { + executorService.shutdownNow(); + } + } + + private void assertIteratorThrows(CloseableIterator iterator, String errorPattern) { + assertThatThrownBy(iterator::hasNext) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining(errorPattern); + } + + private List planTasks(int count) { + return IntStream.range(0, count).mapToObj(i -> "planTask" + i).collect(toList()); + } + + private List fileTasks(int count) { + return IntStream.range(1, count + 1).mapToObj(i -> new MockFileScanTask(i)).collect(toList()); + } + + private List collectAll(CloseableIterator iterator) + throws IOException { + try (iterator) { + return Lists.newArrayList(iterator); + } + } + + private ScanTaskIterable createIterable(List planTasks, List initialTasks) { + return new ScanTaskIterable( + planTasks, + initialTasks, + mockClient, + resourcePaths, + TABLE_IDENTIFIER, + HEADERS, + executorService, + parserContext); + } + + private void mockClientPost(FetchScanTasksResponse... responses) { + if (responses.length == 1) { + when(mockClient.post( + eq(FETCH_TASKS_PATH), + any(FetchScanTasksRequest.class), + eq(FetchScanTasksResponse.class), + eq(HEADERS), + any(), + any(), + eq(parserContext))) + .thenReturn(responses[0]); + } else { + when(mockClient.post( + eq(FETCH_TASKS_PATH), + any(FetchScanTasksRequest.class), + eq(FetchScanTasksResponse.class), + eq(HEADERS), + any(), + any(), + eq(parserContext))) + .thenReturn(responses[0], java.util.Arrays.copyOfRange(responses, 1, responses.length)); + } + } + + private void mockClientPostAnswer(org.mockito.stubbing.Answer answer) { + when(mockClient.post( + eq(FETCH_TASKS_PATH), + any(FetchScanTasksRequest.class), + eq(FetchScanTasksResponse.class), + eq(HEADERS), + any(), + any(), + eq(parserContext))) + .thenAnswer(answer); + } + + // ==================== Nested/Paginated Plan Tasks Tests ==================== + + @Test + public void iterableWithNestedPlanTasks() throws IOException { + // First plan task returns more plan tasks + FetchScanTasksResponse response1 = + FetchScanTasksResponse.builder() + .withPlanTasks(ImmutableList.of("nestedPlanTask1", "nestedPlanTask2")) + .withFileScanTasks(ImmutableList.of(new MockFileScanTask(100))) + .build(); + + FetchScanTasksResponse response2 = + FetchScanTasksResponse.builder() + .withFileScanTasks(ImmutableList.of(new MockFileScanTask(200))) + .build(); + FetchScanTasksResponse response3 = + FetchScanTasksResponse.builder() + .withFileScanTasks(ImmutableList.of(new MockFileScanTask(300))) + .build(); + + mockClientPost(response1, response2, response3); + + ScanTaskIterable iterable = + createIterable(ImmutableList.of("planTask1"), Collections.emptyList()); + + List result = collectAll(iterable.iterator()); + assertThat(result).hasSize(3); + assertThat(result).extracting(FileScanTask::length).containsExactlyInAnyOrder(100L, 200L, 300L); + + verify(mockClient, times(3)) + .post( + eq(FETCH_TASKS_PATH), + any(FetchScanTasksRequest.class), + eq(FetchScanTasksResponse.class), + eq(HEADERS), + any(), + any(), + eq(parserContext)); + } + + @Test + public void iterableWithDeeplyNestedPlanTasks() throws IOException { + FetchScanTasksResponse response1 = + FetchScanTasksResponse.builder().withPlanTasks(ImmutableList.of("level2")).build(); + FetchScanTasksResponse response2 = + FetchScanTasksResponse.builder().withPlanTasks(ImmutableList.of("level3")).build(); + FetchScanTasksResponse response3 = + FetchScanTasksResponse.builder() + .withFileScanTasks(ImmutableList.of(new MockFileScanTask(100))) + .build(); + + mockClientPost(response1, response2, response3); + + ScanTaskIterable iterable = createIterable(ImmutableList.of("level1"), Collections.emptyList()); + + List result = collectAll(iterable.iterator()); + assertThat(result).hasSize(1); + assertThat(result.get(0).length()).isEqualTo(100L); + } + + // ==================== Iterator Behavior Tests ==================== + + @Test + public void iteratorNextWithoutHasNext() throws IOException { + ScanTaskIterable iterable = createIterable(null, ImmutableList.of(new MockFileScanTask(100))); + + try (CloseableIterator iterator = iterable.iterator()) { + FileScanTask task = iterator.next(); + assertThat(task.length()).isEqualTo(100L); + assertThatThrownBy(iterator::next) + .isInstanceOf(NoSuchElementException.class) + .hasMessage("No more scan tasks available"); + } + } + + @Test + public void iteratorMultipleHasNextCallsIdempotent() throws IOException { + ScanTaskIterable iterable = createIterable(null, ImmutableList.of(new MockFileScanTask(100))); + + try (CloseableIterator iterator = iterable.iterator()) { + // Multiple hasNext() calls should be idempotent + assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator.hasNext()).isTrue(); + + FileScanTask task = iterator.next(); + assertThat(task.length()).isEqualTo(100L); + + assertThat(iterator.hasNext()).isFalse(); + assertThat(iterator.hasNext()).isFalse(); + } + } + + // ==================== Error Handling Tests ==================== + + @Test + public void workerFailurePropagatesException() throws IOException { + when(mockClient.post( + eq(FETCH_TASKS_PATH), + any(FetchScanTasksRequest.class), + eq(FetchScanTasksResponse.class), + eq(HEADERS), + any(), + any(), + eq(parserContext))) + .thenThrow(new RuntimeException("Network error")); + + ScanTaskIterable iterable = + createIterable(ImmutableList.of("planTask1"), Collections.emptyList()); + + try (CloseableIterator iterator = iterable.iterator()) { + assertIteratorThrows(iterator, "Worker failed"); + } + } + + // ==================== Chained Plan Tasks Test ==================== + + @Test + public void chainedPlanTasks() throws IOException { + AtomicInteger callCount = new AtomicInteger(0); + + mockClientPostAnswer( + invocation -> { + int count = callCount.incrementAndGet(); + if (count <= 3) { + return FetchScanTasksResponse.builder() + .withPlanTasks(ImmutableList.of("chainedTask" + count)) + .withFileScanTasks(ImmutableList.of(new MockFileScanTask(count * 100L))) + .build(); + } else { + return FetchScanTasksResponse.builder() + .withFileScanTasks(ImmutableList.of(new MockFileScanTask(count * 100L))) + .build(); + } + }); + + ScanTaskIterable iterable = + createIterable(ImmutableList.of("initialTask"), Collections.emptyList()); + + List result = collectAll(iterable.iterator()); + assertThat(result).hasSize(4); + } + + // ==================== Concurrency Tests ==================== + + @Test + public void concurrentWorkersProcessingTasks() throws IOException { + AtomicInteger callCount = new AtomicInteger(0); + + mockClientPostAnswer( + invocation -> { + int count = callCount.incrementAndGet(); + // Simulate some network latency + Thread.sleep(10); + return FetchScanTasksResponse.builder() + .withFileScanTasks(ImmutableList.of(new MockFileScanTask(count * 100))) + .build(); + }); + + // Create many plan tasks to trigger multiple workers + ScanTaskIterable iterable = createIterable(planTasks(50), Collections.emptyList()); + + List result = collectAll(iterable.iterator()); + assertThat(result).hasSize(50); + + // All plan tasks should have been processed exactly once + assertThat(callCount.get()).isEqualTo(50); + } + + @Test + public void slowProducerFastConsumer() throws IOException { + AtomicInteger callCount = new AtomicInteger(0); + + mockClientPostAnswer( + invocation -> { + // Slow producer - simulate network delay + Thread.sleep(50); + int count = callCount.incrementAndGet(); + return FetchScanTasksResponse.builder() + .withFileScanTasks(ImmutableList.of(new MockFileScanTask(count * 100))) + .build(); + }); + + ScanTaskIterable iterable = createIterable(planTasks(3), Collections.emptyList()); + + List result = collectAll(iterable.iterator()); + assertThat(result).hasSize(3); + } + + @Test + public void closeWhileWorkersAreRunning() throws IOException, InterruptedException { + CountDownLatch workerStarted = new CountDownLatch(1); + + mockClientPostAnswer( + invocation -> { + workerStarted.countDown(); + // Simulate a very slow network call + Thread.sleep(5000); + return FetchScanTasksResponse.builder() + .withFileScanTasks(ImmutableList.of(new MockFileScanTask(100))) + .build(); + }); + + ScanTaskIterable iterable = + createIterable(ImmutableList.of("planTask1"), Collections.emptyList()); + + try (CloseableIterator iterator = iterable.iterator()) { + // Wait for worker to start + workerStarted.await(1, TimeUnit.SECONDS); + // Close immediately - should not block + iterator.close(); + // hasNext should return false after close + assertThat(iterator.hasNext()).isFalse(); + } + } + + @Test + public void multipleWorkersWithMixedNestedPlanTasks() throws IOException { + AtomicInteger callCount = new AtomicInteger(0); + + mockClientPostAnswer( + invocation -> { + int count = callCount.incrementAndGet(); + // First few calls return nested plan tasks to generate more work + if (count <= 3) { + return FetchScanTasksResponse.builder() + .withPlanTasks( + ImmutableList.of("nestedTask" + count + "a", "nestedTask" + count + "b")) + .withFileScanTasks(ImmutableList.of(new MockFileScanTask(count * 100))) + .build(); + } else { + return FetchScanTasksResponse.builder() + .withFileScanTasks(ImmutableList.of(new MockFileScanTask(count * 100))) + .build(); + } + }); + + ScanTaskIterable iterable = createIterable(planTasks(3), Collections.emptyList()); + + List result = collectAll(iterable.iterator()); + // 3 initial tasks + 6 nested tasks = 9 total + assertThat(result).hasSize(9); + } + + @Test + public void initialFileScanTasksWithConcurrentPlanTasks() throws IOException { + AtomicInteger callCount = new AtomicInteger(0); + + mockClientPostAnswer( + invocation -> { + int count = callCount.incrementAndGet(); + // Simulate network delay + Thread.sleep(20); + return FetchScanTasksResponse.builder() + .withFileScanTasks(ImmutableList.of(new MockFileScanTask(count * 1000))) + .build(); + }); + + // Initial tasks should be available immediately while plan tasks are being fetched + ScanTaskIterable iterable = createIterable(planTasks(10), fileTasks(10)); + + List result = collectAll(iterable.iterator()); + // 10 initial + 10 from plan tasks = 20 total + assertThat(result).hasSize(20); + } + + @Test + public void workerExceptionDoesNotBlockOtherTasks() throws IOException { + AtomicInteger callCount = new AtomicInteger(0); + + mockClientPostAnswer( + invocation -> { + int count = callCount.incrementAndGet(); + if (count == 1) { + // First call fails + throw new RuntimeException("First call failed"); + } + return FetchScanTasksResponse.builder() + .withFileScanTasks(ImmutableList.of(new MockFileScanTask(count * 100))) + .build(); + }); + + ScanTaskIterable iterable = + createIterable(ImmutableList.of("planTask1"), Collections.emptyList()); + + try (CloseableIterator iterator = iterable.iterator()) { + assertIteratorThrows(iterator, "Worker failed"); + } + } + + @Test + public void multipleWorkerFailuresOnlySignalOnce() throws IOException { + // This test verifies that when multiple workers fail, only one DUMMY_TASK is added + // and the iterator correctly propagates the first failure without hanging + AtomicInteger callCount = new AtomicInteger(0); + + mockClientPostAnswer( + invocation -> { + int count = callCount.incrementAndGet(); + // Add small delay to allow multiple workers to start + Thread.sleep(10); + throw new RuntimeException("Worker " + count + " failed"); + }); + + // Create multiple plan tasks so multiple workers can pick them up and fail + ScanTaskIterable iterable = createIterable(planTasks(10), Collections.emptyList()); + + try (CloseableIterator iterator = iterable.iterator()) { + // Should propagate the first failure without hanging + assertIteratorThrows(iterator, "Worker failed"); + + // Subsequent calls should also throw (not hang waiting for more DUMMY_TASKs) + assertIteratorThrows(iterator, "Worker failed"); + } + } + + @Test + public void workerExceptionWithFullQueueDoesNotHangOtherWorkers() throws Exception { + // This test verifies that when one worker fails and the consumer throws (stops draining), + // other workers don't hang indefinitely trying to put tasks into the full queue. + // Key: consumer does NOT call close() - it just blows up on the exception. + CountDownLatch firstWorkerStarted = new CountDownLatch(1); + CountDownLatch failureTriggered = new CountDownLatch(1); + AtomicInteger callCount = new AtomicInteger(0); + + mockClientPostAnswer( + invocation -> { + int count = callCount.incrementAndGet(); + if (count == 1) { + firstWorkerStarted.countDown(); + // First worker returns MORE tasks than queue capacity (1000) to ensure it blocks on + // offer().Also adds more plan tasks so other workers have work. + return FetchScanTasksResponse.builder() + .withPlanTasks(planTasks(5)) + .withFileScanTasks(fileTasks(1500)) + .build(); + } else if (count == 2) { + // Second worker waits for first worker to start blocking on full queue, then fails. + // This sets shutdown=true, which should unblock worker 1. + Thread.sleep(200); + failureTriggered.countDown(); + throw new RuntimeException("Worker failed"); + } else { + // Other workers also return many tasks - they may also block on full queue + return FetchScanTasksResponse.builder().withFileScanTasks(fileTasks(1500)).build(); + } + }); + + ScanTaskIterable iterable = createIterable(planTasks(3), Collections.emptyList()); + + // Intentionally NOT using try-with-resources - we don't want close() called + CloseableIterator iterator = iterable.iterator(); + try { + // Wait for first worker to start filling the queue + firstWorkerStarted.await(5, TimeUnit.SECONDS); + + // Consume just a few tasks + int consumed = 0; + while (consumed < 5) { + if (iterator.hasNext()) { + iterator.next(); + consumed++; + } + } + + // Wait for failure to be triggered + failureTriggered.await(5, TimeUnit.SECONDS); + + // Verify at least 2 workers ran. Worker 1 produced 1500 tasks (queue capacity is 1000), + // so worker 1 should be blocked on offer() when worker 2 fails and sets shutdown=true. + assertThat(callCount.get()).isGreaterThanOrEqualTo(2); + + // Give time for failure to propagate + Thread.sleep(200); + + // This hasNext() should throw due to worker failure + iterator.hasNext(); + + // Should not reach here + assertThat(false).as("Expected RuntimeException from hasNext()").isTrue(); + } catch (RuntimeException e) { + // Expected - consumer blows up, does NOT call close() + assertThat(e.getMessage()).contains("Worker failed"); + } + // Note: iterator.close() is intentionally NOT called + // Give workers time to see the shutdown flag (set by failing worker) and exit. + Thread.sleep(500); + + // Verify executor can shut down cleanly (workers aren't stuck on offer()) + executorService.shutdown(); + boolean terminated = executorService.awaitTermination(2, TimeUnit.SECONDS); + assertThat(terminated) + .as("Executor should terminate - workers should have exited via shutdown flag") + .isTrue(); + } + + @Test + public void closeWithFullQueueDoesNotHangWorkers() throws Exception { + // This test verifies that when the queue is full and the consumer closes the iterator, + // workers don't hang indefinitely trying to put tasks into the full queue. + // The queue capacity is 1000, so we need to generate more tasks than that. + CountDownLatch workerStarted = new CountDownLatch(1); + AtomicInteger callCount = new AtomicInteger(0); + + mockClientPostAnswer( + invocation -> { + workerStarted.countDown(); + int count = callCount.incrementAndGet(); + // Each response returns many file scan tasks to fill the queue quickly, + // plus more plan tasks to keep workers busy + if (count <= 10) { + return FetchScanTasksResponse.builder() + .withPlanTasks(planTasks(2)) + .withFileScanTasks(fileTasks(500)) + .build(); + } + return FetchScanTasksResponse.builder().withFileScanTasks(fileTasks(500)).build(); + }); + + ScanTaskIterable iterable = createIterable(planTasks(2), Collections.emptyList()); + + try (CloseableIterator iterator = iterable.iterator()) { + // Wait for workers to start producing + workerStarted.await(5, TimeUnit.SECONDS); + + // Consume a few tasks to let the queue fill up + int consumed = 0; + while (iterator.hasNext() && consumed < 10) { + iterator.next(); + consumed++; + } + + // Give workers time to fill the queue (each worker produces 500 tasks, queue capacity is + // 1000) + Thread.sleep(200); + + // Verify enough workers ran to fill the queue beyond capacity. + // With 500 tasks per call and queue capacity 1000, we need 3+ calls to overflow. + assertThat(callCount.get()).isGreaterThanOrEqualTo(3); + } + // iterator.close() called here by try-with-resources + + // Give workers a bit more time to exit after consumer closed + Thread.sleep(500); + + // Verify executor can shut down cleanly (workers aren't stuck on offer()) + executorService.shutdown(); + boolean terminated = executorService.awaitTermination(2, TimeUnit.SECONDS); + assertThat(terminated) + .as("Executor should terminate - workers should have exited gracefully") + .isTrue(); + } +} From 60b42ec0550bcb31c1a05000f34b5ca24016221a Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Wed, 17 Dec 2025 13:54:46 +0900 Subject: [PATCH 106/201] API: Remove redundant } from Transforms javadoc (#14866) --- api/src/main/java/org/apache/iceberg/transforms/Transforms.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/api/src/main/java/org/apache/iceberg/transforms/Transforms.java b/api/src/main/java/org/apache/iceberg/transforms/Transforms.java index aacd4d430069..a3a6a3f6321d 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Transforms.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Transforms.java @@ -29,7 +29,7 @@ * Factory methods for transforms. * *

    Most users should create transforms using a {@link PartitionSpec#builderFor(Schema)} partition - * spec builder}. + * spec builder. * * @see PartitionSpec#builderFor(Schema) The partition spec builder. */ From 26cb7cdd2eff04b5cc7a88b899ee37e7d7f32f18 Mon Sep 17 00:00:00 2001 From: GuoYu <511955993@qq.com> Date: Wed, 17 Dec 2025 19:23:21 +0800 Subject: [PATCH 107/201] Flink: Backport RewriteDataFiles add max file group count (#14861) Backports #14837 --- .../maintenance/api/RewriteDataFiles.java | 13 ++++++++++++ .../maintenance/operator/RewriteUtil.java | 8 +++++++- .../operator/TestDataFileRewritePlanner.java | 20 +++++++++++++++++++ .../maintenance/api/RewriteDataFiles.java | 13 ++++++++++++ .../maintenance/operator/RewriteUtil.java | 8 +++++++- .../operator/TestDataFileRewritePlanner.java | 20 +++++++++++++++++++ 6 files changed, 80 insertions(+), 2 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java index bedf70725a63..3b64a79eee89 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java @@ -181,6 +181,19 @@ public Builder maxFileGroupSizeBytes(long maxFileGroupSizeBytes) { return this; } + /** + * Configures the max file count for rewriting. See {@link + * SizeBasedFileRewritePlanner#MAX_FILE_GROUP_INPUT_FILES} for more details. + * + * @param maxFileGroupInputFiles file count for rewrite + */ + public Builder maxFileGroupInputFiles(long maxFileGroupInputFiles) { + this.rewriteOptions.put( + SizeBasedFileRewritePlanner.MAX_FILE_GROUP_INPUT_FILES, + String.valueOf(maxFileGroupInputFiles)); + return this; + } + /** * Configures max files to rewrite. See {@link BinPackRewriteFilePlanner#MAX_FILES_TO_REWRITE} * for more details. diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java index 68aaf29ac0d1..95992ccd979a 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java @@ -22,6 +22,7 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.List; +import java.util.Map; import java.util.Set; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.ProcessFunctionTestHarnesses; @@ -38,6 +39,11 @@ private RewriteUtil() {} static List planDataFileRewrite(TableLoader tableLoader) throws Exception { + return planDataFileRewrite(tableLoader, ImmutableMap.of(MIN_INPUT_FILES, "2")); + } + + static List planDataFileRewrite( + TableLoader tableLoader, Map rewriterOptions) throws Exception { try (OneInputStreamOperatorTestHarness testHarness = ProcessFunctionTestHarnesses.forProcessFunction( @@ -48,7 +54,7 @@ static List planDataFileRewrite(TableLoader tableLoader, 11, 10_000_000L, - ImmutableMap.of(MIN_INPUT_FILES, "2"), + rewriterOptions, Expressions.alwaysTrue()))) { testHarness.open(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java index 9f4f96e1065b..cb1a41bb4353 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.maintenance.operator; +import static org.apache.iceberg.actions.SizeBasedFileRewritePlanner.MAX_FILE_GROUP_INPUT_FILES; import static org.apache.iceberg.actions.SizeBasedFileRewritePlanner.MIN_INPUT_FILES; import static org.apache.iceberg.flink.maintenance.operator.RewriteUtil.newDataFiles; import static org.apache.iceberg.flink.maintenance.operator.RewriteUtil.planDataFileRewrite; @@ -182,6 +183,25 @@ void testMaxRewriteBytes() throws Exception { } } + @Test + void testMaxFileGroupCount() throws Exception { + Table table = createPartitionedTable(); + insertPartitioned(table, 1, "p1"); + insertPartitioned(table, 2, "p1"); + insertPartitioned(table, 3, "p2"); + insertPartitioned(table, 4, "p2"); + insertPartitioned(table, 5, "p2"); + insertPartitioned(table, 6, "p2"); + + List planWithNoLimit = planDataFileRewrite(tableLoader()); + assertThat(planWithNoLimit).hasSize(2); + + List planWithMaxFileGroupCount = + planDataFileRewrite( + tableLoader(), ImmutableMap.of(MIN_INPUT_FILES, "2", MAX_FILE_GROUP_INPUT_FILES, "2")); + assertThat(planWithMaxFileGroupCount).hasSize(3); + } + void assertRewriteFileGroup( DataFileRewritePlanner.PlannedGroup plannedGroup, Table table, Set files) { assertThat(plannedGroup.table().currentSnapshot().snapshotId()) diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java index bedf70725a63..3b64a79eee89 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java @@ -181,6 +181,19 @@ public Builder maxFileGroupSizeBytes(long maxFileGroupSizeBytes) { return this; } + /** + * Configures the max file count for rewriting. See {@link + * SizeBasedFileRewritePlanner#MAX_FILE_GROUP_INPUT_FILES} for more details. + * + * @param maxFileGroupInputFiles file count for rewrite + */ + public Builder maxFileGroupInputFiles(long maxFileGroupInputFiles) { + this.rewriteOptions.put( + SizeBasedFileRewritePlanner.MAX_FILE_GROUP_INPUT_FILES, + String.valueOf(maxFileGroupInputFiles)); + return this; + } + /** * Configures max files to rewrite. See {@link BinPackRewriteFilePlanner#MAX_FILES_TO_REWRITE} * for more details. diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java index 68aaf29ac0d1..95992ccd979a 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java @@ -22,6 +22,7 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.List; +import java.util.Map; import java.util.Set; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.ProcessFunctionTestHarnesses; @@ -38,6 +39,11 @@ private RewriteUtil() {} static List planDataFileRewrite(TableLoader tableLoader) throws Exception { + return planDataFileRewrite(tableLoader, ImmutableMap.of(MIN_INPUT_FILES, "2")); + } + + static List planDataFileRewrite( + TableLoader tableLoader, Map rewriterOptions) throws Exception { try (OneInputStreamOperatorTestHarness testHarness = ProcessFunctionTestHarnesses.forProcessFunction( @@ -48,7 +54,7 @@ static List planDataFileRewrite(TableLoader tableLoader, 11, 10_000_000L, - ImmutableMap.of(MIN_INPUT_FILES, "2"), + rewriterOptions, Expressions.alwaysTrue()))) { testHarness.open(); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java index 9f4f96e1065b..cb1a41bb4353 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewritePlanner.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.maintenance.operator; +import static org.apache.iceberg.actions.SizeBasedFileRewritePlanner.MAX_FILE_GROUP_INPUT_FILES; import static org.apache.iceberg.actions.SizeBasedFileRewritePlanner.MIN_INPUT_FILES; import static org.apache.iceberg.flink.maintenance.operator.RewriteUtil.newDataFiles; import static org.apache.iceberg.flink.maintenance.operator.RewriteUtil.planDataFileRewrite; @@ -182,6 +183,25 @@ void testMaxRewriteBytes() throws Exception { } } + @Test + void testMaxFileGroupCount() throws Exception { + Table table = createPartitionedTable(); + insertPartitioned(table, 1, "p1"); + insertPartitioned(table, 2, "p1"); + insertPartitioned(table, 3, "p2"); + insertPartitioned(table, 4, "p2"); + insertPartitioned(table, 5, "p2"); + insertPartitioned(table, 6, "p2"); + + List planWithNoLimit = planDataFileRewrite(tableLoader()); + assertThat(planWithNoLimit).hasSize(2); + + List planWithMaxFileGroupCount = + planDataFileRewrite( + tableLoader(), ImmutableMap.of(MIN_INPUT_FILES, "2", MAX_FILE_GROUP_INPUT_FILES, "2")); + assertThat(planWithMaxFileGroupCount).hasSize(3); + } + void assertRewriteFileGroup( DataFileRewritePlanner.PlannedGroup plannedGroup, Table table, Set files) { assertThat(plannedGroup.table().currentSnapshot().snapshotId()) From 9ca8029b4f7932ca2c22b034a0abc0edf8972975 Mon Sep 17 00:00:00 2001 From: Ajay Yadav Date: Wed, 17 Dec 2025 19:51:33 +0000 Subject: [PATCH 108/201] GCS: bump up gcs-analytics-core version from 1.2.1 to 1.2.3 (#14873) --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 2bae69505450..fea5bdb93afe 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -50,7 +50,7 @@ flink120 = { strictly = "1.20.1"} flink20 = { strictly = "2.0.0"} flink21 = { strictly = "2.1.0"} google-libraries-bom = "26.72.0" -gcs-analytics-core = "1.2.1" +gcs-analytics-core = "1.2.3" guava = "33.5.0-jre" hadoop3 = "3.4.2" httpcomponents-httpclient5 = "5.5.1" From 33cab35b67e3f58d06fc17f8f1ba8c3a99622c94 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 18 Dec 2025 16:07:06 +0100 Subject: [PATCH 109/201] Spark: Enable remote scan planning with REST catalog (#14822) --- .../iceberg/RequiresRemoteScanPlanning.java | 22 +++++++ .../org/apache/iceberg/rest/RESTTable.java | 3 +- .../extensions/TestRemoteScanPlanning.java | 64 +++++++++++++++++++ .../spark/source/SparkScanBuilder.java | 5 +- .../apache/iceberg/spark/sql/TestSelect.java | 33 ++++++++++ 5 files changed, 125 insertions(+), 2 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/RequiresRemoteScanPlanning.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java diff --git a/core/src/main/java/org/apache/iceberg/RequiresRemoteScanPlanning.java b/core/src/main/java/org/apache/iceberg/RequiresRemoteScanPlanning.java new file mode 100644 index 000000000000..440ea70007ee --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/RequiresRemoteScanPlanning.java @@ -0,0 +1,22 @@ +/* + * 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; + +/** Marker interface to indicate whether a Table requires remote scan planning */ +public interface RequiresRemoteScanPlanning {} diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTTable.java b/core/src/main/java/org/apache/iceberg/rest/RESTTable.java index 052e1432703f..0abe41e25f50 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTTable.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTTable.java @@ -25,12 +25,13 @@ import org.apache.iceberg.BatchScan; import org.apache.iceberg.BatchScanAdapter; import org.apache.iceberg.ImmutableTableScanContext; +import org.apache.iceberg.RequiresRemoteScanPlanning; import org.apache.iceberg.TableOperations; import org.apache.iceberg.TableScan; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.metrics.MetricsReporter; -class RESTTable extends BaseTable { +class RESTTable extends BaseTable implements RequiresRemoteScanPlanning { private final RESTClient client; private final Supplier> headers; private final MetricsReporter reporter; diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java new file mode 100644 index 000000000000..6e389730f7a7 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.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.extensions; + +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.RESTCatalogProperties; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.sql.TestSelect; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRemoteScanPlanning extends TestSelect { + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, binaryTableName = {3}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.REST.catalogName(), + SparkCatalogConfig.REST.implementation(), + ImmutableMap.builder() + .putAll(SparkCatalogConfig.REST.properties()) + .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) + // this flag is typically only set by the server, but we set it from the client for + // testing + .put(RESTCatalogProperties.REST_SCAN_PLANNING_ENABLED, "true") + .build(), + SparkCatalogConfig.REST.catalogName() + ".default.binary_table" + } + }; + } + + @TestTemplate + @Disabled( + "binary filter that is used by Spark is not working because ExpressionParser.fromJSON doesn't have the Schema to properly parse the filter expression") + public void testBinaryInFilter() { + super.testBinaryInFilter(); + } + + @TestTemplate + @Disabled("Metadata tables are currently not supported") + public void testMetadataTables() { + super.testMetadataTables(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 11b0ba58af51..dd914f1617bd 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -34,6 +34,7 @@ import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.MetricsModes; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RequiresRemoteScanPlanning; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SparkDistributedDataScan; @@ -760,7 +761,9 @@ public StructType readSchema() { } private BatchScan newBatchScan() { - if (table instanceof BaseTable && readConf.distributedPlanningEnabled()) { + if (table instanceof RequiresRemoteScanPlanning) { + return table.newBatchScan(); + } else if (table instanceof BaseTable && readConf.distributedPlanningEnabled()) { return new SparkDistributedDataScan(spark, table, readConf); } else { return table.newBatchScan(); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 2d4d63c1a127..cf4ccd62dbc3 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.TableProperties.SPLIT_SIZE; 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.sql.Timestamp; import java.text.SimpleDateFormat; @@ -36,6 +37,7 @@ 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; import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.Spark3Util; @@ -679,4 +681,35 @@ public void simpleTypesInFilter() { sql("DROP TABLE IF EXISTS %s", tableName); } + + @TestTemplate + public void variantTypeInFilter() { + assumeThat(validationCatalog) + .as("Variant is not supported in Hive catalog") + .isNotInstanceOf(HiveCatalog.class); + + String tableName = tableName("variant_table"); + sql( + "CREATE TABLE %s (id BIGINT, v1 VARIANT, v2 VARIANT) USING iceberg TBLPROPERTIES ('format-version'='3')", + tableName); + + String v1r1 = "{\"a\":5}"; + String v1r2 = "{\"a\":10}"; + String v2r1 = "{\"x\":15}"; + String v2r2 = "{\"x\":20}"; + + sql("INSERT INTO %s SELECT 1, parse_json('%s'), parse_json('%s')", tableName, v1r1, v2r1); + sql("INSERT INTO %s SELECT 2, parse_json('%s'), parse_json('%s')", tableName, v1r2, v2r2); + + assertThat( + sql( + "SELECT id, try_variant_get(v1, '$.a', 'int') FROM %s WHERE try_variant_get(v1, '$.a', 'int') > 5", + tableName)) + .containsExactly(row(2L, 10)); + assertThat( + sql( + "SELECT id, try_variant_get(v2, '$.x', 'int') FROM %s WHERE try_variant_get(v2, '$.x', 'int') < 100", + tableName)) + .containsExactlyInAnyOrder(row(1L, 15), row(2L, 20)); + } } From 8ea92ff5e1e164324950512bd8fde522976c58b4 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 18 Dec 2025 18:32:27 +0100 Subject: [PATCH 110/201] Core: Simplify handling of the current planId in client side of remote planning (#14883) --- .../apache/iceberg/rest/RESTTableScan.java | 21 +++++++------------ 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java b/core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java index 741f6e549779..dd886f9d2717 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java @@ -60,7 +60,7 @@ class RESTTableScan extends DataTableScan { private final TableIdentifier tableIdentifier; private final Set supportedEndpoints; private final ParserContext parserContext; - private String currentPlanId = null; + private String planId = null; RESTTableScan( Table table, @@ -149,31 +149,27 @@ private CloseableIterable planTableScan(PlanTableScanRequest planT stringStringMap -> {}, parserContext); + this.planId = response.planId(); PlanStatus planStatus = response.planStatus(); switch (planStatus) { case COMPLETED: - currentPlanId = response.planId(); return scanTasksIterable(response.planTasks(), response.fileScanTasks()); case SUBMITTED: Endpoint.check(supportedEndpoints, Endpoint.V1_FETCH_TABLE_SCAN_PLAN); - return fetchPlanningResult(response.planId()); + return fetchPlanningResult(); case FAILED: throw new IllegalStateException( - String.format( - "Received status: %s for planId: %s", PlanStatus.FAILED, response.planId())); + String.format("Received status: %s for planId: %s", PlanStatus.FAILED, planId)); case CANCELLED: throw new IllegalStateException( - String.format( - "Received status: %s for planId: %s", PlanStatus.CANCELLED, response.planId())); + String.format("Received status: %s for planId: %s", PlanStatus.CANCELLED, planId)); default: throw new IllegalStateException( - String.format("Invalid planStatus: %s for planId: %s", planStatus, response.planId())); + String.format("Invalid planStatus: %s for planId: %s", planStatus, planId)); } } - private CloseableIterable fetchPlanningResult(String planId) { - currentPlanId = planId; - + private CloseableIterable fetchPlanningResult() { RetryPolicy retryPolicy = RetryPolicy.builder() .handleResultIf(response -> response.planStatus() == PlanStatus.SUBMITTED) @@ -260,7 +256,6 @@ private CloseableIterable scanTasksIterable( @VisibleForTesting @SuppressWarnings("checkstyle:RegexpMultiline") public boolean cancelPlan() { - String planId = currentPlanId; if (planId == null || !supportedEndpoints.contains(Endpoint.V1_CANCEL_TABLE_SCAN_PLAN)) { return false; } @@ -272,7 +267,7 @@ public boolean cancelPlan() { null, headers, ErrorHandlers.planErrorHandler()); - currentPlanId = null; + this.planId = null; return true; } catch (Exception e) { // Plan might have already completed or failed, which is acceptable From 05998edb8808ae21739072cca17b621925a2081a Mon Sep 17 00:00:00 2001 From: Prashant Singh <35593236+singhpk234@users.noreply.github.com> Date: Thu, 18 Dec 2025 11:22:46 -0800 Subject: [PATCH 111/201] Fix: Enable metadata tables support for REST scan planning (#14881) Co-authored-by: Prashant Kumar Singh --- .../org/apache/iceberg/rest/RESTSessionCatalog.java | 10 ++++++---- .../spark/extensions/TestRemoteScanPlanning.java | 6 ------ 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 72dd24975e26..61e25d3d4fc6 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -483,11 +483,13 @@ public Table loadTable(SessionContext context, TableIdentifier identifier) { trackFileIO(ops); - RESTTable restTable = restTableForScanPlanning(ops, finalIdentifier, tableClient); - // RestTable should be only be returned for non-metadata tables, because client would + // RestTable should only be returned for non-metadata tables, because client would // not have access to metadata files for example manifests, since all it needs is catalog. - if (restTable != null) { - return restTable; + if (metadataType == null) { + RESTTable restTable = restTableForScanPlanning(ops, finalIdentifier, tableClient); + if (restTable != null) { + return restTable; + } } BaseTable table = diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java index 6e389730f7a7..14e6c358898c 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java @@ -55,10 +55,4 @@ protected static Object[][] parameters() { public void testBinaryInFilter() { super.testBinaryInFilter(); } - - @TestTemplate - @Disabled("Metadata tables are currently not supported") - public void testMetadataTables() { - super.testMetadataTables(); - } } From 00bf964f1df2454b24f93bbe7d36278ec229fb48 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 19 Dec 2025 14:53:33 +0100 Subject: [PATCH 112/201] Spark: Order results to fix test flakiness with remote scan planning (#14894) --- .../apache/iceberg/spark/sql/TestSelect.java | 87 +++++++++++-------- 1 file changed, 53 insertions(+), 34 deletions(-) diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index cf4ccd62dbc3..8b146d925bca 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -212,14 +212,14 @@ public void testMetadataTables() { public void testSnapshotInTableName() { // get the snapshot ID of the last write and get the current row set as expected long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER by id", tableName); // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); String prefix = "snapshot_id_"; // read the table at the snapshot - List actual = sql("SELECT * FROM %s.%s", tableName, prefix + snapshotId); + List actual = sql("SELECT * FROM %s.%s ORDER by id", tableName, prefix + snapshotId); assertEquals("Snapshot at specific ID, prefix " + prefix, expected, actual); // read the table using DataFrameReader option @@ -228,7 +228,8 @@ public void testSnapshotInTableName() { .read() .format("iceberg") .option(SparkReadOptions.SNAPSHOT_ID, snapshotId) - .load(tableName); + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at specific ID " + snapshotId, expected, fromDF); } @@ -238,14 +239,14 @@ 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); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER by id", tableName); // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); String prefix = "at_timestamp_"; // read the table at the snapshot - List actual = sql("SELECT * FROM %s.%s", tableName, prefix + timestamp); + List actual = sql("SELECT * FROM %s.%s ORDER by id", tableName, prefix + timestamp); assertEquals("Snapshot at timestamp, prefix " + prefix, expected, actual); // read the table using DataFrameReader option @@ -254,7 +255,8 @@ public void testTimestampInTableName() { .read() .format("iceberg") .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) - .load(tableName); + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at timestamp " + timestamp, expected, fromDF); } @@ -263,19 +265,20 @@ public void testTimestampInTableName() { public void testVersionAsOf() { // get the snapshot ID of the last write and get the current row set as expected long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); // read the table at the snapshot - List actual1 = sql("SELECT * FROM %s VERSION AS OF %s", tableName, snapshotId); + List actual1 = + sql("SELECT * FROM %s VERSION AS OF %s ORDER BY id", tableName, snapshotId); assertEquals("Snapshot at specific ID", expected, actual1); // read the table at the snapshot // HIVE time travel syntax List actual2 = - sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF %s", tableName, snapshotId); + sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF %s ORDER BY id", tableName, snapshotId); assertEquals("Snapshot at specific ID", expected, actual2); // read the table using DataFrameReader option: versionAsOf @@ -284,7 +287,8 @@ public void testVersionAsOf() { .read() .format("iceberg") .option(SparkReadOptions.VERSION_AS_OF, snapshotId) - .load(tableName); + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at specific ID " + snapshotId, expected, fromDF); } @@ -294,28 +298,35 @@ public void testTagReference() { Table table = validationCatalog.loadTable(tableIdent); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createTag("test_tag", snapshotId).commit(); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER by id", tableName); // create a second snapshot, read the table at the tag sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); - List actual1 = sql("SELECT * FROM %s VERSION AS OF 'test_tag'", tableName); + List actual1 = + sql("SELECT * FROM %s VERSION AS OF 'test_tag' ORDER by id", tableName); assertEquals("Snapshot at specific tag reference name", expected, actual1); // read the table at the tag // HIVE time travel syntax - List actual2 = sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_tag'", tableName); + List actual2 = + sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_tag' ORDER by id", tableName); assertEquals("Snapshot at specific tag reference name", expected, actual2); // Spark session catalog does not support extended table names if (!"spark_catalog".equals(catalogName)) { // read the table using the "tag_" prefix in the table name - List actual3 = sql("SELECT * FROM %s.tag_test_tag", tableName); + List actual3 = sql("SELECT * FROM %s.tag_test_tag ORDER by id", tableName); assertEquals("Snapshot at specific tag reference name, prefix", expected, actual3); } // read the table using DataFrameReader option: tag Dataset df = - spark.read().format("iceberg").option(SparkReadOptions.TAG, "test_tag").load(tableName); + spark + .read() + .format("iceberg") + .option(SparkReadOptions.TAG, "test_tag") + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at specific tag reference name", expected, fromDF); } @@ -326,7 +337,7 @@ public void testUseSnapshotIdForTagReferenceAsOf() { long snapshotId1 = table.currentSnapshot().snapshotId(); // create a second snapshot, read the table at the snapshot - List actual = sql("SELECT * FROM %s", tableName); + List actual = sql("SELECT * FROM %s ORDER by id", tableName); sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); table.refresh(); @@ -337,11 +348,11 @@ public void testUseSnapshotIdForTagReferenceAsOf() { // this means if a tag name matches a snapshot ID, it will always choose snapshotID to travel // to. List travelWithStringResult = - sql("SELECT * FROM %s VERSION AS OF '%s'", tableName, snapshotId1); + sql("SELECT * FROM %s VERSION AS OF '%s' ORDER by id", tableName, snapshotId1); assertEquals("Snapshot at specific tag reference name", actual, travelWithStringResult); List travelWithLongResult = - sql("SELECT * FROM %s VERSION AS OF %s", tableName, snapshotId1); + sql("SELECT * FROM %s VERSION AS OF %s ORDER by id", tableName, snapshotId1); assertEquals("Snapshot at specific tag reference name", actual, travelWithLongResult); } @@ -350,23 +361,24 @@ public void testBranchReference() { Table table = validationCatalog.loadTable(tableIdent); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createBranch("test_branch", snapshotId).commit(); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER by id", tableName); // create a second snapshot, read the table at the branch sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); - List actual1 = sql("SELECT * FROM %s VERSION AS OF 'test_branch'", tableName); + List actual1 = + sql("SELECT * FROM %s VERSION AS OF 'test_branch' ORDER by id", tableName); assertEquals("Snapshot at specific branch reference name", expected, actual1); // read the table at the branch // HIVE time travel syntax List actual2 = - sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_branch'", tableName); + sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_branch' ORDER by id", tableName); assertEquals("Snapshot at specific branch reference name", expected, actual2); // Spark session catalog does not support extended table names if (!"spark_catalog".equals(catalogName)) { // read the table using the "branch_" prefix in the table name - List actual3 = sql("SELECT * FROM %s.branch_test_branch", tableName); + List actual3 = sql("SELECT * FROM %s.branch_test_branch ORDER by id", tableName); assertEquals("Snapshot at specific branch reference name, prefix", expected, actual3); } @@ -376,7 +388,8 @@ public void testBranchReference() { .read() .format("iceberg") .option(SparkReadOptions.BRANCH, "test_branch") - .load(tableName); + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at specific branch reference name", expected, fromDF); } @@ -400,16 +413,16 @@ public void readAndWriteWithBranchAfterSchemaChange() { // time-travel query using snapshot id should return the snapshot's schema long branchSnapshotId = table.refs().get(branchName).snapshotId(); - assertThat(sql("SELECT * FROM %s VERSION AS OF %s", tableName, branchSnapshotId)) + assertThat(sql("SELECT * FROM %s VERSION AS OF %s ORDER by id", tableName, branchSnapshotId)) .containsExactlyElementsOf(expected); // querying the head of the branch should return the table's schema - assertThat(sql("SELECT * FROM %s VERSION AS OF '%s'", tableName, branchName)) + assertThat(sql("SELECT * FROM %s VERSION AS OF '%s' ORDER by id", tableName, branchName)) .containsExactly(row(1L, "a", null), row(2L, "b", null), row(3L, "c", null)); if (!"spark_catalog".equals(catalogName)) { // querying the head of the branch using 'branch_' should return the table's schema - assertThat(sql("SELECT * FROM %s.branch_%s", tableName, branchName)) + assertThat(sql("SELECT * FROM %s.branch_%s ORDER by id", tableName, branchName)) .containsExactly(row(1L, "a", null), row(2L, "b", null), row(3L, "c", null)); } @@ -419,7 +432,7 @@ public void readAndWriteWithBranchAfterSchemaChange() { tableName, branchName); // querying the head of the branch returns the table's schema - assertThat(sql("SELECT * FROM %s VERSION AS OF '%s'", tableName, branchName)) + assertThat(sql("SELECT * FROM %s VERSION AS OF '%s' ORDER by id", tableName, branchName)) .containsExactlyInAnyOrder( row(1L, "a", null), row(2L, "b", null), @@ -429,7 +442,12 @@ public void readAndWriteWithBranchAfterSchemaChange() { // using DataFrameReader with the 'branch' option should return the table's schema Dataset df = - spark.read().format("iceberg").option(SparkReadOptions.BRANCH, branchName).load(tableName); + spark + .read() + .format("iceberg") + .option(SparkReadOptions.BRANCH, branchName) + .load(tableName) + .orderBy("id"); assertThat(rowsToJava(df.collectAsList())) .containsExactlyInAnyOrder( row(1L, "a", null), @@ -456,30 +474,30 @@ public void testTimestampAsOf() { SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); String formattedDate = sdf.format(new Date(timestamp)); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); // read the table at the timestamp in long format i.e 1656507980463. List actualWithLongFormat = - sql("SELECT * FROM %s TIMESTAMP AS OF %s", tableName, timestampInSeconds); + sql("SELECT * FROM %s TIMESTAMP AS OF %s ORDER BY id", tableName, timestampInSeconds); assertEquals("Snapshot at timestamp", expected, actualWithLongFormat); // read the table at the timestamp in date format i.e 2022-06-29 18:40:37 List actualWithDateFormat = - sql("SELECT * FROM %s TIMESTAMP AS OF '%s'", tableName, formattedDate); + sql("SELECT * FROM %s TIMESTAMP AS OF '%s' ORDER BY id", tableName, formattedDate); assertEquals("Snapshot at timestamp", expected, actualWithDateFormat); // HIVE time travel syntax // read the table at the timestamp in long format i.e 1656507980463. List actualWithLongFormatInHiveSyntax = - sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF %s", tableName, timestampInSeconds); + sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF %s ORDER BY id", tableName, timestampInSeconds); assertEquals("Snapshot at specific ID", expected, actualWithLongFormatInHiveSyntax); // read the table at the timestamp in date format i.e 2022-06-29 18:40:37 List actualWithDateFormatInHiveSyntax = - sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF '%s'", tableName, formattedDate); + sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF '%s' ORDER BY id", tableName, formattedDate); assertEquals("Snapshot at specific ID", expected, actualWithDateFormatInHiveSyntax); // read the table using DataFrameReader option @@ -488,7 +506,8 @@ public void testTimestampAsOf() { .read() .format("iceberg") .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedDate) - .load(tableName); + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at timestamp " + timestamp, expected, fromDF); } From 2a006ba8d0f9b668efcfbc6c523c87ceddf35b94 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 19 Dec 2025 17:44:00 +0100 Subject: [PATCH 113/201] Core: Close planFiles() iterable in CatalogHandler (#14891) --- .../apache/iceberg/rest/CatalogHandlers.java | 57 ++++++++++--------- 1 file changed, 31 insertions(+), 26 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java index e3eff1228af5..229497576af9 100644 --- a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java @@ -23,6 +23,7 @@ import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES_DEFAULT; import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT; +import java.io.IOException; import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.util.Collections; @@ -64,6 +65,7 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; +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.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -805,35 +807,38 @@ static void clearPlanningState() { */ private static Pair, String> planFilesFor( Scan scan, String planId, String tableId, int tasksPerPlanTask) { - Iterable planTasks = scan.planFiles(); - String planTaskPrefix = planId + "-" + tableId + "-"; - - // Handle empty table scans - if (!planTasks.iterator().hasNext()) { - String planTaskKey = planTaskPrefix + "0"; - // Add empty scan to planning state so async calls know the scan completed - IN_MEMORY_PLANNING_STATE.addPlanTask(planTaskKey, Collections.emptyList()); - return Pair.of(Collections.emptyList(), planTaskKey); - } - - Iterable> taskGroupings = Iterables.partition(planTasks, tasksPerPlanTask); - int planTaskSequence = 0; - String previousPlanTask = null; - String firstPlanTaskKey = null; - List initialFileScanTasks = null; - for (List taskGrouping : taskGroupings) { - String planTaskKey = planTaskPrefix + planTaskSequence++; - IN_MEMORY_PLANNING_STATE.addPlanTask(planTaskKey, taskGrouping); - if (previousPlanTask != null) { - IN_MEMORY_PLANNING_STATE.addNextPlanTask(previousPlanTask, planTaskKey); - } else { - firstPlanTaskKey = planTaskKey; - initialFileScanTasks = taskGrouping; + try (CloseableIterable planTasks = scan.planFiles()) { + String planTaskPrefix = planId + "-" + tableId + "-"; + + // Handle empty table scans + if (!planTasks.iterator().hasNext()) { + String planTaskKey = planTaskPrefix + "0"; + // Add empty scan to planning state so async calls know the scan completed + IN_MEMORY_PLANNING_STATE.addPlanTask(planTaskKey, Collections.emptyList()); + return Pair.of(Collections.emptyList(), planTaskKey); } - previousPlanTask = planTaskKey; + Iterable> taskGroupings = Iterables.partition(planTasks, tasksPerPlanTask); + int planTaskSequence = 0; + String previousPlanTask = null; + String firstPlanTaskKey = null; + List initialFileScanTasks = null; + for (List taskGrouping : taskGroupings) { + String planTaskKey = planTaskPrefix + planTaskSequence++; + IN_MEMORY_PLANNING_STATE.addPlanTask(planTaskKey, taskGrouping); + if (previousPlanTask != null) { + IN_MEMORY_PLANNING_STATE.addNextPlanTask(previousPlanTask, planTaskKey); + } else { + firstPlanTaskKey = planTaskKey; + initialFileScanTasks = taskGrouping; + } + + previousPlanTask = planTaskKey; + } + return Pair.of(initialFileScanTasks, firstPlanTaskKey); + } catch (IOException e) { + throw new RuntimeException(e); } - return Pair.of(initialFileScanTasks, firstPlanTaskKey); } @SuppressWarnings("FutureReturnValueIgnored") From 59280a106ca5f378edfd47d3576c4d10011e5e80 Mon Sep 17 00:00:00 2001 From: wuya Date: Sat, 20 Dec 2025 01:02:45 +0800 Subject: [PATCH 114/201] Hive: Update view query in HMS when replacing view (#14831) --- .../iceberg/hive/HMSTablePropertyHelper.java | 8 ++++- .../iceberg/hive/HiveViewOperations.java | 3 +- .../iceberg/hive/TestHiveViewCommits.java | 34 ++++++++++++++++++- 3 files changed, 42 insertions(+), 3 deletions(-) diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HMSTablePropertyHelper.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HMSTablePropertyHelper.java index a50210bfc5db..1895ecf831aa 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HMSTablePropertyHelper.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HMSTablePropertyHelper.java @@ -136,7 +136,8 @@ public static void updateHmsTableForIcebergView( ViewMetadata metadata, Set obsoleteProps, long maxHiveTablePropertySize, - String currentLocation) { + String currentLocation, + String sqlQuery) { Map parameters = Optional.ofNullable(tbl.getParameters()).orElseGet(Maps::newHashMap); @@ -154,6 +155,11 @@ public static void updateHmsTableForIcebergView( metadata.schema(), maxHiveTablePropertySize); tbl.setParameters(parameters); + + if (sqlQuery != null) { + tbl.setViewExpandedText(sqlQuery); + tbl.setViewOriginalText(sqlQuery); + } } private static void setCommonParameters( diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java index b58369b6bbaa..395a355455dd 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java @@ -182,7 +182,8 @@ public void doCommit(ViewMetadata base, ViewMetadata metadata) { metadata, removedProps, maxHiveTablePropertySize, - currentMetadataLocation()); + currentMetadataLocation(), + sqlFor(metadata)); lock.ensureActive(); try { diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCommits.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCommits.java index ee0a88f46b95..738eac5b5adb 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCommits.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCommits.java @@ -37,6 +37,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.Table; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.Schema; @@ -48,6 +49,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; import org.apache.iceberg.view.BaseView; +import org.apache.iceberg.view.ImmutableSQLViewRepresentation; import org.apache.iceberg.view.View; import org.apache.iceberg.view.ViewMetadata; import org.apache.thrift.TException; @@ -70,6 +72,7 @@ public class TestHiveViewCommits { required(3, "id", Types.IntegerType.get(), "unique ID"), required(4, "data", Types.StringType.get())); private static final TableIdentifier VIEW_IDENTIFIER = TableIdentifier.of(NS, VIEW_NAME); + private static final String VIEW_QUERY = "select * from ns.tbl"; @RegisterExtension protected static final HiveMetastoreExtension HIVE_METASTORE_EXTENSION = @@ -100,7 +103,7 @@ public void createTestView() { .buildView(VIEW_IDENTIFIER) .withSchema(SCHEMA) .withDefaultNamespace(NS) - .withQuery("hive", "select * from ns.tbl") + .withQuery("hive", VIEW_QUERY) .create(); viewLocation = new Path(view.location()); } @@ -111,6 +114,35 @@ public void dropTestView() throws IOException { catalog.dropView(VIEW_IDENTIFIER); } + @Test + public void testViewQueryIsUpdatedOnCommit() throws Exception { + HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); + assertThat(view.currentVersion().representations()) + .containsExactly( + ImmutableSQLViewRepresentation.builder().sql(VIEW_QUERY).dialect("hive").build()); + + Table hmsTable = ops.loadHmsTable(); + assertThat(hmsTable.getViewOriginalText()).isEqualTo(VIEW_QUERY); + assertThat(hmsTable.getViewExpandedText()).isEqualTo(VIEW_QUERY); + + String newQuery = "select * from ns.tbl2 limit 10"; + view = + catalog + .buildView(VIEW_IDENTIFIER) + .withSchema(SCHEMA) + .withDefaultNamespace(NS) + .withQuery("hive", newQuery) + .replace(); + + assertThat(view.currentVersion().representations()) + .containsExactly( + ImmutableSQLViewRepresentation.builder().sql(newQuery).dialect("hive").build()); + + Table updatedHmsTable = ops.loadHmsTable(); + assertThat(updatedHmsTable.getViewOriginalText()).isEqualTo(newQuery); + assertThat(updatedHmsTable.getViewExpandedText()).isEqualTo(newQuery); + } + @Test public void testSuppressUnlockExceptions() { HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); From 554a3c1d2ad3faf1397f763c8ae9b1e69c9bb55d Mon Sep 17 00:00:00 2001 From: Joy Haldar Date: Sat, 20 Dec 2025 03:46:55 +0530 Subject: [PATCH 115/201] GCP: Add service account impersonation support for BigQueryMetastoreCatalog (#14447) * Add supporting impersonation in BigQueryMetastoreCatalog Co-authored-by: Joy Haldar --- .../bigquery/BigQueryMetastoreCatalog.java | 46 ++-- .../bigquery/BigQueryMetastoreClientImpl.java | 8 +- .../gcp/bigquery/BigQueryProperties.java | 197 ++++++++++++++++++ .../gcp/bigquery/TestBigQueryCatalog.java | 2 +- .../gcp/bigquery/TestBigQueryProperties.java | 95 +++++++++ .../bigquery/TestBigQueryTableOperations.java | 2 +- .../org/apache/iceberg/gcp/GCPProperties.java | 79 +++++++ .../iceberg/gcp/gcs/PrefixedStorage.java | 24 +++ .../iceberg/gcp/gcs/TestPrefixedStorage.java | 41 ++++ 9 files changed, 466 insertions(+), 28 deletions(-) create mode 100644 bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryProperties.java create mode 100644 bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryProperties.java diff --git a/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryMetastoreCatalog.java b/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryMetastoreCatalog.java index 34ec7a62d5b5..dd01246cb01f 100644 --- a/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryMetastoreCatalog.java +++ b/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryMetastoreCatalog.java @@ -23,7 +23,6 @@ import com.google.api.services.bigquery.model.DatasetReference; import com.google.api.services.bigquery.model.ExternalCatalogDatasetOptions; import com.google.api.services.bigquery.model.TableReference; -import com.google.cloud.ServiceOptions; import com.google.cloud.bigquery.BigQueryOptions; import java.io.IOException; import java.io.UncheckedIOException; @@ -58,14 +57,25 @@ public class BigQueryMetastoreCatalog extends BaseMetastoreCatalog implements SupportsNamespaces, Configurable { - // User provided properties. - public static final String PROJECT_ID = "gcp.bigquery.project-id"; - public static final String GCP_LOCATION = "gcp.bigquery.location"; - public static final String LIST_ALL_TABLES = "gcp.bigquery.list-all-tables"; + /** + * @deprecated since 1.11.0, will be removed in 1.12.0; use {@link BigQueryProperties#PROJECT_ID} + * instead. + */ + @Deprecated public static final String PROJECT_ID = "gcp.bigquery.project-id"; - private static final Logger LOG = LoggerFactory.getLogger(BigQueryMetastoreCatalog.class); + /** + * @deprecated since 1.11.0, will be removed in 1.12.0; use {@link + * BigQueryProperties#GCP_LOCATION} instead. + */ + @Deprecated public static final String GCP_LOCATION = "gcp.bigquery.location"; + + /** + * @deprecated since 1.11.0, will be removed in 1.12.0; use {@link + * BigQueryProperties#LIST_ALL_TABLES} instead. + */ + @Deprecated public static final String LIST_ALL_TABLES = "gcp.bigquery.list-all-tables"; - private static final String DEFAULT_GCP_LOCATION = "us"; + private static final Logger LOG = LoggerFactory.getLogger(BigQueryMetastoreCatalog.class); private String catalogName; private Map catalogProperties; @@ -81,23 +91,17 @@ public BigQueryMetastoreCatalog() {} @Override public void initialize(String name, Map properties) { - Preconditions.checkArgument( - properties.containsKey(PROJECT_ID), - "Invalid GCP project: %s must be specified", - PROJECT_ID); - this.projectId = properties.get(PROJECT_ID); - this.projectLocation = properties.getOrDefault(GCP_LOCATION, DEFAULT_GCP_LOCATION); + BigQueryProperties bigQueryProperties = new BigQueryProperties(properties); + + this.projectId = bigQueryProperties.projectId(); + this.projectLocation = bigQueryProperties.location(); + this.listAllTables = bigQueryProperties.listAllTables(); - BigQueryOptions options = - BigQueryOptions.newBuilder() - .setProjectId(projectId) - .setLocation(projectLocation) - .setRetrySettings(ServiceOptions.getDefaultRetrySettings()) - .build(); + BigQueryOptions bigQueryOptions = bigQueryProperties.metastoreOptions(); try { - client = new BigQueryMetastoreClientImpl(options); + client = new BigQueryMetastoreClientImpl(bigQueryOptions); } catch (IOException e) { throw new UncheckedIOException("Creating BigQuery client failed", e); } catch (GeneralSecurityException e) { @@ -134,8 +138,6 @@ void initialize( CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.io.ResolvingFileIO"), properties, conf); - - this.listAllTables = Boolean.parseBoolean(properties.getOrDefault(LIST_ALL_TABLES, "true")); } @Override diff --git a/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryMetastoreClientImpl.java b/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryMetastoreClientImpl.java index f61c37fb08de..beee73628975 100644 --- a/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryMetastoreClientImpl.java +++ b/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryMetastoreClientImpl.java @@ -28,7 +28,6 @@ import com.google.api.client.json.gson.GsonFactory; import com.google.api.client.util.Data; import com.google.api.services.bigquery.Bigquery; -import com.google.api.services.bigquery.BigqueryScopes; import com.google.api.services.bigquery.model.Dataset; import com.google.api.services.bigquery.model.DatasetList; import com.google.api.services.bigquery.model.DatasetList.Datasets; @@ -127,9 +126,10 @@ public BigQueryMetastoreClientImpl(BigQueryOptions options) throws IOException, GeneralSecurityException { // Initialize client that will be used to send requests. This client only needs to be created // once, and can be reused for multiple requests - HttpCredentialsAdapter httpCredentialsAdapter = - new HttpCredentialsAdapter( - GoogleCredentials.getApplicationDefault().createScoped(BigqueryScopes.all())); + + GoogleCredentials credentials = (GoogleCredentials) options.getCredentials(); + HttpCredentialsAdapter httpCredentialsAdapter = new HttpCredentialsAdapter(credentials); + this.client = new Bigquery.Builder( GoogleNetHttpTransport.newTrustedTransport(), diff --git a/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryProperties.java b/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryProperties.java new file mode 100644 index 000000000000..3e352e66bd4d --- /dev/null +++ b/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryProperties.java @@ -0,0 +1,197 @@ +/* + * 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.gcp.bigquery; + +import com.google.api.services.bigquery.BigqueryScopes; +import com.google.auth.oauth2.GoogleCredentials; +import com.google.auth.oauth2.ImpersonatedCredentials; +import com.google.cloud.ServiceOptions; +import com.google.cloud.bigquery.BigQueryOptions; +import java.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class BigQueryProperties implements Serializable { + + private static final Logger LOG = LoggerFactory.getLogger(BigQueryProperties.class); + + // User provided properties. + public static final String PROJECT_ID = "gcp.bigquery.project-id"; + public static final String GCP_LOCATION = "gcp.bigquery.location"; + public static final String LIST_ALL_TABLES = "gcp.bigquery.list-all-tables"; + + // Service account impersonation properties. + public static final String IMPERSONATE_SERVICE_ACCOUNT = + "gcp.bigquery.impersonate.service-account"; + public static final String IMPERSONATE_LIFETIME_SECONDS = + "gcp.bigquery.impersonate.lifetime-seconds"; + public static final String IMPERSONATE_SCOPES = "gcp.bigquery.impersonate.scopes"; + public static final String IMPERSONATE_DELEGATES = "gcp.bigquery.impersonate.delegates"; + + public static final String DEFAULT_GCP_LOCATION = "us"; + private static final int DEFAULT_LIFETIME_SECONDS = 3600; + private static final List DEFAULT_SCOPES = + ImmutableList.of("https://www.googleapis.com/auth/cloud-platform"); + + private final String projectId; + private final String location; + private final boolean listAllTables; + private final String impersonateServiceAccount; + private final int lifetimeSeconds; + private final List scopes; + private final List delegates; + + @VisibleForTesting + List parseCommaSeparatedList(String input, List defaultValue) { + if (input == null || input.trim().isEmpty()) { + return defaultValue; + } + return Arrays.stream(input.split(",")) + .map(String::trim) + .filter(str -> !str.isEmpty()) + .distinct() + .collect(Collectors.toList()); + } + + @VisibleForTesting + List expandScopes(List inputScopes) { + if (inputScopes == null || inputScopes.isEmpty()) { + return inputScopes; + } + return inputScopes.stream() + .map( + inputScope -> { + if (inputScope.startsWith("https://")) { + return inputScope; + } + if (inputScope.startsWith("http://")) { + return inputScope.replace("http://", "https://"); + } + + return "https://www.googleapis.com/auth/" + inputScope; + }) + .collect(Collectors.toList()); + } + + BigQueryProperties(Map properties) { + Preconditions.checkNotNull(properties, "Properties cannot be null"); + + this.projectId = properties.get(PROJECT_ID); + Preconditions.checkArgument( + projectId != null, "Invalid GCP project: %s must be specified", PROJECT_ID); + + this.location = properties.getOrDefault(GCP_LOCATION, DEFAULT_GCP_LOCATION); + + this.listAllTables = Boolean.parseBoolean(properties.getOrDefault(LIST_ALL_TABLES, "true")); + + // Impersonation properties, optional + this.impersonateServiceAccount = properties.get(IMPERSONATE_SERVICE_ACCOUNT); + + this.lifetimeSeconds = + Integer.parseInt( + properties.getOrDefault( + IMPERSONATE_LIFETIME_SECONDS, String.valueOf(DEFAULT_LIFETIME_SECONDS))); + + List rawScopes = + parseCommaSeparatedList(properties.get(IMPERSONATE_SCOPES), DEFAULT_SCOPES); + this.scopes = expandScopes(rawScopes); + + this.delegates = parseCommaSeparatedList(properties.get(IMPERSONATE_DELEGATES), null); + + if (impersonateServiceAccount != null) { + LOG.info( + "BigQuery impersonation configured for service account: {}, lifetime: {} seconds", + impersonateServiceAccount, + lifetimeSeconds); + } + } + + String projectId() { + return projectId; + } + + String location() { + return location; + } + + boolean listAllTables() { + return listAllTables; + } + + BigQueryOptions metastoreOptions() { + BigQueryOptions.Builder builder = + BigQueryOptions.newBuilder() + .setProjectId(projectId) + .setLocation(location) + .setRetrySettings(ServiceOptions.getDefaultRetrySettings()); + + if (impersonateServiceAccount != null) { + builder.setCredentials(buildImpersonatedCredentials()); + } else { + builder.setCredentials(buildApplicationDefaultCredentials()); + } + + return builder.build(); + } + + private GoogleCredentials buildApplicationDefaultCredentials() { + try { + GoogleCredentials applicationDefaultCredentials = + GoogleCredentials.getApplicationDefault().createScoped(BigqueryScopes.all()); + + LOG.debug( + "Created application default credentials for BigQuery: {}", + applicationDefaultCredentials); + + return applicationDefaultCredentials; + } catch (IOException e) { + throw new UncheckedIOException("Failed to get application default credentials", e); + } + } + + private ImpersonatedCredentials buildImpersonatedCredentials() { + try { + GoogleCredentials sourceCredentials = GoogleCredentials.getApplicationDefault(); + + ImpersonatedCredentials impersonatedCredentials = + ImpersonatedCredentials.create( + sourceCredentials, impersonateServiceAccount, delegates, scopes, lifetimeSeconds); + + // refresh to validate credentials and get intial token + impersonatedCredentials.refresh(); + + LOG.debug( + "Created impersonated credentials for BigQuery: Target={}", impersonateServiceAccount); + + return impersonatedCredentials; + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to create impersonated credentials for " + impersonateServiceAccount, e); + } + } +} diff --git a/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryCatalog.java b/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryCatalog.java index f7594f78ffac..cdeaa1ef1e63 100644 --- a/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryCatalog.java +++ b/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryCatalog.java @@ -20,7 +20,7 @@ import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_BIGQUERY; -import static org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog.PROJECT_ID; +import static org.apache.iceberg.gcp.bigquery.BigQueryProperties.PROJECT_ID; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; diff --git a/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryProperties.java b/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryProperties.java new file mode 100644 index 000000000000..83fb526ae739 --- /dev/null +++ b/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryProperties.java @@ -0,0 +1,95 @@ +/* + * 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.gcp.bigquery; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Arrays; +import java.util.Map; +import org.junit.jupiter.api.Test; + +public class TestBigQueryProperties { + + @Test + public void testInitializeWithValidProperties() { + Map properties = + Map.of( + BigQueryProperties.PROJECT_ID, "test-project", + BigQueryProperties.GCP_LOCATION, "us-central1"); + BigQueryProperties bigQueryProperties = new BigQueryProperties(properties); + + assertThat(bigQueryProperties.projectId()).isEqualTo("test-project"); + assertThat(bigQueryProperties.location()).isEqualTo("us-central1"); + } + + @Test + public void testInitializeWithDefaultLocation() { + Map properties = Map.of(BigQueryProperties.PROJECT_ID, "test-project"); + BigQueryProperties bigQueryProperties = new BigQueryProperties(properties); + + assertThat(bigQueryProperties.projectId()).isEqualTo("test-project"); + assertThat(bigQueryProperties.location()).isEqualTo("us"); + } + + @Test + public void testInitializeWithNullProperties() { + assertThatThrownBy(() -> new BigQueryProperties(null)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("Properties cannot be null"); + } + + @Test + public void testInitializeWithoutProjectId() { + Map properties = Map.of(BigQueryProperties.GCP_LOCATION, "us-central1"); + assertThatThrownBy(() -> new BigQueryProperties(properties)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("gcp.bigquery.project-id"); + } + + @Test + public void testExpandScopesMixedFormats() { + Map properties = Map.of(BigQueryProperties.PROJECT_ID, "test-project"); + BigQueryProperties bigQueryProperties = new BigQueryProperties(properties); + + assertThat( + bigQueryProperties.expandScopes( + Arrays.asList( + "bigquery", + "http://www.googleapis.com/auth/devstorage.read_write", + "https://www.googleapis.com/auth/cloud-platform"))) + .containsExactly( + "https://www.googleapis.com/auth/bigquery", + "https://www.googleapis.com/auth/devstorage.read_write", + "https://www.googleapis.com/auth/cloud-platform"); + } + + @Test + public void testParseCommaSeparatedList() { + Map properties = Map.of(BigQueryProperties.PROJECT_ID, "test-project"); + BigQueryProperties bigQueryProperties = new BigQueryProperties(properties); + + assertThat(bigQueryProperties.parseCommaSeparatedList("a, b, c", null)) + .containsExactly("a", "b", "c"); + assertThat(bigQueryProperties.parseCommaSeparatedList(null, Arrays.asList("default"))) + .containsExactly("default"); + assertThat(bigQueryProperties.parseCommaSeparatedList("", Arrays.asList("default"))) + .containsExactly("default"); + } +} diff --git a/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryTableOperations.java b/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryTableOperations.java index 340410688295..4666ec61f4d3 100644 --- a/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryTableOperations.java +++ b/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryTableOperations.java @@ -19,7 +19,7 @@ package org.apache.iceberg.gcp.bigquery; import static org.apache.iceberg.BaseMetastoreTableOperations.METADATA_LOCATION_PROP; -import static org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog.PROJECT_ID; +import static org.apache.iceberg.gcp.bigquery.BigQueryProperties.PROJECT_ID; import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatNoException; diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java b/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java index 52abd850031a..2702ce565d4e 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java @@ -19,11 +19,16 @@ package org.apache.iceberg.gcp; import java.io.Serializable; +import java.util.Arrays; import java.util.Date; +import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.stream.Collectors; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; 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.rest.RESTUtil; import org.apache.iceberg.util.PropertyUtil; @@ -49,6 +54,15 @@ public class GCPProperties implements Serializable { public static final String GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT = "gcs.oauth2.refresh-credentials-endpoint"; + // Impersonation properties + public static final String GCS_IMPERSONATE_SERVICE_ACCOUNT = "gcs.impersonate.service-account"; + public static final String GCS_IMPERSONATE_LIFETIME_SECONDS = "gcs.impersonate.lifetime-seconds"; + public static final String GCS_IMPERSONATE_DELEGATES = "gcs.impersonate.delegates"; + public static final String GCS_IMPERSONATE_SCOPES = "gcs.impersonate.scopes"; + public static final int GCS_IMPERSONATE_LIFETIME_SECONDS_DEFAULT = 3600; + private static final List GCS_IMPERSONATE_SCOPES_DEFAULT = + ImmutableList.of("https://www.googleapis.com/auth/cloud-platform"); + /** Controls whether vended credentials should be refreshed or not. Defaults to true. */ public static final String GCS_OAUTH2_REFRESH_CREDENTIALS_ENABLED = "gcs.oauth2.refresh-credentials-enabled"; @@ -85,8 +99,45 @@ public class GCPProperties implements Serializable { private boolean gcsOauth2RefreshCredentialsEnabled; private boolean gcsAnalyticsCoreEnabled; + private String gcsImpersonateServiceAccount; + private int gcsImpersonateLifetimeSeconds; + private List gcsImpersonateDelegates; + private List gcsImpersonateScopes; + private int gcsDeleteBatchSize = GCS_DELETE_BATCH_SIZE_DEFAULT; + @VisibleForTesting + List parseCommaSeparatedList(String input, List defaultValue) { + if (input == null || input.trim().isEmpty()) { + return defaultValue; + } + return Arrays.stream(input.split(",")) + .map(String::trim) + .filter(str -> !str.isEmpty()) + .distinct() + .collect(Collectors.toList()); + } + + @VisibleForTesting + List expandScopes(List inputScopes) { + if (inputScopes == null || inputScopes.isEmpty()) { + return inputScopes; + } + return inputScopes.stream() + .map( + inputScope -> { + if (inputScope.startsWith("https://")) { + return inputScope; + } + if (inputScope.startsWith("http://")) { + return inputScope.replace("http://", "https://"); + } + + return "https://www.googleapis.com/auth/" + inputScope; + }) + .collect(Collectors.toList()); + } + public GCPProperties() { this.allProperties = ImmutableMap.of(); } @@ -132,6 +183,18 @@ public GCPProperties(Map properties) { gcsDeleteBatchSize = PropertyUtil.propertyAsInt( properties, GCS_DELETE_BATCH_SIZE, GCS_DELETE_BATCH_SIZE_DEFAULT); + + gcsImpersonateServiceAccount = properties.get(GCS_IMPERSONATE_SERVICE_ACCOUNT); + gcsImpersonateLifetimeSeconds = + PropertyUtil.propertyAsInt( + properties, GCS_IMPERSONATE_LIFETIME_SECONDS, GCS_IMPERSONATE_LIFETIME_SECONDS_DEFAULT); + gcsImpersonateDelegates = + parseCommaSeparatedList(properties.get(GCS_IMPERSONATE_DELEGATES), null); + List rawScopes = + parseCommaSeparatedList( + properties.get(GCS_IMPERSONATE_SCOPES), GCS_IMPERSONATE_SCOPES_DEFAULT); + gcsImpersonateScopes = expandScopes(rawScopes); + gcsAnalyticsCoreEnabled = PropertyUtil.propertyAsBoolean(properties, GCS_ANALYTICS_CORE_ENABLED, false); } @@ -180,6 +243,22 @@ public Optional oauth2TokenExpiresAt() { return Optional.ofNullable(gcsOAuth2TokenExpiresAt); } + public Optional impersonateServiceAccount() { + return Optional.ofNullable(gcsImpersonateServiceAccount); + } + + public int impersonateLifetimeSeconds() { + return gcsImpersonateLifetimeSeconds; + } + + public List impersonateDelegates() { + return gcsImpersonateDelegates; + } + + public List impersonateScopes() { + return gcsImpersonateScopes; + } + public int deleteBatchSize() { return gcsDeleteBatchSize; } diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/PrefixedStorage.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/PrefixedStorage.java index 1f60db346d65..06017bb2e894 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/PrefixedStorage.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/PrefixedStorage.java @@ -20,6 +20,8 @@ import com.google.api.gax.rpc.FixedHeaderProvider; import com.google.auth.Credentials; +import com.google.auth.oauth2.GoogleCredentials; +import com.google.auth.oauth2.ImpersonatedCredentials; import com.google.cloud.NoCredentials; import com.google.cloud.gcs.analyticscore.client.GcsFileSystem; import com.google.cloud.gcs.analyticscore.client.GcsFileSystemImpl; @@ -141,11 +143,33 @@ private Credentials credentials(GCPProperties properties) { } else if (properties.noAuth()) { // Explicitly allow "no credentials" for testing purposes return NoCredentials.getInstance(); + } else if (properties.impersonateServiceAccount().isPresent()) { + return buildImpersonatedCredentials(properties); } else { return null; } } + private Credentials buildImpersonatedCredentials(GCPProperties properties) { + try { + GoogleCredentials sourceCredentials = GoogleCredentials.getApplicationDefault(); + + ImpersonatedCredentials impersonatedCredentials = + ImpersonatedCredentials.create( + sourceCredentials, + properties.impersonateServiceAccount().get(), + properties.impersonateDelegates(), + properties.impersonateScopes(), + properties.impersonateLifetimeSeconds()); + + // Refresh to get initial token + impersonatedCredentials.refresh(); + return impersonatedCredentials; + } catch (IOException e) { + throw new UncheckedIOException("Failed to create impersonated credentials for GCS", e); + } + } + private SerializableSupplier gcsFileSystemSupplier( Map properties) { ImmutableMap.Builder propertiesWithUserAgent = diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestPrefixedStorage.java b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestPrefixedStorage.java index cfabb2f65edd..dd68b037665e 100644 --- a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestPrefixedStorage.java +++ b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestPrefixedStorage.java @@ -74,6 +74,47 @@ public void userAgentPrefix() { .isEqualTo("gcsfileio/" + EnvironmentContext.get()); } + @Test + public void impersonationPropertiesAreRead() { + Map properties = + ImmutableMap.of( + GCPProperties.GCS_PROJECT_ID, "myProject", + GCPProperties.GCS_IMPERSONATE_SERVICE_ACCOUNT, + "test-sa@project.iam.gserviceaccount.com", + GCPProperties.GCS_IMPERSONATE_DELEGATES, "delegate-sa@project.iam.gserviceaccount.com", + GCPProperties.GCS_IMPERSONATE_LIFETIME_SECONDS, "1800", + GCPProperties.GCS_IMPERSONATE_SCOPES, "bigquery,devstorage.read_only"); + + GCPProperties gcpProperties = new GCPProperties(properties); + + assertThat(gcpProperties.impersonateServiceAccount()) + .contains("test-sa@project.iam.gserviceaccount.com"); + assertThat(gcpProperties.impersonateDelegates()) + .contains("delegate-sa@project.iam.gserviceaccount.com"); + assertThat(gcpProperties.impersonateLifetimeSeconds()).isEqualTo(1800); + assertThat(gcpProperties.impersonateScopes()) + .containsExactly( + "https://www.googleapis.com/auth/bigquery", + "https://www.googleapis.com/auth/devstorage.read_only"); + } + + @Test + public void impersonationPropertiesWithDefaults() { + Map properties = + ImmutableMap.of( + GCPProperties.GCS_PROJECT_ID, "myProject", + GCPProperties.GCS_IMPERSONATE_SERVICE_ACCOUNT, + "test-sa@project.iam.gserviceaccount.com"); + + GCPProperties gcpProperties = new GCPProperties(properties); + + assertThat(gcpProperties.impersonateServiceAccount()) + .contains("test-sa@project.iam.gserviceaccount.com"); + assertThat(gcpProperties.impersonateDelegates()).isNull(); + assertThat(gcpProperties.impersonateLifetimeSeconds()) + .isEqualTo(GCPProperties.GCS_IMPERSONATE_LIFETIME_SECONDS_DEFAULT); + } + @Test public void gcsFileSystem() { Map properties = From 0fb1e3cac5fcae43b3b2f7c77ff26457f5d715dd Mon Sep 17 00:00:00 2001 From: Christian Date: Fri, 19 Dec 2025 22:44:29 +0000 Subject: [PATCH 116/201] OpenAPI: Etag for CommitTableResponse (#14760) --- open-api/rest-catalog-open-api.yaml | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 4d009071f2e2..d322b0c7c7c0 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -963,8 +963,8 @@ paths: in: header description: An optional header that allows the server to return 304 (Not Modified) if the metadata - is current. The content is the value of the ETag received in a CreateTableResponse or - LoadTableResponse. + is current. The content is the value of the ETag received in a CreateTableResponse, + LoadTableResponse or CommitTableResponse. required: false schema: type: string @@ -4858,6 +4858,9 @@ components: application/json: schema: $ref: '#/components/schemas/CommitTableResponse' + headers: + etag: + $ref: '#/components/parameters/etag' LoadCredentialsResponse: description: Table credentials result when loading credentials for a table From 76fcd47ff00254b7079224a5960a49d906343cdf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 20 Dec 2025 21:55:32 -0800 Subject: [PATCH 117/201] Build: Bump io.netty:netty-buffer from 4.2.8.Final to 4.2.9.Final (#14897) Bumps [io.netty:netty-buffer](https://github.com/netty/netty) from 4.2.8.Final to 4.2.9.Final. - [Commits](https://github.com/netty/netty/compare/netty-4.2.8.Final...netty-4.2.9.Final) --- updated-dependencies: - dependency-name: io.netty:netty-buffer dependency-version: 4.2.9.Final dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index fea5bdb93afe..1c74b92d039b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -76,7 +76,7 @@ microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.106.0" -netty-buffer = "4.2.8.Final" +netty-buffer = "4.2.9.Final" object-client-bundle = "3.3.2" orc = "1.9.7" parquet = "1.16.0" From da67268ee4d83fabcbdc8ab4ce9b790772efab8c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 20 Dec 2025 21:55:52 -0800 Subject: [PATCH 118/201] Build: Bump testcontainers from 2.0.2 to 2.0.3 (#14898) Bumps `testcontainers` from 2.0.2 to 2.0.3. Updates `org.testcontainers:testcontainers` from 2.0.2 to 2.0.3 - [Release notes](https://github.com/testcontainers/testcontainers-java/releases) - [Changelog](https://github.com/testcontainers/testcontainers-java/blob/main/CHANGELOG.md) - [Commits](https://github.com/testcontainers/testcontainers-java/compare/2.0.2...2.0.3) Updates `org.testcontainers:testcontainers-junit-jupiter` from 2.0.2 to 2.0.3 - [Release notes](https://github.com/testcontainers/testcontainers-java/releases) - [Changelog](https://github.com/testcontainers/testcontainers-java/blob/main/CHANGELOG.md) - [Commits](https://github.com/testcontainers/testcontainers-java/compare/2.0.2...2.0.3) Updates `org.testcontainers:testcontainers-minio` from 2.0.2 to 2.0.3 - [Release notes](https://github.com/testcontainers/testcontainers-java/releases) - [Changelog](https://github.com/testcontainers/testcontainers-java/blob/main/CHANGELOG.md) - [Commits](https://github.com/testcontainers/testcontainers-java/compare/2.0.2...2.0.3) --- updated-dependencies: - dependency-name: org.testcontainers:testcontainers dependency-version: 2.0.3 dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.testcontainers:testcontainers-junit-jupiter dependency-version: 2.0.3 dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.testcontainers:testcontainers-minio dependency-version: 2.0.3 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 1c74b92d039b..8b173b5f06ff 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -88,7 +88,7 @@ spark34 = "3.4.4" spark35 = "3.5.7" spark40 = "4.0.1" sqlite-jdbc = "3.51.1.0" -testcontainers = "2.0.2" +testcontainers = "2.0.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above [libraries] From 73c2aa2d1cce7a563c32c2e668bd7234cf5bfd55 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 20 Dec 2025 21:55:59 -0800 Subject: [PATCH 119/201] Build: Bump software.amazon.awssdk:bom from 2.40.8 to 2.40.13 (#14904) Bumps software.amazon.awssdk:bom from 2.40.8 to 2.40.13. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-version: 2.40.13 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 8b173b5f06ff..7529aa51694e 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -31,7 +31,7 @@ arrow = "15.0.2" avro = "1.12.1" assertj-core = "3.27.6" awaitility = "4.3.0" -awssdk-bom = "2.40.8" +awssdk-bom = "2.40.13" azuresdk-bom = "1.3.3" awssdk-s3accessgrants = "2.3.0" bson-ver = "4.11.5" From 830cbc9b6840d233889a64d09260913a315f9464 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 20 Dec 2025 21:56:14 -0800 Subject: [PATCH 120/201] Build: Bump net.snowflake:snowflake-jdbc from 3.27.1 to 3.28.0 (#14899) Bumps [net.snowflake:snowflake-jdbc](https://github.com/snowflakedb/snowflake-jdbc) from 3.27.1 to 3.28.0. - [Release notes](https://github.com/snowflakedb/snowflake-jdbc/releases) - [Changelog](https://github.com/snowflakedb/snowflake-jdbc/blob/master/CHANGELOG.md) - [Commits](https://github.com/snowflakedb/snowflake-jdbc/compare/v3.27.1...v3.28.0) --- updated-dependencies: - dependency-name: net.snowflake:snowflake-jdbc dependency-version: 3.28.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 7529aa51694e..a524f9fb5dbb 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -83,7 +83,7 @@ parquet = "1.16.0" roaringbitmap = "1.3.0" scala-collection-compat = "2.14.0" slf4j = "2.0.17" -snowflake-jdbc = "3.27.1" +snowflake-jdbc = "3.28.0" spark34 = "3.4.4" spark35 = "3.5.7" spark40 = "4.0.1" From e8f6e90f67e2cf25e4bc330fde781fad6d2b6138 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 20 Dec 2025 21:56:29 -0800 Subject: [PATCH 121/201] Build: Bump com.google.cloud:libraries-bom from 26.72.0 to 26.73.0 (#14902) Bumps [com.google.cloud:libraries-bom](https://github.com/googleapis/java-cloud-bom) from 26.72.0 to 26.73.0. - [Release notes](https://github.com/googleapis/java-cloud-bom/releases) - [Changelog](https://github.com/googleapis/java-cloud-bom/blob/main/release-please-config.json) - [Commits](https://github.com/googleapis/java-cloud-bom/compare/v26.72.0...v26.73.0) --- updated-dependencies: - dependency-name: com.google.cloud:libraries-bom dependency-version: 26.73.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index a524f9fb5dbb..107cfdcddb6f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -49,7 +49,7 @@ findbugs-jsr305 = "3.0.2" flink120 = { strictly = "1.20.1"} flink20 = { strictly = "2.0.0"} flink21 = { strictly = "2.1.0"} -google-libraries-bom = "26.72.0" +google-libraries-bom = "26.73.0" gcs-analytics-core = "1.2.3" guava = "33.5.0-jre" hadoop3 = "3.4.2" From d6d44a7803b218e7da6d514e96ad98282f22a2b6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 20 Dec 2025 21:56:57 -0800 Subject: [PATCH 122/201] Build: Bump org.apache.httpcomponents.client5:httpclient5 (#14900) Bumps [org.apache.httpcomponents.client5:httpclient5](https://github.com/apache/httpcomponents-client) from 5.5.1 to 5.6. - [Changelog](https://github.com/apache/httpcomponents-client/blob/master/RELEASE_NOTES.txt) - [Commits](https://github.com/apache/httpcomponents-client/compare/rel/v5.5.1...rel/v5.6) --- updated-dependencies: - dependency-name: org.apache.httpcomponents.client5:httpclient5 dependency-version: '5.6' dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 107cfdcddb6f..5e2d5435eb14 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -53,7 +53,7 @@ google-libraries-bom = "26.73.0" gcs-analytics-core = "1.2.3" guava = "33.5.0-jre" hadoop3 = "3.4.2" -httpcomponents-httpclient5 = "5.5.1" +httpcomponents-httpclient5 = "5.6" hive2 = { strictly = "2.3.10"} # see rich version usage explanation above immutables-value = "2.12.0" jackson-annotations = "2.20" From 1c5bb017c2161aa568d75e17a4de7f42bcb37d02 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Tue, 23 Dec 2025 01:03:51 +0900 Subject: [PATCH 123/201] Build: Bump datamodel-code-generator from 0.43.1 to 0.46.0 (#14905) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.43.1 to 0.46.0. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Changelog](https://github.com/koxudaxi/datamodel-code-generator/blob/main/CHANGELOG.md) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.43.1...0.46.0) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-version: 0.46.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- open-api/rest-catalog-open-api.py | 508 +++++++++++++++--------------- 2 files changed, 251 insertions(+), 259 deletions(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index cee1d4c6aa00..51d78c1ed46f 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,5 +16,5 @@ # under the License. openapi-spec-validator==0.7.2 -datamodel-code-generator==0.43.1 +datamodel-code-generator==0.46.0 yamllint==1.37.1 diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index fd8667e0f059..c39416388e7e 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -18,7 +18,7 @@ from __future__ import annotations from datetime import date, timedelta -from typing import Dict, List, Literal, Optional, Union +from typing import Literal from uuid import UUID from pydantic import BaseModel, Extra, Field @@ -38,7 +38,7 @@ class ErrorModel(BaseModel): code: int = Field( ..., description='HTTP response code', example=404, ge=400, le=600 ) - stack: Optional[List[str]] = None + stack: list[str] | None = None class CatalogConfig(BaseModel): @@ -46,15 +46,15 @@ class CatalogConfig(BaseModel): Server-provided configuration for the catalog. """ - overrides: Dict[str, str] = Field( + overrides: dict[str, str] = Field( ..., description='Properties that should be used to override client configuration; applied after defaults and client configuration.', ) - defaults: Dict[str, str] = Field( + defaults: dict[str, str] = Field( ..., description='Properties that should be used as default configuration; applied before client configuration.', ) - endpoints: Optional[List[str]] = Field( + endpoints: list[str] | None = Field( None, description='A list of endpoints that the server supports. The format of each endpoint must be " ". The HTTP verb and the resource path must be separated by a space character.', example=[ @@ -65,7 +65,7 @@ class CatalogConfig(BaseModel): 'GET /v1/{prefix}/namespaces/{namespace}/views/{view}', ], ) - idempotency_key_lifetime: Optional[timedelta] = Field( + idempotency_key_lifetime: timedelta | None = Field( None, alias='idempotency-key-lifetime', description='Client reuse window for an Idempotency-Key (ISO-8601 duration, e.g., PT30M, PT24H). Interpreted as the maximum time from the first submission using a key to the last retry during which a client may reuse that key. Servers SHOULD accept retries for at least this duration and MAY include a grace period to account for delays/clock skew. Clients SHOULD NOT reuse an Idempotency-Key after this window elapses; they SHOULD generate a new key for any subsequent attempt. Presence of this field indicates the server supports Idempotency-Key semantics for mutation endpoints. If absent, clients MUST assume idempotency is not supported.', @@ -74,12 +74,10 @@ class CatalogConfig(BaseModel): class UpdateNamespacePropertiesRequest(BaseModel): - removals: Optional[List[str]] = Field( + removals: list[str] | None = Field( None, example=['department', 'access_group'], unique_items=True ) - updates: Optional[Dict[str, str]] = Field( - None, example={'owner': 'Hank Bendickson'} - ) + updates: dict[str, str] | None = Field(None, example={'owner': 'Hank Bendickson'}) class Namespace(BaseModel): @@ -87,7 +85,7 @@ class Namespace(BaseModel): Reference to one or more levels of a namespace """ - __root__: List[str] = Field( + __root__: list[str] = Field( ..., description='Reference to one or more levels of a namespace', example=['accounting', 'tax'], @@ -95,7 +93,7 @@ class Namespace(BaseModel): class PageToken(BaseModel): - __root__: Optional[str] = Field( + __root__: str | None = Field( None, description='An opaque token that allows clients to make use of pagination for list APIs (e.g. ListTables). Clients may initiate the first paginated request by sending an empty query parameter `pageToken` to the server.\nServers that support pagination should identify the `pageToken` parameter and return a `next-page-token` in the response if there are more results available. After the initial request, the value of `next-page-token` from each response must be used as the `pageToken` parameter value for the next request. The server must return `null` value for the `next-page-token` in the last response.\nServers that support pagination must return all results in a single response with the value of `next-page-token` set to `null` if the query parameter `pageToken` is not set in the request.\nServers that do not support pagination should ignore the `pageToken` parameter and return all results in a single response. The `next-page-token` must be omitted from the response.\nClients must interpret either `null` or missing response value of `next-page-token` as the end of the listing results.', ) @@ -165,15 +163,15 @@ class Transform(BaseModel): class PartitionField(BaseModel): - field_id: Optional[int] = Field(None, alias='field-id') + field_id: int | None = Field(None, alias='field-id') source_id: int = Field(..., alias='source-id') name: str transform: Transform class PartitionSpec(BaseModel): - spec_id: Optional[int] = Field(None, alias='spec-id') - fields: List[PartitionField] + spec_id: int | None = Field(None, alias='spec-id') + fields: list[PartitionField] class SortDirection(BaseModel): @@ -193,14 +191,14 @@ class SortField(BaseModel): class SortOrder(BaseModel): order_id: int = Field(..., alias='order-id') - fields: List[SortField] + fields: list[SortField] class EncryptedKey(BaseModel): key_id: str = Field(..., alias='key-id') encrypted_key_metadata: str = Field(..., alias='encrypted-key-metadata') - encrypted_by_id: Optional[str] = Field(None, alias='encrypted-by-id') - properties: Optional[Dict[str, str]] = None + encrypted_by_id: str | None = Field(None, alias='encrypted-by-id') + properties: dict[str, str] | None = None class Summary(BaseModel): @@ -209,38 +207,38 @@ class Summary(BaseModel): class Snapshot(BaseModel): snapshot_id: int = Field(..., alias='snapshot-id') - parent_snapshot_id: Optional[int] = Field(None, alias='parent-snapshot-id') - sequence_number: Optional[int] = Field(None, alias='sequence-number') + parent_snapshot_id: int | None = Field(None, alias='parent-snapshot-id') + sequence_number: int | None = Field(None, alias='sequence-number') timestamp_ms: int = Field(..., alias='timestamp-ms') manifest_list: str = Field( ..., alias='manifest-list', description="Location of the snapshot's manifest list file", ) - first_row_id: Optional[int] = Field( + first_row_id: int | None = Field( None, alias='first-row-id', description='The first _row_id assigned to the first row in the first data file in the first manifest', ) - added_rows: Optional[int] = Field( + added_rows: int | None = Field( None, alias='added-rows', description='The upper bound of the number of rows with assigned row IDs', ) summary: Summary - schema_id: Optional[int] = Field(None, alias='schema-id') + schema_id: int | None = Field(None, alias='schema-id') class SnapshotReference(BaseModel): type: Literal['tag', 'branch'] snapshot_id: int = Field(..., alias='snapshot-id') - max_ref_age_ms: Optional[int] = Field(None, alias='max-ref-age-ms') - max_snapshot_age_ms: Optional[int] = Field(None, alias='max-snapshot-age-ms') - min_snapshots_to_keep: Optional[int] = Field(None, alias='min-snapshots-to-keep') + max_ref_age_ms: int | None = Field(None, alias='max-ref-age-ms') + max_snapshot_age_ms: int | None = Field(None, alias='max-snapshot-age-ms') + min_snapshots_to_keep: int | None = Field(None, alias='min-snapshots-to-keep') class SnapshotReferences(BaseModel): - __root__: Dict[str, SnapshotReference] + __root__: dict[str, SnapshotReference] class SnapshotLogItem(BaseModel): @@ -249,7 +247,7 @@ class SnapshotLogItem(BaseModel): class SnapshotLog(BaseModel): - __root__: List[SnapshotLogItem] + __root__: list[SnapshotLogItem] class MetadataLogItem(BaseModel): @@ -258,7 +256,7 @@ class MetadataLogItem(BaseModel): class MetadataLog(BaseModel): - __root__: List[MetadataLogItem] + __root__: list[MetadataLogItem] class SQLViewRepresentation(BaseModel): @@ -284,9 +282,9 @@ class ViewVersion(BaseModel): alias='schema-id', description='Schema ID to set as current, or -1 to set last added schema', ) - summary: Dict[str, str] - representations: List[ViewRepresentation] - default_catalog: Optional[str] = Field(None, alias='default-catalog') + summary: dict[str, str] + representations: list[ViewRepresentation] + default_catalog: str | None = Field(None, alias='default-catalog') default_namespace: Namespace = Field(..., alias='default-namespace') @@ -355,14 +353,14 @@ class SetSnapshotRefUpdate(BaseModel): ref_name: str = Field(..., alias='ref-name') type: Literal['tag', 'branch'] snapshot_id: int = Field(..., alias='snapshot-id') - max_ref_age_ms: Optional[int] = Field(None, alias='max-ref-age-ms') - max_snapshot_age_ms: Optional[int] = Field(None, alias='max-snapshot-age-ms') - min_snapshots_to_keep: Optional[int] = Field(None, alias='min-snapshots-to-keep') + max_ref_age_ms: int | None = Field(None, alias='max-ref-age-ms') + max_snapshot_age_ms: int | None = Field(None, alias='max-snapshot-age-ms') + min_snapshots_to_keep: int | None = Field(None, alias='min-snapshots-to-keep') class RemoveSnapshotsUpdate(BaseUpdate): action: str = Field('remove-snapshots', const=True) - snapshot_ids: List[int] = Field(..., alias='snapshot-ids') + snapshot_ids: list[int] = Field(..., alias='snapshot-ids') class RemoveSnapshotRefUpdate(BaseUpdate): @@ -377,12 +375,12 @@ class SetLocationUpdate(BaseUpdate): class SetPropertiesUpdate(BaseUpdate): action: str = Field('set-properties', const=True) - updates: Dict[str, str] + updates: dict[str, str] class RemovePropertiesUpdate(BaseUpdate): action: str = Field('remove-properties', const=True) - removals: List[str] + removals: list[str] class AddViewVersionUpdate(BaseUpdate): @@ -411,12 +409,12 @@ class RemovePartitionStatisticsUpdate(BaseUpdate): class RemovePartitionSpecsUpdate(BaseUpdate): action: str = Field('remove-partition-specs', const=True) - spec_ids: List[int] = Field(..., alias='spec-ids') + spec_ids: list[int] = Field(..., alias='spec-ids') class RemoveSchemasUpdate(BaseUpdate): action: str = Field('remove-schemas', const=True) - schema_ids: List[int] = Field(..., alias='schema-ids') + schema_ids: list[int] = Field(..., alias='schema-ids') class AddEncryptionKeyUpdate(BaseUpdate): @@ -522,11 +520,11 @@ class StorageCredential(BaseModel): ..., description='Indicates a storage location prefix where the credential is relevant. Clients should choose the most specific prefix (by selecting the longest prefix) if several credentials of the same type are available.', ) - config: Dict[str, str] + config: dict[str, str] class LoadCredentialsResponse(BaseModel): - storage_credentials: List[StorageCredential] = Field( + storage_credentials: list[StorageCredential] = Field( ..., alias='storage-credentials' ) @@ -555,7 +553,7 @@ class PlanStatus(BaseModel): class RegisterTableRequest(BaseModel): name: str metadata_location: str = Field(..., alias='metadata-location') - overwrite: Optional[bool] = Field( + overwrite: bool | None = Field( False, description='Whether to overwrite table metadata if the table already exists', ) @@ -585,7 +583,7 @@ class OAuthClientCredentialsRequest(BaseModel): """ grant_type: Literal['client_credentials'] - scope: Optional[str] = None + scope: str | None = None client_id: str = Field( ..., description='Client ID\n\nThis can be sent in the request body, but OAuth2 recommends sending it in a Basic Authorization header.', @@ -606,20 +604,20 @@ class OAuthTokenExchangeRequest(BaseModel): """ grant_type: Literal['urn:ietf:params:oauth:grant-type:token-exchange'] - scope: Optional[str] = None - requested_token_type: Optional[TokenType] = None + scope: str | None = None + requested_token_type: TokenType | None = None subject_token: str = Field( ..., description='Subject token for token exchange request' ) subject_token_type: TokenType - actor_token: Optional[str] = Field( + actor_token: str | None = Field( None, description='Actor token for token exchange request' ) - actor_token_type: Optional[TokenType] = None + actor_token_type: TokenType | None = None class OAuthTokenRequest(BaseModel): - __root__: Union[OAuthClientCredentialsRequest, OAuthTokenExchangeRequest] = Field( + __root__: OAuthClientCredentialsRequest | OAuthTokenExchangeRequest = Field( ..., description='The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this spec, see description of the endpoint.', ) @@ -637,11 +635,11 @@ class TimerResult(BaseModel): class MetricResult(BaseModel): - __root__: Union[CounterResult, TimerResult] + __root__: CounterResult | TimerResult class Metrics(BaseModel): - __root__: Dict[str, MetricResult] + __root__: dict[str, MetricResult] class CommitReport(BaseModel): @@ -650,7 +648,7 @@ class CommitReport(BaseModel): sequence_number: int = Field(..., alias='sequence-number') operation: str metrics: Metrics - metadata: Optional[Dict[str, str]] = None + metadata: dict[str, str] | None = None class OAuthError(BaseModel): @@ -666,8 +664,8 @@ class OAuthError(BaseModel): 'unsupported_grant_type', 'invalid_scope', ] - error_description: Optional[str] = None - error_uri: Optional[str] = None + error_description: str | None = None + error_uri: str | None = None class OAuthTokenResponse(BaseModel): @@ -682,15 +680,15 @@ class OAuthTokenResponse(BaseModel): ..., description='Access token type for client credentials or token exchange\n\nSee https://datatracker.ietf.org/doc/html/rfc6749#section-7.1', ) - expires_in: Optional[int] = Field( + expires_in: int | None = Field( None, description='Lifetime of the access token in seconds for client credentials or token exchange', ) - issued_token_type: Optional[TokenType] = None - refresh_token: Optional[str] = Field( + issued_token_type: TokenType | None = None + refresh_token: str | None = Field( None, description='Refresh token for client credentials or token exchange' ) - scope: Optional[str] = Field( + scope: str | None = Field( None, description='Authorization scope for client credentials or token exchange' ) @@ -708,7 +706,7 @@ class Config: class CreateNamespaceResponse(BaseModel): namespace: Namespace - properties: Optional[Dict[str, str]] = Field( + properties: dict[str, str] | None = Field( {}, description='Properties stored on the namespace, if supported by the server.', example={'owner': 'Ralph', 'created_at': '1452120468'}, @@ -717,7 +715,7 @@ class CreateNamespaceResponse(BaseModel): class GetNamespaceResponse(BaseModel): namespace: Namespace - properties: Optional[Dict[str, str]] = Field( + properties: dict[str, str] | None = Field( {}, description='Properties stored on the namespace, if supported by the server. If the server does not support namespace properties, it should return null for this field. If namespace properties are supported, but none are set, it should return an empty object.', example={'owner': 'Ralph', 'transient_lastDdlTime': '1452120468'}, @@ -725,23 +723,23 @@ class GetNamespaceResponse(BaseModel): class ListTablesResponse(BaseModel): - next_page_token: Optional[PageToken] = Field(None, alias='next-page-token') - identifiers: Optional[List[TableIdentifier]] = Field(None, unique_items=True) + next_page_token: PageToken | None = Field(None, alias='next-page-token') + identifiers: list[TableIdentifier] | None = Field(None, unique_items=True) class ListNamespacesResponse(BaseModel): - next_page_token: Optional[PageToken] = Field(None, alias='next-page-token') - namespaces: Optional[List[Namespace]] = Field(None, unique_items=True) + next_page_token: PageToken | None = Field(None, alias='next-page-token') + namespaces: list[Namespace] | None = Field(None, unique_items=True) class UpdateNamespacePropertiesResponse(BaseModel): - updated: List[str] = Field( + updated: list[str] = Field( ..., description='List of property keys that were added or updated', unique_items=True, ) - removed: List[str] = Field(..., description='List of properties that were removed') - missing: Optional[List[str]] = Field( + removed: list[str] = Field(..., description='List of properties that were removed') + missing: list[str] | None = Field( None, description="List of properties requested for removal that were not found in the namespace's properties. Represents a partial success response. Server's do not need to implement this.", ) @@ -751,8 +749,8 @@ class BlobMetadata(BaseModel): type: str snapshot_id: int = Field(..., alias='snapshot-id') sequence_number: int = Field(..., alias='sequence-number') - fields: List[int] - properties: Optional[Dict[str, str]] = None + fields: list[int] + properties: dict[str, str] | None = None class PartitionStatisticsFile(BaseModel): @@ -869,33 +867,33 @@ class BinaryTypeValue(BaseModel): class CountMap(BaseModel): - keys: Optional[List[IntegerTypeValue]] = Field( + keys: list[IntegerTypeValue] | None = Field( None, description='List of integer column ids for each corresponding value' ) - values: Optional[List[LongTypeValue]] = Field( + values: list[LongTypeValue] | None = Field( None, description="List of Long values, matched to 'keys' by index" ) class PrimitiveTypeValue(BaseModel): - __root__: Union[ - BooleanTypeValue, - IntegerTypeValue, - LongTypeValue, - FloatTypeValue, - DoubleTypeValue, - DecimalTypeValue, - StringTypeValue, - UUIDTypeValue, - DateTypeValue, - TimeTypeValue, - TimestampTypeValue, - TimestampTzTypeValue, - TimestampNanoTypeValue, - TimestampTzNanoTypeValue, - FixedTypeValue, - BinaryTypeValue, - ] + __root__: ( + BooleanTypeValue + | IntegerTypeValue + | LongTypeValue + | FloatTypeValue + | DoubleTypeValue + | DecimalTypeValue + | StringTypeValue + | UUIDTypeValue + | DateTypeValue + | TimeTypeValue + | TimestampTypeValue + | TimestampTzTypeValue + | TimestampNanoTypeValue + | TimestampTzNanoTypeValue + | FixedTypeValue + | BinaryTypeValue + ) class FileFormat(BaseModel): @@ -907,7 +905,7 @@ class ContentFile(BaseModel): file_path: str = Field(..., alias='file-path') file_format: FileFormat = Field(..., alias='file-format') spec_id: int = Field(..., alias='spec-id') - partition: List[PrimitiveTypeValue] = Field( + partition: list[PrimitiveTypeValue] = Field( ..., description='A list of partition field values ordered based on the fields of the partition spec specified by the `spec-id`', example=[1, 'bar'], @@ -918,23 +916,23 @@ class ContentFile(BaseModel): record_count: int = Field( ..., alias='record-count', description='Number of records in the file' ) - key_metadata: Optional[BinaryTypeValue] = Field( + key_metadata: BinaryTypeValue | None = Field( None, alias='key-metadata', description='Encryption key metadata blob' ) - split_offsets: Optional[List[int]] = Field( + split_offsets: list[int] | None = Field( None, alias='split-offsets', description='List of splittable offsets' ) - sort_order_id: Optional[int] = Field(None, alias='sort-order-id') + sort_order_id: int | None = Field(None, alias='sort-order-id') class PositionDeleteFile(ContentFile): content: Literal['position-deletes'] = Field(..., const=True) - content_offset: Optional[int] = Field( + content_offset: int | None = Field( None, alias='content-offset', description='Offset within the delete file of delete content', ) - content_size_in_bytes: Optional[int] = Field( + content_size_in_bytes: int | None = Field( None, alias='content-size-in-bytes', description='Length, in bytes, of the delete content; required if content-offset is present', @@ -943,7 +941,7 @@ class PositionDeleteFile(ContentFile): class EqualityDeleteFile(ContentFile): content: Literal['equality-deletes'] = Field(..., const=True) - equality_ids: Optional[List[int]] = Field( + equality_ids: list[int] | None = Field( None, alias='equality-ids', description='List of equality field IDs' ) @@ -985,7 +983,7 @@ class ResidualFilter3(FalseExpression, ResidualFilter1): class CreateNamespaceRequest(BaseModel): namespace: Namespace - properties: Optional[Dict[str, str]] = Field( + properties: dict[str, str] | None = Field( {}, description='Configured string to string map of properties for the namespace', example={'owner': 'Hank Bendickson'}, @@ -1031,49 +1029,49 @@ class StatisticsFile(BaseModel): statistics_path: str = Field(..., alias='statistics-path') file_size_in_bytes: int = Field(..., alias='file-size-in-bytes') file_footer_size_in_bytes: int = Field(..., alias='file-footer-size-in-bytes') - blob_metadata: List[BlobMetadata] = Field(..., alias='blob-metadata') + blob_metadata: list[BlobMetadata] = Field(..., alias='blob-metadata') class ValueMap(BaseModel): - keys: Optional[List[IntegerTypeValue]] = Field( + keys: list[IntegerTypeValue] | None = Field( None, description='List of integer column ids for each corresponding value' ) - values: Optional[List[PrimitiveTypeValue]] = Field( + values: list[PrimitiveTypeValue] | None = Field( None, description="List of primitive type values, matched to 'keys' by index" ) class DataFile(ContentFile): content: str = Field(..., const=True) - first_row_id: Optional[int] = Field( + first_row_id: int | None = Field( None, alias='first-row-id', description='The first row ID assigned to the first row in the data file', ) - column_sizes: Optional[CountMap] = Field( + column_sizes: CountMap | None = Field( None, alias='column-sizes', description='Map of column id to total count, including null and NaN', ) - value_counts: Optional[CountMap] = Field( + value_counts: CountMap | None = Field( None, alias='value-counts', description='Map of column id to null value count' ) - null_value_counts: Optional[CountMap] = Field( + null_value_counts: CountMap | None = Field( None, alias='null-value-counts', description='Map of column id to null value count', ) - nan_value_counts: Optional[CountMap] = Field( + nan_value_counts: CountMap | None = Field( None, alias='nan-value-counts', description='Map of column id to number of NaN values in the column', ) - lower_bounds: Optional[ValueMap] = Field( + lower_bounds: ValueMap | None = Field( None, alias='lower-bounds', description='Map of column id to lower bound primitive type values', ) - upper_bounds: Optional[ValueMap] = Field( + upper_bounds: ValueMap | None = Field( None, alias='upper-bounds', description='Map of column id to upper bound primitive type values', @@ -1081,7 +1079,7 @@ class DataFile(ContentFile): class DeleteFile(BaseModel): - __root__: Union[PositionDeleteFile, EqualityDeleteFile] = Field( + __root__: PositionDeleteFile | EqualityDeleteFile = Field( ..., discriminator='content' ) @@ -1091,12 +1089,12 @@ class FetchScanTasksRequest(BaseModel): class Term(BaseModel): - __root__: Union[Reference, TransformTerm] + __root__: Reference | TransformTerm class SetStatisticsUpdate(BaseUpdate): action: str = Field('set-statistics', const=True) - snapshot_id: Optional[int] = Field( + snapshot_id: int | None = Field( None, alias='snapshot-id', description='This optional field is **DEPRECATED for REMOVAL** since it contains redundant information. Clients should use the `statistics.snapshot-id` field instead.', @@ -1120,7 +1118,7 @@ class LiteralExpression(BaseModel): class SetExpression(BaseModel): type: Literal['in', 'not-in'] term: Term - values: List[PrimitiveTypeValue] + values: list[PrimitiveTypeValue] class ResidualFilter6(SetExpression, ResidualFilter1): @@ -1149,14 +1147,14 @@ class StructField(BaseModel): name: str type: Type required: bool - doc: Optional[str] = None - initial_default: Optional[PrimitiveTypeValue] = Field(None, alias='initial-default') - write_default: Optional[PrimitiveTypeValue] = Field(None, alias='write-default') + doc: str | None = None + initial_default: PrimitiveTypeValue | None = Field(None, alias='initial-default') + write_default: PrimitiveTypeValue | None = Field(None, alias='write-default') class StructType(BaseModel): type: str = Field('struct', const=True) - fields: List[StructField] + fields: list[StructField] class ListType(BaseModel): @@ -1176,19 +1174,19 @@ class MapType(BaseModel): class Type(BaseModel): - __root__: Union[PrimitiveType, StructType, ListType, MapType] + __root__: PrimitiveType | StructType | ListType | MapType class Expression(BaseModel): - __root__: Union[ - TrueExpression, - FalseExpression, - AndOrExpression, - NotExpression, - SetExpression, - LiteralExpression, - UnaryExpression, - ] + __root__: ( + TrueExpression + | FalseExpression + | AndOrExpression + | NotExpression + | SetExpression + | LiteralExpression + | UnaryExpression + ) class AndOrExpression(BaseModel): @@ -1205,33 +1203,31 @@ class NotExpression(BaseModel): class TableMetadata(BaseModel): format_version: int = Field(..., alias='format-version', ge=1, le=3) table_uuid: str = Field(..., alias='table-uuid') - location: Optional[str] = None - last_updated_ms: Optional[int] = Field(None, alias='last-updated-ms') - next_row_id: Optional[int] = Field( + location: str | None = None + last_updated_ms: int | None = Field(None, alias='last-updated-ms') + next_row_id: int | None = Field( None, alias='next-row-id', description="A long higher than all assigned row IDs; the next snapshot's first-row-id.", ) - properties: Optional[Dict[str, str]] = None - schemas: Optional[List[Schema]] = None - current_schema_id: Optional[int] = Field(None, alias='current-schema-id') - last_column_id: Optional[int] = Field(None, alias='last-column-id') - partition_specs: Optional[List[PartitionSpec]] = Field( - None, alias='partition-specs' - ) - default_spec_id: Optional[int] = Field(None, alias='default-spec-id') - last_partition_id: Optional[int] = Field(None, alias='last-partition-id') - sort_orders: Optional[List[SortOrder]] = Field(None, alias='sort-orders') - default_sort_order_id: Optional[int] = Field(None, alias='default-sort-order-id') - encryption_keys: Optional[List[EncryptedKey]] = Field(None, alias='encryption-keys') - snapshots: Optional[List[Snapshot]] = None - refs: Optional[SnapshotReferences] = None - current_snapshot_id: Optional[int] = Field(None, alias='current-snapshot-id') - last_sequence_number: Optional[int] = Field(None, alias='last-sequence-number') - snapshot_log: Optional[SnapshotLog] = Field(None, alias='snapshot-log') - metadata_log: Optional[MetadataLog] = Field(None, alias='metadata-log') - statistics: Optional[List[StatisticsFile]] = None - partition_statistics: Optional[List[PartitionStatisticsFile]] = Field( + properties: dict[str, str] | None = None + schemas: list[Schema] | None = None + current_schema_id: int | None = Field(None, alias='current-schema-id') + last_column_id: int | None = Field(None, alias='last-column-id') + partition_specs: list[PartitionSpec] | None = Field(None, alias='partition-specs') + default_spec_id: int | None = Field(None, alias='default-spec-id') + last_partition_id: int | None = Field(None, alias='last-partition-id') + sort_orders: list[SortOrder] | None = Field(None, alias='sort-orders') + default_sort_order_id: int | None = Field(None, alias='default-sort-order-id') + encryption_keys: list[EncryptedKey] | None = Field(None, alias='encryption-keys') + snapshots: list[Snapshot] | None = None + refs: SnapshotReferences | None = None + current_snapshot_id: int | None = Field(None, alias='current-snapshot-id') + last_sequence_number: int | None = Field(None, alias='last-sequence-number') + snapshot_log: SnapshotLog | None = Field(None, alias='snapshot-log') + metadata_log: MetadataLog | None = Field(None, alias='metadata-log') + statistics: list[StatisticsFile] | None = None + partition_statistics: list[PartitionStatisticsFile] | None = Field( None, alias='partition-statistics' ) @@ -1241,16 +1237,16 @@ class ViewMetadata(BaseModel): format_version: int = Field(..., alias='format-version', ge=1, le=1) location: str current_version_id: int = Field(..., alias='current-version-id') - versions: List[ViewVersion] - version_log: List[ViewHistoryEntry] = Field(..., alias='version-log') - schemas: List[Schema] - properties: Optional[Dict[str, str]] = None + versions: list[ViewVersion] + version_log: list[ViewHistoryEntry] = Field(..., alias='version-log') + schemas: list[Schema] + properties: dict[str, str] | None = None class AddSchemaUpdate(BaseUpdate): action: str = Field('add-schema', const=True) schema_: Schema = Field(..., alias='schema') - last_column_id: Optional[int] = Field( + last_column_id: int | None = Field( None, alias='last-column-id', description="This optional field is **DEPRECATED for REMOVAL** since it more safe to handle this internally, and shouldn't be exposed to the clients.\nThe highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. When omitted, it will be computed on the server side.", @@ -1258,42 +1254,42 @@ class AddSchemaUpdate(BaseUpdate): class TableUpdate(BaseModel): - __root__: Union[ - AssignUUIDUpdate, - UpgradeFormatVersionUpdate, - AddSchemaUpdate, - SetCurrentSchemaUpdate, - AddPartitionSpecUpdate, - SetDefaultSpecUpdate, - AddSortOrderUpdate, - SetDefaultSortOrderUpdate, - AddSnapshotUpdate, - SetSnapshotRefUpdate, - RemoveSnapshotsUpdate, - RemoveSnapshotRefUpdate, - SetLocationUpdate, - SetPropertiesUpdate, - RemovePropertiesUpdate, - SetStatisticsUpdate, - RemoveStatisticsUpdate, - RemovePartitionSpecsUpdate, - RemoveSchemasUpdate, - AddEncryptionKeyUpdate, - RemoveEncryptionKeyUpdate, - ] + __root__: ( + AssignUUIDUpdate + | UpgradeFormatVersionUpdate + | AddSchemaUpdate + | SetCurrentSchemaUpdate + | AddPartitionSpecUpdate + | SetDefaultSpecUpdate + | AddSortOrderUpdate + | SetDefaultSortOrderUpdate + | AddSnapshotUpdate + | SetSnapshotRefUpdate + | RemoveSnapshotsUpdate + | RemoveSnapshotRefUpdate + | SetLocationUpdate + | SetPropertiesUpdate + | RemovePropertiesUpdate + | SetStatisticsUpdate + | RemoveStatisticsUpdate + | RemovePartitionSpecsUpdate + | RemoveSchemasUpdate + | AddEncryptionKeyUpdate + | RemoveEncryptionKeyUpdate + ) class ViewUpdate(BaseModel): - __root__: Union[ - AssignUUIDUpdate, - UpgradeFormatVersionUpdate, - AddSchemaUpdate, - SetLocationUpdate, - SetPropertiesUpdate, - RemovePropertiesUpdate, - AddViewVersionUpdate, - SetCurrentViewVersionUpdate, - ] + __root__: ( + AssignUUIDUpdate + | UpgradeFormatVersionUpdate + | AddSchemaUpdate + | SetLocationUpdate + | SetPropertiesUpdate + | RemovePropertiesUpdate + | AddViewVersionUpdate + | SetCurrentViewVersionUpdate + ) class LoadTableResult(BaseModel): @@ -1331,14 +1327,14 @@ class LoadTableResult(BaseModel): """ - metadata_location: Optional[str] = Field( + metadata_location: str | None = Field( None, alias='metadata-location', description='May be null if the table is staged as part of a transaction', ) metadata: TableMetadata - config: Optional[Dict[str, str]] = None - storage_credentials: Optional[List[StorageCredential]] = Field( + config: dict[str, str] | None = None + storage_credentials: list[StorageCredential] | None = Field( None, alias='storage-credentials' ) @@ -1357,32 +1353,32 @@ class ScanTasks(BaseModel): """ - delete_files: Optional[List[DeleteFile]] = Field( + delete_files: list[DeleteFile] | None = Field( None, alias='delete-files', description='Delete files referenced by file scan tasks', ) - file_scan_tasks: Optional[List[FileScanTask]] = Field(None, alias='file-scan-tasks') - plan_tasks: Optional[List[PlanTask]] = Field(None, alias='plan-tasks') + file_scan_tasks: list[FileScanTask] | None = Field(None, alias='file-scan-tasks') + plan_tasks: list[PlanTask] | None = Field(None, alias='plan-tasks') class FetchPlanningResult(BaseModel): - __root__: Union[ - CompletedPlanningResult, FailedPlanningResult, EmptyPlanningResult - ] = Field( - ..., - description='Result of server-side scan planning for fetchPlanningResult', - discriminator='status', + __root__: CompletedPlanningResult | FailedPlanningResult | EmptyPlanningResult = ( + Field( + ..., + description='Result of server-side scan planning for fetchPlanningResult', + discriminator='status', + ) ) class PlanTableScanResult(BaseModel): - __root__: Union[ - CompletedPlanningWithIDResult, - FailedPlanningResult, - AsyncPlanningResult, - EmptyPlanningResult, - ] = Field( + __root__: ( + CompletedPlanningWithIDResult + | FailedPlanningResult + | AsyncPlanningResult + | EmptyPlanningResult + ) = Field( ..., description='Result of server-side scan planning for planTableScan', discriminator='status', @@ -1390,57 +1386,55 @@ class PlanTableScanResult(BaseModel): class CommitTableRequest(BaseModel): - identifier: Optional[TableIdentifier] = Field( + identifier: TableIdentifier | None = Field( None, description='Table identifier to update; must be present for CommitTransactionRequest', ) - requirements: List[ - Union[ - AssertCreate, - AssertTableUUID, - AssertRefSnapshotId, - AssertLastAssignedFieldId, - AssertCurrentSchemaId, - AssertLastAssignedPartitionId, - AssertDefaultSpecId, - AssertDefaultSortOrderId, - ] + requirements: list[ + AssertCreate + | AssertTableUUID + | AssertRefSnapshotId + | AssertLastAssignedFieldId + | AssertCurrentSchemaId + | AssertLastAssignedPartitionId + | AssertDefaultSpecId + | AssertDefaultSortOrderId ] - updates: List[TableUpdate] + updates: list[TableUpdate] class CommitViewRequest(BaseModel): - identifier: Optional[TableIdentifier] = Field( + identifier: TableIdentifier | None = Field( None, description='View identifier to update' ) - requirements: Optional[List[ViewRequirement]] = None - updates: List[ViewUpdate] + requirements: list[ViewRequirement] | None = None + updates: list[ViewUpdate] class CommitTransactionRequest(BaseModel): - table_changes: List[CommitTableRequest] = Field(..., alias='table-changes') + table_changes: list[CommitTableRequest] = Field(..., alias='table-changes') class CreateTableRequest(BaseModel): name: str - location: Optional[str] = None + location: str | None = None schema_: Schema = Field(..., alias='schema') - partition_spec: Optional[PartitionSpec] = Field(None, alias='partition-spec') - write_order: Optional[SortOrder] = Field(None, alias='write-order') - stage_create: Optional[bool] = Field(None, alias='stage-create') - properties: Optional[Dict[str, str]] = None + partition_spec: PartitionSpec | None = Field(None, alias='partition-spec') + write_order: SortOrder | None = Field(None, alias='write-order') + stage_create: bool | None = Field(None, alias='stage-create') + properties: dict[str, str] | None = None class CreateViewRequest(BaseModel): name: str - location: Optional[str] = None + location: str | None = None schema_: Schema = Field(..., alias='schema') view_version: ViewVersion = Field( ..., alias='view-version', description='The view version to create, will replace the schema-id sent within the view-version with the id assigned to the provided schema', ) - properties: Dict[str, str] + properties: dict[str, str] class LoadViewResult(BaseModel): @@ -1463,11 +1457,11 @@ class LoadViewResult(BaseModel): metadata_location: str = Field(..., alias='metadata-location') metadata: ViewMetadata - config: Optional[Dict[str, str]] = None + config: dict[str, str] | None = None class ReportMetricsRequest(BaseModel): - __root__: Union[ReportMetricsRequest1, ReportMetricsRequest2] + __root__: ReportMetricsRequest1 | ReportMetricsRequest2 class ScanReport(BaseModel): @@ -1475,10 +1469,10 @@ class ScanReport(BaseModel): snapshot_id: int = Field(..., alias='snapshot-id') filter: Expression schema_id: int = Field(..., alias='schema-id') - projected_field_ids: List[int] = Field(..., alias='projected-field-ids') - projected_field_names: List[str] = Field(..., alias='projected-field-names') + projected_field_ids: list[int] = Field(..., alias='projected-field-ids') + projected_field_names: list[str] = Field(..., alias='projected-field-names') metrics: Metrics - metadata: Optional[Dict[str, str]] = None + metadata: dict[str, str] | None = None class CommitTableResponse(BaseModel): @@ -1487,43 +1481,43 @@ class CommitTableResponse(BaseModel): class PlanTableScanRequest(BaseModel): - snapshot_id: Optional[int] = Field( + snapshot_id: int | None = Field( None, alias='snapshot-id', description='Identifier for the snapshot to scan in a point-in-time scan', ) - select: Optional[List[FieldName]] = Field( + select: list[FieldName] | None = Field( None, description='List of selected schema fields' ) - filter: Optional[Expression] = Field( + filter: Expression | None = Field( None, description='Expression used to filter the table data' ) - min_rows_requested: Optional[int] = Field( + min_rows_requested: int | None = Field( None, alias='min-rows-requested', description='The minimum number of rows requested for the scan. This is used as a hint to the server to not have to return more rows than necessary. It is not required for the server to return that many rows since the scan may not produce that many rows. The server can also return more rows than requested.', ) - case_sensitive: Optional[bool] = Field( + case_sensitive: bool | None = Field( True, alias='case-sensitive', description='Enables case sensitive field matching for filter and select', ) - use_snapshot_schema: Optional[bool] = Field( + use_snapshot_schema: bool | None = Field( False, alias='use-snapshot-schema', description='Whether to use the schema at the time the snapshot was written.\nWhen time travelling, the snapshot schema should be used (true). When scanning a branch, the table schema should be used (false).', ) - start_snapshot_id: Optional[int] = Field( + start_snapshot_id: int | None = Field( None, alias='start-snapshot-id', description='Starting snapshot ID for an incremental scan (exclusive)', ) - end_snapshot_id: Optional[int] = Field( + end_snapshot_id: int | None = Field( None, alias='end-snapshot-id', description='Ending snapshot ID for an incremental scan (inclusive).\nRequired when start-snapshot-id is specified.', ) - stats_fields: Optional[List[FieldName]] = Field( + stats_fields: list[FieldName] | None = Field( None, alias='stats-fields', description='List of fields for which the service should send column stats.', @@ -1536,15 +1530,15 @@ class ResidualFilter(BaseModel): If the residual is not present, the client must produce the residual or use the original filter. """ - __root__: Union[ - ResidualFilter2, - ResidualFilter3, - ResidualFilter4, - ResidualFilter5, - ResidualFilter6, - ResidualFilter7, - ResidualFilter8, - ] = Field( + __root__: ( + ResidualFilter2 + | ResidualFilter3 + | ResidualFilter4 + | ResidualFilter5 + | ResidualFilter6 + | ResidualFilter7 + | ResidualFilter8 + ) = Field( ..., description='An optional filter to be applied to rows in this file scan task.\nIf the residual is not present, the client must produce the residual or use the original filter.', ) @@ -1552,12 +1546,12 @@ class ResidualFilter(BaseModel): class FileScanTask(BaseModel): data_file: DataFile = Field(..., alias='data-file') - delete_file_references: Optional[List[int]] = Field( + delete_file_references: list[int] | None = Field( None, alias='delete-file-references', description='A list of indices in the delete files array (0-based)', ) - residual_filter: Optional[ResidualFilter] = Field( + residual_filter: ResidualFilter | None = Field( None, alias='residual-filter', description='An optional filter to be applied to rows in this file scan task.\nIf the residual is not present, the client must produce the residual or use the original filter.', @@ -1565,10 +1559,8 @@ class FileScanTask(BaseModel): class Schema(StructType): - schema_id: Optional[int] = Field(None, alias='schema-id') - identifier_field_ids: Optional[List[int]] = Field( - None, alias='identifier-field-ids' - ) + schema_id: int | None = Field(None, alias='schema-id') + identifier_field_ids: list[int] | None = Field(None, alias='identifier-field-ids') class ResidualFilter4(AndOrExpression, ResidualFilter1): @@ -1591,7 +1583,7 @@ class CompletedPlanningResult(ScanTasks): """ status: Literal['completed'] = Field(..., const=True) - storage_credentials: Optional[List[StorageCredential]] = Field( + storage_credentials: list[StorageCredential] | None = Field( None, alias='storage-credentials', description='Storage credentials for accessing the files returned in the scan result.\nIf the server returns storage credentials as part of the completed scan planning response, the expectation is for the client to use these credentials to read the files returned in the FileScanTasks as part of the scan result.', From f4005868942a929436576389ac764be9841ec128 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 22 Dec 2025 14:15:46 -0800 Subject: [PATCH 124/201] Site: Updates for 1.10.1 Release (#14907) * Site: Updates for 1.10.1 Release * add missing space * remove duplicate items --- site/docs/releases.md | 51 +++++++++++++++++++++++++++++++++++++++++-- site/mkdocs.yml | 2 +- site/nav.yml | 3 ++- 3 files changed, 52 insertions(+), 4 deletions(-) diff --git a/site/docs/releases.md b/site/docs/releases.md index 04d55e4287d6..0fc0b5d77323 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -67,6 +67,55 @@ To add a dependency on Iceberg in Maven, add the following to your `pom.xml`: ``` +### 1.10.1 release + +Apache Iceberg 1.10.1 was released on Dec 21, 2025. + +The 1.10.1 release contains bug fixes. For full release notes visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.10.1) + +* API + - API: required nested fields within optional structs can produce null ([\#13804](https://github.com/apache/iceberg/pull/13804)) + - API: Detect whether required fields nested within optionals can produce nulls ([\#14270](https://github.com/apache/iceberg/pull/14270)) +* Core + - Prevent dropping a namespace when it contains views ([\#14456](https://github.com/apache/iceberg/pull/14456)) + - Fix overflow due to default value on timestamp nanos ([\#14359](https://github.com/apache/iceberg/pull/14359)) + - Fix the unpartitioned check in replace partitions ([\#14186](https://github.com/apache/iceberg/pull/14186)) + - Fix for respecting custom location providers in SerializableTable and actions ([\#14280](https://github.com/apache/iceberg/pull/14280)) + - Fix `NAN_VALUE_COUNTS` serialization for ContentFile ([\#14721](https://github.com/apache/iceberg/pull/14721)) + - Handle unknown type during deletes ([\#14356](https://github.com/apache/iceberg/pull/14356)) + - REST: Fix validation of PlanTableScanRequest ([\#14561](https://github.com/apache/iceberg/pull/14561)) + - REST: Fix validation of PlanTableScanResponse ([\#14562](https://github.com/apache/iceberg/pull/14562)) + - REST: Fix RESTFileScanTaskParser to handle empty delete file references list ([\#14568](https://github.com/apache/iceberg/pull/14568)) + - REST: Fix serde of tasks with multiple deletes ([\#14573](https://github.com/apache/iceberg/pull/14573)) + - REST: Align PlanTableScanRequest filter with the OpenAPI spec ([\#14658](https://github.com/apache/iceberg/pull/14658)) + - REST: Make plan status consistent with the REST spec ([\#14643](https://github.com/apache/iceberg/pull/14643)) +* Parquet + - Fix NameMapping loss in `ParquetUtil.footerMetrics` ([\#14617](https://github.com/apache/iceberg/pull/14617)) + - Add variant type support to `ParquetTypeVisitor` ([\#14588](https://github.com/apache/iceberg/pull/14588)) + - Fix variant type filtering in `ParquetMetricsRowGroupFilter` ([\#14081](https://github.com/apache/iceberg/pull/14081)) + - handle NPE for `VariantLogicalType` in `TypeWithSchemaVisitor` ([\#14261](https://github.com/apache/iceberg/pull/14261)) + - Fix `UUID` `ClassCastException` when reading Parquet files with UUIDs ([\#14027](https://github.com/apache/iceberg/pull/14027)) +* Flink + - Ensure `DynamicCommitter` idempotence in the presence of failures ([\#14182](https://github.com/apache/iceberg/pull/14182)) + - Backport Ensure `DynamicCommitter` idempotence in the presence of failures ([\#14213](https://github.com/apache/iceberg/pull/14213)) + - Prevent recreation of `ManifestOutputFileFactory` during flushing ([\#14358](https://github.com/apache/iceberg/pull/14358)) + - Backport Prevent recreation of `ManifestOutputFileFactory` during flushing ([\#14385](https://github.com/apache/iceberg/pull/14385)) + - Fix cache refreshing in `DynamicIcebergSink` ([\#14406](https://github.com/apache/iceberg/pull/14406)) + - Port fix cache refreshing in `DynamicIcebergSink` ([\#14765](https://github.com/apache/iceberg/pull/14765)) +* Spark + - 3.4: Pass format-version when creating a snapshot ([\#14170](https://github.com/apache/iceberg/pull/14170)) + - 3.5: Pass format-version when creating a snapshot ([\#14169](https://github.com/apache/iceberg/pull/14169)) + - 4.0: Pass format-version when creating a snapshot in table migration actions ([\#14163](https://github.com/apache/iceberg/pull/14163)) + - Fix Z-order UDF to correctly handle `DateType` ([\#14108](https://github.com/apache/iceberg/pull/14108)) +* Kafka Connect + - Merge control topic and last persisted offsets ([\#14525](https://github.com/apache/iceberg/pull/14525)) +* Vendor integrations + - AWS: Configure builder for reuse of HTTP connection pool in SDK v2 ([\#14161](https://github.com/apache/iceberg/pull/14161)) + - AWS: Fix leaked credentials when contacting multiple catalogs ([\#14178](https://github.com/apache/iceberg/pull/14178)) + - AWS: Exclude logging dependencies from the bundle ([\#14225](https://github.com/apache/iceberg/pull/14225)) + +## Past releases + ### 1.10.0 release Apache Iceberg 1.10.0 was released on September 11, 2025. @@ -196,8 +245,6 @@ The 1.10.0 release contains bug fixes and new features. For full release notes v - Comet: 0.5.0 -> 0.8.1 - Apache httpclient: 5.4.3 -> 5.4.4 -## Past releases - ### 1.9.2 release Apache Iceberg 1.9.2 was released on Jul 16, 2025. diff --git a/site/mkdocs.yml b/site/mkdocs.yml index 6ea9ceb3636b..5bbf64bf0013 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -84,7 +84,7 @@ markdown_extensions: permalink: 🔗 extra: - icebergVersion: '1.10.0' + icebergVersion: '1.10.1' nessieVersion: '0.104.5' flinkVersion: '2.0.0' flinkVersionMajor: '2.0' diff --git a/site/nav.yml b/site/nav.yml index 58f7dc4d2aa4..8b1ffaa25b90 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -23,8 +23,9 @@ nav: - Docs: - Java: - Nightly: '!include docs/docs/nightly/mkdocs.yml' - - Latest (1.10.0): '!include docs/docs/latest/mkdocs.yml' + - Latest (1.10.1): '!include docs/docs/latest/mkdocs.yml' - Previous: + - 1.10.0: '!include docs/docs/1.10.0/mkdocs.yml' - 1.9.2: '!include docs/docs/1.9.2/mkdocs.yml' - 1.9.1: '!include docs/docs/1.9.1/mkdocs.yml' - 1.9.0: '!include docs/docs/1.9.0/mkdocs.yml' From f1e02730c04bbb33f9bfd4bcf2602d7cb5951ceb Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 22 Dec 2025 16:29:52 -0800 Subject: [PATCH 125/201] Issue template: add 1.10.1 to version dropdown (#14916) --- .github/ISSUE_TEMPLATE/iceberg_bug_report.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index b383de4e766b..2e507128434f 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml @@ -34,7 +34,8 @@ body: description: What Apache Iceberg version are you using? multiple: false options: - - "1.10.0 (latest release)" + - "1.10.1 (latest release)" + - "1.10.0" - "1.9.2" - "1.9.1" - "1.9.0" From 752a2820cd774600a5e803ef12732c2596a2b303 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 22 Dec 2025 16:52:45 -0800 Subject: [PATCH 126/201] Site: correct release time for 1.10.1 (#14918) --- site/docs/releases.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/docs/releases.md b/site/docs/releases.md index 0fc0b5d77323..d3a5ed9a4904 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -69,7 +69,7 @@ To add a dependency on Iceberg in Maven, add the following to your `pom.xml`: ### 1.10.1 release -Apache Iceberg 1.10.1 was released on Dec 21, 2025. +Apache Iceberg 1.10.1 was released on Dec 22, 2025. The 1.10.1 release contains bug fixes. For full release notes visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.10.1) From 3149892438ec7d4114fc2d79e970188dd3c90a44 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Tue, 23 Dec 2025 09:34:58 +0800 Subject: [PATCH 127/201] Spark: Move 4.0 as 4.1 --- spark/{v4.0 => v4.1}/build.gradle | 0 .../apache/iceberg/DeleteFileIndexBenchmark.java | 0 .../spark/MergeCardinalityCheckBenchmark.java | 0 .../org/apache/iceberg/spark/PlanningBenchmark.java | 0 .../iceberg/spark/TaskGroupPlanningBenchmark.java | 0 .../iceberg/spark/UpdateProjectionBenchmark.java | 0 .../IcebergSqlExtensions.g4 | 0 .../extensions/IcebergSparkSessionExtensions.scala | 0 .../spark/sql/catalyst/analysis/CheckViews.scala | 0 .../spark/sql/catalyst/analysis/ResolveViews.scala | 0 .../sql/catalyst/analysis/RewriteViewCommands.scala | 0 .../spark/sql/catalyst/analysis/ViewUtil.scala | 0 .../catalyst/optimizer/ReplaceStaticInvoke.scala | 0 .../IcebergSparkSqlExtensionsParser.scala | 0 .../extensions/IcebergSqlExtensionsAstBuilder.scala | 0 .../catalyst/plans/logical/AddPartitionField.scala | 0 .../sql/catalyst/plans/logical/BranchOptions.scala | 0 .../plans/logical/CreateOrReplaceBranch.scala | 0 .../catalyst/plans/logical/CreateOrReplaceTag.scala | 0 .../sql/catalyst/plans/logical/DropBranch.scala | 0 .../plans/logical/DropIdentifierFields.scala | 0 .../catalyst/plans/logical/DropPartitionField.scala | 0 .../spark/sql/catalyst/plans/logical/DropTag.scala | 0 .../plans/logical/ReplacePartitionField.scala | 0 .../plans/logical/SetIdentifierFields.scala | 0 .../sql/catalyst/plans/logical/TagOptions.scala | 0 .../plans/logical/views/CreateIcebergView.scala | 0 .../plans/logical/views/DropIcebergView.scala | 0 .../plans/logical/views/ResolvedV2View.scala | 0 .../plans/logical/views/ShowIcebergViews.scala | 0 .../datasources/v2/AddPartitionFieldExec.scala | 0 .../v2/AlterV2ViewSetPropertiesExec.scala | 0 .../v2/AlterV2ViewUnsetPropertiesExec.scala | 0 .../datasources/v2/CreateOrReplaceBranchExec.scala | 0 .../datasources/v2/CreateOrReplaceTagExec.scala | 0 .../execution/datasources/v2/CreateV2ViewExec.scala | 0 .../datasources/v2/DescribeV2ViewExec.scala | 0 .../execution/datasources/v2/DropBranchExec.scala | 0 .../datasources/v2/DropIdentifierFieldsExec.scala | 0 .../datasources/v2/DropPartitionFieldExec.scala | 0 .../sql/execution/datasources/v2/DropTagExec.scala | 0 .../execution/datasources/v2/DropV2ViewExec.scala | 0 .../v2/ExtendedDataSourceV2Strategy.scala | 0 .../execution/datasources/v2/RenameV2ViewExec.scala | 0 .../datasources/v2/ReplacePartitionFieldExec.scala | 0 .../datasources/v2/SetIdentifierFieldsExec.scala | 0 .../v2/SetWriteDistributionAndOrderingExec.scala | 0 .../datasources/v2/ShowCreateV2ViewExec.scala | 0 .../datasources/v2/ShowV2ViewPropertiesExec.scala | 0 .../execution/datasources/v2/ShowV2ViewsExec.scala | 0 .../apache/iceberg/spark/TestExtendedParser.java | 0 .../apache/iceberg/spark/extensions/Employee.java | 0 .../spark/extensions/ExtensionsTestBase.java | 0 .../iceberg/spark/extensions/ProcedureUtil.java | 0 .../iceberg/spark/extensions/SparkPlanUtil.java | 0 .../extensions/SparkRowLevelOperationsTestBase.java | 0 .../spark/extensions/TestAddFilesProcedure.java | 0 .../extensions/TestAlterTablePartitionFields.java | 0 .../spark/extensions/TestAlterTableSchema.java | 0 .../spark/extensions/TestAncestorsOfProcedure.java | 0 .../iceberg/spark/extensions/TestBranchDDL.java | 0 .../spark/extensions/TestChangelogTable.java | 0 .../extensions/TestCherrypickSnapshotProcedure.java | 0 .../TestComputePartitionStatsProcedure.java | 0 .../extensions/TestComputeTableStatsProcedure.java | 0 .../spark/extensions/TestConflictValidation.java | 0 .../spark/extensions/TestCopyOnWriteDelete.java | 0 .../spark/extensions/TestCopyOnWriteMerge.java | 0 .../spark/extensions/TestCopyOnWriteUpdate.java | 0 .../extensions/TestCopyOnWriteWithLineage.java | 0 .../TestCreateChangelogViewProcedure.java | 0 .../apache/iceberg/spark/extensions/TestDelete.java | 0 .../extensions/TestExpireSnapshotsProcedure.java | 0 .../extensions/TestFastForwardBranchProcedure.java | 0 .../apache/iceberg/spark/extensions/TestMerge.java | 0 .../spark/extensions/TestMergeOnReadDelete.java | 0 .../spark/extensions/TestMergeOnReadMerge.java | 0 .../spark/extensions/TestMergeOnReadUpdate.java | 0 .../extensions/TestMergeOnReadWithLineage.java | 0 .../TestMetaColumnProjectionWithStageScan.java | 0 .../spark/extensions/TestMetadataTables.java | 0 .../spark/extensions/TestMigrateTableProcedure.java | 0 .../extensions/TestPublishChangesProcedure.java | 0 .../extensions/TestRegisterTableProcedure.java | 0 .../spark/extensions/TestRemoteScanPlanning.java | 0 .../extensions/TestRemoveOrphanFilesProcedure.java | 0 .../iceberg/spark/extensions/TestReplaceBranch.java | 0 .../TestRequiredDistributionAndOrdering.java | 0 .../extensions/TestRewriteDataFilesProcedure.java | 0 .../extensions/TestRewriteManifestsProcedure.java | 0 .../extensions/TestRewritePositionDeleteFiles.java | 0 .../TestRewritePositionDeleteFilesProcedure.java | 0 .../extensions/TestRewriteTablePathProcedure.java | 0 .../extensions/TestRollbackToSnapshotProcedure.java | 0 .../TestRollbackToTimestampProcedure.java | 0 .../TestRowLevelOperationsWithLineage.java | 0 .../extensions/TestSetCurrentSnapshotProcedure.java | 0 .../TestSetWriteDistributionAndOrdering.java | 0 .../extensions/TestSnapshotTableProcedure.java | 0 ...StoragePartitionedJoinsInRowLevelOperations.java | 0 .../extensions/TestSystemFunctionPushDownDQL.java | 0 ...tSystemFunctionPushDownInRowLevelOperations.java | 0 .../apache/iceberg/spark/extensions/TestTagDDL.java | 0 .../apache/iceberg/spark/extensions/TestUpdate.java | 0 .../apache/iceberg/spark/extensions/TestViews.java | 0 .../iceberg/spark/extensions/TestWriteAborts.java | 0 .../org/apache/iceberg/spark/source/PlanUtils.java | 0 spark/{v4.0 => v4.1}/spark-runtime/LICENSE | 0 spark/{v4.0 => v4.1}/spark-runtime/NOTICE | 0 .../org/apache/iceberg/spark/TestRoundTrip.java | 0 .../apache/iceberg/spark/SparkBenchmarkUtil.java | 0 .../spark/action/DeleteOrphanFilesBenchmark.java | 0 .../action/IcebergSortCompactionBenchmark.java | 0 .../iceberg/spark/action/RandomGeneratingUDF.java | 0 .../SparkParquetReadersFlatDataBenchmark.java | 0 .../SparkParquetReadersNestedDataBenchmark.java | 0 .../SparkParquetWritersFlatDataBenchmark.java | 0 .../SparkParquetWritersNestedDataBenchmark.java | 0 .../org/apache/iceberg/spark/source/Action.java | 0 .../iceberg/spark/source/DVReaderBenchmark.java | 0 .../iceberg/spark/source/DVWriterBenchmark.java | 0 .../spark/source/IcebergSourceBenchmark.java | 0 .../spark/source/IcebergSourceDeleteBenchmark.java | 0 .../source/IcebergSourceFlatDataBenchmark.java | 0 .../source/IcebergSourceNestedDataBenchmark.java | 0 .../IcebergSourceNestedListDataBenchmark.java | 0 .../iceberg/spark/source/WritersBenchmark.java | 0 .../spark/source/avro/AvroWritersBenchmark.java | 0 .../IcebergSourceFlatAvroDataReadBenchmark.java | 0 .../IcebergSourceNestedAvroDataReadBenchmark.java | 0 .../orc/IcebergSourceFlatORCDataBenchmark.java | 0 .../orc/IcebergSourceFlatORCDataReadBenchmark.java | 0 ...cebergSourceNestedListORCDataWriteBenchmark.java | 0 .../IcebergSourceNestedORCDataReadBenchmark.java | 0 ...IcebergSourceFlatParquetDataFilterBenchmark.java | 0 .../IcebergSourceFlatParquetDataReadBenchmark.java | 0 .../IcebergSourceFlatParquetDataWriteBenchmark.java | 0 ...rgSourceNestedListParquetDataWriteBenchmark.java | 0 ...ebergSourceNestedParquetDataFilterBenchmark.java | 0 ...IcebergSourceNestedParquetDataReadBenchmark.java | 0 ...cebergSourceNestedParquetDataWriteBenchmark.java | 0 .../IcebergSourceParquetEqDeleteBenchmark.java | 0 ...cebergSourceParquetMultiDeleteFileBenchmark.java | 0 .../IcebergSourceParquetPosDeleteBenchmark.java | 0 ...rgSourceParquetWithUnrelatedDeleteBenchmark.java | 0 .../source/parquet/ParquetWritersBenchmark.java | 0 ...adDictionaryEncodedFlatParquetDataBenchmark.java | 0 .../VectorizedReadFlatParquetDataBenchmark.java | 0 .../VectorizedReadParquetDecimalBenchmark.java | 0 .../apache/iceberg/SparkDistributedDataScan.java | 0 .../java/org/apache/iceberg/spark/BaseCatalog.java | 0 .../iceberg/spark/BaseFileRewriteCoordinator.java | 0 .../org/apache/iceberg/spark/ChangelogIterator.java | 0 .../org/apache/iceberg/spark/CommitMetadata.java | 0 .../apache/iceberg/spark/ComputeUpdateIterator.java | 0 .../org/apache/iceberg/spark/ExtendedParser.java | 0 .../iceberg/spark/FileRewriteCoordinator.java | 0 .../java/org/apache/iceberg/spark/IcebergSpark.java | 0 .../java/org/apache/iceberg/spark/JobGroupInfo.java | 0 .../org/apache/iceberg/spark/JobGroupUtils.java | 0 .../org/apache/iceberg/spark/OrcBatchReadConf.java | 0 .../apache/iceberg/spark/ParquetBatchReadConf.java | 0 .../org/apache/iceberg/spark/ParquetReaderType.java | 0 .../org/apache/iceberg/spark/PathIdentifier.java | 0 .../spark/PositionDeletesRewriteCoordinator.java | 0 .../spark/PruneColumnsWithoutReordering.java | 0 .../iceberg/spark/RemoveCarryoverIterator.java | 0 .../iceberg/spark/RemoveNetCarryoverIterator.java | 0 .../apache/iceberg/spark/RollbackStagedTable.java | 0 .../apache/iceberg/spark/ScanTaskSetManager.java | 0 .../org/apache/iceberg/spark/SortOrderToSpark.java | 0 .../java/org/apache/iceberg/spark/Spark3Util.java | 0 .../org/apache/iceberg/spark/SparkAggregates.java | 0 .../iceberg/spark/SparkCachedTableCatalog.java | 0 .../java/org/apache/iceberg/spark/SparkCatalog.java | 0 .../apache/iceberg/spark/SparkCompressionUtil.java | 0 .../org/apache/iceberg/spark/SparkConfParser.java | 0 .../org/apache/iceberg/spark/SparkContentFile.java | 0 .../org/apache/iceberg/spark/SparkDataFile.java | 0 .../org/apache/iceberg/spark/SparkDeleteFile.java | 0 .../apache/iceberg/spark/SparkExceptionUtil.java | 0 .../apache/iceberg/spark/SparkExecutorCache.java | 0 .../java/org/apache/iceberg/spark/SparkFilters.java | 0 .../org/apache/iceberg/spark/SparkFixupTypes.java | 0 .../apache/iceberg/spark/SparkFunctionCatalog.java | 0 .../org/apache/iceberg/spark/SparkReadConf.java | 0 .../org/apache/iceberg/spark/SparkReadOptions.java | 0 .../apache/iceberg/spark/SparkSQLProperties.java | 0 .../org/apache/iceberg/spark/SparkSchemaUtil.java | 0 .../apache/iceberg/spark/SparkSessionCatalog.java | 0 .../org/apache/iceberg/spark/SparkStructLike.java | 0 .../org/apache/iceberg/spark/SparkTableCache.java | 0 .../org/apache/iceberg/spark/SparkTableUtil.java | 0 .../org/apache/iceberg/spark/SparkTypeToType.java | 0 .../org/apache/iceberg/spark/SparkTypeVisitor.java | 0 .../java/org/apache/iceberg/spark/SparkUtil.java | 0 .../org/apache/iceberg/spark/SparkV2Filters.java | 0 .../apache/iceberg/spark/SparkValueConverter.java | 0 .../org/apache/iceberg/spark/SparkWriteConf.java | 0 .../org/apache/iceberg/spark/SparkWriteOptions.java | 0 .../iceberg/spark/SparkWriteRequirements.java | 0 .../org/apache/iceberg/spark/SparkWriteUtil.java | 0 .../org/apache/iceberg/spark/SupportsFunctions.java | 0 .../apache/iceberg/spark/SupportsReplaceView.java | 0 .../org/apache/iceberg/spark/TypeToSparkType.java | 0 .../actions/BaseSnapshotUpdateSparkAction.java | 0 .../iceberg/spark/actions/BaseSparkAction.java | 0 .../spark/actions/BaseTableCreationSparkAction.java | 0 .../actions/ComputePartitionStatsSparkAction.java | 0 .../spark/actions/ComputeTableStatsSparkAction.java | 0 .../spark/actions/DeleteOrphanFilesSparkAction.java | 0 .../actions/DeleteReachableFilesSparkAction.java | 0 .../spark/actions/ExpireSnapshotsSparkAction.java | 0 .../org/apache/iceberg/spark/actions/FileInfo.java | 0 .../iceberg/spark/actions/ManifestFileBean.java | 0 .../spark/actions/MigrateTableSparkAction.java | 0 .../apache/iceberg/spark/actions/NDVSketchUtil.java | 0 .../actions/RemoveDanglingDeletesSparkAction.java | 0 .../spark/actions/RewriteDataFilesSparkAction.java | 0 .../spark/actions/RewriteManifestsSparkAction.java | 0 .../RewritePositionDeleteFilesSparkAction.java | 0 .../spark/actions/RewriteTablePathSparkAction.java | 0 .../iceberg/spark/actions/SetAccumulator.java | 0 .../spark/actions/SnapshotTableSparkAction.java | 0 .../apache/iceberg/spark/actions/SparkActions.java | 0 .../actions/SparkBinPackFileRewriteRunner.java | 0 .../spark/actions/SparkDataFileRewriteRunner.java | 0 .../actions/SparkRewritePositionDeleteRunner.java | 0 .../iceberg/spark/actions/SparkRewriteRunner.java | 0 .../actions/SparkShufflingDataRewritePlanner.java | 0 .../actions/SparkShufflingFileRewriteRunner.java | 0 .../spark/actions/SparkSortFileRewriteRunner.java | 0 .../spark/actions/SparkZOrderFileRewriteRunner.java | 0 .../iceberg/spark/actions/SparkZOrderUDF.java | 0 .../spark/data/AvroWithSparkSchemaVisitor.java | 0 .../spark/data/ParquetWithSparkSchemaVisitor.java | 0 .../apache/iceberg/spark/data/SparkAvroWriter.java | 0 .../apache/iceberg/spark/data/SparkOrcReader.java | 0 .../iceberg/spark/data/SparkOrcValueReaders.java | 0 .../iceberg/spark/data/SparkOrcValueWriters.java | 0 .../apache/iceberg/spark/data/SparkOrcWriter.java | 0 .../iceberg/spark/data/SparkParquetReaders.java | 0 .../iceberg/spark/data/SparkParquetWriters.java | 0 .../iceberg/spark/data/SparkPlannedAvroReader.java | 0 .../iceberg/spark/data/SparkValueReaders.java | 0 .../iceberg/spark/data/SparkValueWriters.java | 0 .../data/vectorized/ArrowVectorAccessorFactory.java | 0 .../spark/data/vectorized/ArrowVectorAccessors.java | 0 .../spark/data/vectorized/ColumnVectorBuilder.java | 0 .../data/vectorized/ColumnVectorWithFilter.java | 0 .../spark/data/vectorized/ColumnarBatchReader.java | 0 .../spark/data/vectorized/ColumnarBatchUtil.java | 0 .../spark/data/vectorized/CometColumnReader.java | 0 .../data/vectorized/CometColumnarBatchReader.java | 0 .../data/vectorized/CometConstantColumnReader.java | 0 .../data/vectorized/CometDeleteColumnReader.java | 0 .../data/vectorized/CometDeletedColumnVector.java | 0 .../data/vectorized/CometPositionColumnReader.java | 0 .../vectorized/CometVectorizedReaderBuilder.java | 0 .../spark/data/vectorized/ConstantColumnVector.java | 0 .../spark/data/vectorized/DeletedColumnVector.java | 0 .../data/vectorized/IcebergArrowColumnVector.java | 0 .../data/vectorized/RowPositionColumnVector.java | 0 .../vectorized/UpdatableDeletedColumnVector.java | 0 .../data/vectorized/VectorizedSparkOrcReaders.java | 0 .../vectorized/VectorizedSparkParquetReaders.java | 0 .../iceberg/spark/functions/BaseScalarFunction.java | 0 .../iceberg/spark/functions/BucketFunction.java | 0 .../iceberg/spark/functions/DaysFunction.java | 0 .../iceberg/spark/functions/HoursFunction.java | 0 .../spark/functions/IcebergVersionFunction.java | 0 .../iceberg/spark/functions/MonthsFunction.java | 0 .../iceberg/spark/functions/SparkFunctions.java | 0 .../iceberg/spark/functions/TruncateFunction.java | 0 .../spark/functions/UnaryUnboundFunction.java | 0 .../iceberg/spark/functions/YearsFunction.java | 0 .../iceberg/spark/procedures/AddFilesProcedure.java | 0 .../spark/procedures/AncestorsOfProcedure.java | 0 .../iceberg/spark/procedures/BaseProcedure.java | 0 .../procedures/CherrypickSnapshotProcedure.java | 0 .../procedures/ComputePartitionStatsProcedure.java | 0 .../procedures/ComputeTableStatsProcedure.java | 0 .../procedures/CreateChangelogViewProcedure.java | 0 .../spark/procedures/ExpireSnapshotsProcedure.java | 0 .../procedures/FastForwardBranchProcedure.java | 0 .../spark/procedures/MigrateTableProcedure.java | 0 .../iceberg/spark/procedures/ProcedureInput.java | 0 .../spark/procedures/PublishChangesProcedure.java | 0 .../spark/procedures/RegisterTableProcedure.java | 0 .../procedures/RemoveOrphanFilesProcedure.java | 0 .../spark/procedures/RewriteDataFilesProcedure.java | 0 .../spark/procedures/RewriteManifestsProcedure.java | 0 .../RewritePositionDeleteFilesProcedure.java | 0 .../spark/procedures/RewriteTablePathProcedure.java | 0 .../procedures/RollbackToSnapshotProcedure.java | 0 .../procedures/RollbackToTimestampProcedure.java | 0 .../procedures/SetCurrentSnapshotProcedure.java | 0 .../spark/procedures/SnapshotTableProcedure.java | 0 .../iceberg/spark/procedures/SparkProcedures.java | 0 .../iceberg/spark/source/BaseBatchReader.java | 0 .../org/apache/iceberg/spark/source/BaseReader.java | 0 .../apache/iceberg/spark/source/BaseRowReader.java | 0 .../iceberg/spark/source/BatchDataReader.java | 0 .../iceberg/spark/source/ChangelogRowReader.java | 0 .../org/apache/iceberg/spark/source/DVIterator.java | 0 .../spark/source/EqualityDeleteRowReader.java | 0 .../iceberg/spark/source/ExtractRowLineage.java | 0 .../iceberg/spark/source/HasIcebergCatalog.java | 0 .../apache/iceberg/spark/source/IcebergSource.java | 0 .../iceberg/spark/source/InternalRowWrapper.java | 0 .../spark/source/PositionDeletesRowReader.java | 0 .../apache/iceberg/spark/source/RowDataReader.java | 0 .../spark/source/SerializableTableWithSize.java | 0 .../org/apache/iceberg/spark/source/SparkBatch.java | 0 .../iceberg/spark/source/SparkBatchQueryScan.java | 0 .../iceberg/spark/source/SparkChangelogScan.java | 0 .../iceberg/spark/source/SparkChangelogTable.java | 0 .../iceberg/spark/source/SparkCleanupUtil.java | 0 .../iceberg/spark/source/SparkColumnStatistics.java | 0 .../spark/source/SparkColumnarReaderFactory.java | 0 .../spark/source/SparkCopyOnWriteOperation.java | 0 .../iceberg/spark/source/SparkCopyOnWriteScan.java | 0 .../spark/source/SparkFileWriterFactory.java | 0 .../iceberg/spark/source/SparkInputPartition.java | 0 .../apache/iceberg/spark/source/SparkLocalScan.java | 0 .../iceberg/spark/source/SparkMetadataColumn.java | 0 .../iceberg/spark/source/SparkMicroBatchStream.java | 0 .../spark/source/SparkPartitioningAwareScan.java | 0 .../iceberg/spark/source/SparkPlanningUtil.java | 0 .../spark/source/SparkPositionDeletesRewrite.java | 0 .../source/SparkPositionDeletesRewriteBuilder.java | 0 .../spark/source/SparkPositionDeltaOperation.java | 0 .../spark/source/SparkPositionDeltaWrite.java | 0 .../source/SparkPositionDeltaWriteBuilder.java | 0 .../spark/source/SparkRowLevelOperationBuilder.java | 0 .../iceberg/spark/source/SparkRowReaderFactory.java | 0 .../org/apache/iceberg/spark/source/SparkScan.java | 0 .../iceberg/spark/source/SparkScanBuilder.java | 0 .../iceberg/spark/source/SparkStagedScan.java | 0 .../spark/source/SparkStagedScanBuilder.java | 0 .../org/apache/iceberg/spark/source/SparkTable.java | 0 .../org/apache/iceberg/spark/source/SparkView.java | 0 .../org/apache/iceberg/spark/source/SparkWrite.java | 0 .../iceberg/spark/source/SparkWriteBuilder.java | 0 .../iceberg/spark/source/StagedSparkTable.java | 0 .../java/org/apache/iceberg/spark/source/Stats.java | 0 .../iceberg/spark/source/StreamingOffset.java | 0 .../iceberg/spark/source/StructInternalRow.java | 0 .../spark/source/metrics/EqualityDeleteFiles.java | 0 .../spark/source/metrics/IndexedDeleteFiles.java | 0 .../iceberg/spark/source/metrics/NumDeletes.java | 0 .../iceberg/spark/source/metrics/NumSplits.java | 0 .../spark/source/metrics/PositionalDeleteFiles.java | 0 .../spark/source/metrics/ResultDataFiles.java | 0 .../spark/source/metrics/ResultDeleteFiles.java | 0 .../spark/source/metrics/ScannedDataManifests.java | 0 .../source/metrics/ScannedDeleteManifests.java | 0 .../spark/source/metrics/SkippedDataFiles.java | 0 .../spark/source/metrics/SkippedDataManifests.java | 0 .../spark/source/metrics/SkippedDeleteFiles.java | 0 .../source/metrics/SkippedDeleteManifests.java | 0 .../source/metrics/TaskEqualityDeleteFiles.java | 0 .../source/metrics/TaskIndexedDeleteFiles.java | 0 .../spark/source/metrics/TaskNumDeletes.java | 0 .../iceberg/spark/source/metrics/TaskNumSplits.java | 0 .../source/metrics/TaskPositionalDeleteFiles.java | 0 .../spark/source/metrics/TaskResultDataFiles.java | 0 .../spark/source/metrics/TaskResultDeleteFiles.java | 0 .../source/metrics/TaskScannedDataManifests.java | 0 .../source/metrics/TaskScannedDeleteManifests.java | 0 .../spark/source/metrics/TaskSkippedDataFiles.java | 0 .../source/metrics/TaskSkippedDataManifests.java | 0 .../source/metrics/TaskSkippedDeleteFiles.java | 0 .../source/metrics/TaskSkippedDeleteManifests.java | 0 .../spark/source/metrics/TaskTotalDataFileSize.java | 0 .../source/metrics/TaskTotalDataManifests.java | 0 .../source/metrics/TaskTotalDeleteFileSize.java | 0 .../source/metrics/TaskTotalDeleteManifests.java | 0 .../source/metrics/TaskTotalPlanningDuration.java | 0 .../spark/source/metrics/TotalDataFileSize.java | 0 .../spark/source/metrics/TotalDataManifests.java | 0 .../spark/source/metrics/TotalDeleteFileSize.java | 0 .../spark/source/metrics/TotalDeleteManifests.java | 0 .../spark/source/metrics/TotalPlanningDuration.java | 0 .../catalyst/analysis/IcebergAnalysisException.java | 0 .../org.apache.spark.sql.sources.DataSourceRegister | 0 .../catalyst/plans/logical/OrderAwareCoalesce.scala | 0 .../logical/SetWriteDistributionAndOrdering.scala | 0 .../plans/logical/SortOrderParserUtil.scala | 0 .../apache/spark/sql/catalyst/utils/PlanUtils.scala | 0 .../sql/execution/OrderAwareCoalesceExec.scala | 0 .../datasources/SparkExpressionConverter.scala | 0 .../org/apache/spark/sql/stats/ThetaSketchAgg.scala | 0 .../test/java/org/apache/iceberg/KryoHelpers.java | 0 .../iceberg/SparkDistributedDataScanTestBase.java | 0 .../java/org/apache/iceberg/TaskCheckHelper.java | 0 .../apache/iceberg/TestDataFileSerialization.java | 0 .../org/apache/iceberg/TestFileIOSerialization.java | 0 .../TestHadoopMetricsContextSerialization.java | 0 .../iceberg/TestManifestFileSerialization.java | 0 .../apache/iceberg/TestScanTaskSerialization.java | 0 .../TestSparkDistributedDataScanDeletes.java | 0 .../TestSparkDistributedDataScanFilterFiles.java | 0 ...stSparkDistributedDataScanJavaSerialization.java | 0 ...stSparkDistributedDataScanKryoSerialization.java | 0 .../TestSparkDistributedDataScanReporting.java | 0 .../org/apache/iceberg/TestTableSerialization.java | 0 .../java/org/apache/iceberg/ValidationHelpers.java | 0 .../org/apache/iceberg/spark/CatalogTestBase.java | 0 .../java/org/apache/iceberg/spark/Employee.java | 0 .../apache/iceberg/spark/SparkCatalogConfig.java | 0 .../apache/iceberg/spark/SparkTestHelperBase.java | 0 .../iceberg/spark/SystemFunctionPushDownHelper.java | 0 .../java/org/apache/iceberg/spark/TestBase.java | 0 .../apache/iceberg/spark/TestBaseWithCatalog.java | 0 .../apache/iceberg/spark/TestChangelogIterator.java | 0 .../iceberg/spark/TestFileRewriteCoordinator.java | 0 .../apache/iceberg/spark/TestFunctionCatalog.java | 0 .../org/apache/iceberg/spark/TestSpark3Util.java | 0 .../iceberg/spark/TestSparkCachedTableCatalog.java | 0 .../iceberg/spark/TestSparkCatalogOperations.java | 0 .../iceberg/spark/TestSparkCompressionUtil.java | 0 .../spark/TestSparkDistributionAndOrderingUtil.java | 0 .../iceberg/spark/TestSparkExecutorCache.java | 0 .../org/apache/iceberg/spark/TestSparkFilters.java | 0 .../apache/iceberg/spark/TestSparkSchemaUtil.java | 0 .../iceberg/spark/TestSparkSessionCatalog.java | 0 .../apache/iceberg/spark/TestSparkTableUtil.java | 0 .../apache/iceberg/spark/TestSparkV2Filters.java | 0 .../iceberg/spark/TestSparkValueConverter.java | 0 .../apache/iceberg/spark/TestSparkWriteConf.java | 0 .../actions/TestComputePartitionStatsAction.java | 0 .../spark/actions/TestComputeTableStatsAction.java | 0 .../iceberg/spark/actions/TestCreateActions.java | 0 .../actions/TestDeleteReachableFilesAction.java | 0 .../spark/actions/TestExpireSnapshotsAction.java | 0 .../spark/actions/TestMigrateTableAction.java | 0 .../actions/TestRemoveDanglingDeleteAction.java | 0 .../spark/actions/TestRemoveOrphanFilesAction.java | 0 .../spark/actions/TestRemoveOrphanFilesAction3.java | 0 .../spark/actions/TestRewriteDataFilesAction.java | 0 .../spark/actions/TestRewriteManifestsAction.java | 0 .../TestRewritePositionDeleteFilesAction.java | 0 .../spark/actions/TestRewriteTablePathsAction.java | 0 .../spark/actions/TestSnapshotTableAction.java | 0 .../spark/actions/TestSparkFileRewriteRunners.java | 0 .../TestSparkShufflingDataRewritePlanner.java | 0 .../apache/iceberg/spark/data/AvroDataTestBase.java | 0 .../apache/iceberg/spark/data/GenericsHelpers.java | 0 .../org/apache/iceberg/spark/data/RandomData.java | 0 .../org/apache/iceberg/spark/data/TestHelpers.java | 0 .../org/apache/iceberg/spark/data/TestOrcWrite.java | 0 .../iceberg/spark/data/TestParquetAvroReader.java | 0 .../iceberg/spark/data/TestParquetAvroWriter.java | 0 .../iceberg/spark/data/TestSparkAvroEnums.java | 0 .../iceberg/spark/data/TestSparkAvroReader.java | 0 .../iceberg/spark/data/TestSparkDateTimes.java | 0 .../spark/data/TestSparkOrcReadMetadataColumns.java | 0 .../iceberg/spark/data/TestSparkOrcReader.java | 0 .../data/TestSparkParquetReadMetadataColumns.java | 0 .../iceberg/spark/data/TestSparkParquetReader.java | 0 .../iceberg/spark/data/TestSparkParquetWriter.java | 0 .../spark/data/TestSparkRecordOrcReaderWriter.java | 0 .../iceberg/spark/data/TestSparkVariants.java | 0 .../spark/data/TestVectorizedOrcDataReader.java | 0 .../data/vectorized/TestColumnarBatchUtil.java | 0 ...TestParquetDictionaryEncodedVectorizedReads.java | 0 ...onaryFallbackToPlainEncodingVectorizedReads.java | 0 .../parquet/TestParquetVectorizedReads.java | 0 .../iceberg/spark/functions/TestSparkFunctions.java | 0 .../iceberg/spark/source/BatchReaderUtil.java | 0 .../apache/iceberg/spark/source/ComplexRecord.java | 0 .../spark/source/DataFrameWriteTestBase.java | 0 .../spark/source/FilePathLastModifiedRecord.java | 0 .../iceberg/spark/source/FourColumnRecord.java | 0 .../org/apache/iceberg/spark/source/LogMessage.java | 0 .../apache/iceberg/spark/source/ManualSource.java | 0 .../apache/iceberg/spark/source/NestedRecord.java | 0 .../apache/iceberg/spark/source/ScanTestBase.java | 0 .../apache/iceberg/spark/source/SimpleRecord.java | 0 .../spark/source/SparkSQLExecutionHelper.java | 0 .../spark/source/TestAvroDataFrameWrite.java | 0 .../apache/iceberg/spark/source/TestAvroScan.java | 0 .../apache/iceberg/spark/source/TestBaseReader.java | 0 .../iceberg/spark/source/TestChangelogReader.java | 0 .../spark/source/TestCompressionSettings.java | 0 .../iceberg/spark/source/TestDataFrameWriterV2.java | 0 .../spark/source/TestDataFrameWriterV2Coercion.java | 0 .../iceberg/spark/source/TestDataSourceOptions.java | 0 .../iceberg/spark/source/TestFilteredScan.java | 0 .../spark/source/TestForwardCompatibility.java | 0 .../iceberg/spark/source/TestIcebergSource.java | 0 .../spark/source/TestIcebergSourceHadoopTables.java | 0 .../spark/source/TestIcebergSourceHiveTables.java | 0 .../spark/source/TestIcebergSourceTablesBase.java | 0 .../iceberg/spark/source/TestIcebergSpark.java | 0 .../spark/source/TestIdentityPartitionData.java | 0 .../spark/source/TestInternalRowWrapper.java | 0 .../source/TestMetadataTableReadableMetrics.java | 0 .../TestMetadataTablesWithPartitionEvolution.java | 0 .../iceberg/spark/source/TestORCDataFrameWrite.java | 0 .../source/TestParquetCometVectorizedScan.java | 0 .../spark/source/TestParquetDataFrameWrite.java | 0 .../iceberg/spark/source/TestParquetScan.java | 0 .../spark/source/TestParquetVectorizedScan.java | 0 .../iceberg/spark/source/TestPartitionPruning.java | 0 .../iceberg/spark/source/TestPartitionValues.java | 0 .../iceberg/spark/source/TestPathIdentifier.java | 0 .../spark/source/TestPositionDeletesReader.java | 0 .../spark/source/TestPositionDeletesTable.java | 0 .../iceberg/spark/source/TestReadProjection.java | 0 .../source/TestRequiredDistributionAndOrdering.java | 0 .../iceberg/spark/source/TestRuntimeFiltering.java | 0 .../iceberg/spark/source/TestSnapshotSelection.java | 0 .../iceberg/spark/source/TestSparkAggregates.java | 0 .../iceberg/spark/source/TestSparkCatalog.java | 0 .../source/TestSparkCatalogCacheExpiration.java | 0 .../source/TestSparkCatalogHadoopOverrides.java | 0 .../iceberg/spark/source/TestSparkDVWriters.java | 0 .../iceberg/spark/source/TestSparkDataFile.java | 0 .../iceberg/spark/source/TestSparkDataWrite.java | 0 .../spark/source/TestSparkFileWriterFactory.java | 0 .../spark/source/TestSparkMetadataColumns.java | 0 .../spark/source/TestSparkPartitioningWriters.java | 0 .../iceberg/spark/source/TestSparkPlanningUtil.java | 0 .../spark/source/TestSparkPositionDeltaWriters.java | 0 .../iceberg/spark/source/TestSparkReadMetrics.java | 0 .../spark/source/TestSparkReadProjection.java | 0 .../spark/source/TestSparkReaderDeletes.java | 0 .../source/TestSparkReaderWithBloomFilter.java | 0 .../spark/source/TestSparkRollingFileWriters.java | 0 .../apache/iceberg/spark/source/TestSparkScan.java | 0 .../iceberg/spark/source/TestSparkStagedScan.java | 0 .../apache/iceberg/spark/source/TestSparkTable.java | 0 .../spark/source/TestSparkWriterMetrics.java | 0 .../iceberg/spark/source/TestStreamingOffset.java | 0 .../spark/source/TestStructInternalRowVariant.java | 0 .../spark/source/TestStructuredStreaming.java | 0 .../spark/source/TestStructuredStreamingRead3.java | 0 .../org/apache/iceberg/spark/source/TestTables.java | 0 .../spark/source/TestTimestampWithoutZone.java | 0 .../spark/source/TestWriteMetricsConfig.java | 0 .../iceberg/spark/source/ThreeColumnRecord.java | 0 .../spark/sql/PartitionedWritesTestBase.java | 0 .../iceberg/spark/sql/TestAggregatePushDown.java | 0 .../apache/iceberg/spark/sql/TestAlterTable.java | 0 .../iceberg/spark/sql/TestCTASEncryption.java | 0 .../apache/iceberg/spark/sql/TestCreateTable.java | 0 .../iceberg/spark/sql/TestCreateTableAsSelect.java | 0 .../apache/iceberg/spark/sql/TestDeleteFrom.java | 0 .../org/apache/iceberg/spark/sql/TestDropTable.java | 0 .../iceberg/spark/sql/TestFilterPushDown.java | 0 .../apache/iceberg/spark/sql/TestNamespaceSQL.java | 0 .../iceberg/spark/sql/TestPartitionedWrites.java | 0 .../spark/sql/TestPartitionedWritesAsSelect.java | 0 .../spark/sql/TestPartitionedWritesToBranch.java | 0 .../spark/sql/TestPartitionedWritesToWapBranch.java | 0 .../apache/iceberg/spark/sql/TestRefreshTable.java | 0 .../org/apache/iceberg/spark/sql/TestSelect.java | 0 .../iceberg/spark/sql/TestSparkBucketFunction.java | 0 .../iceberg/spark/sql/TestSparkDaysFunction.java | 0 .../iceberg/spark/sql/TestSparkDefaultValues.java | 0 .../iceberg/spark/sql/TestSparkHoursFunction.java | 0 .../iceberg/spark/sql/TestSparkMonthsFunction.java | 0 .../spark/sql/TestSparkTruncateFunction.java | 0 .../iceberg/spark/sql/TestSparkVariantRead.java | 0 .../iceberg/spark/sql/TestSparkYearsFunction.java | 0 .../spark/sql/TestStoragePartitionedJoins.java | 0 .../iceberg/spark/sql/TestTableEncryption.java | 0 .../iceberg/spark/sql/TestTimestampWithoutZone.java | 0 .../iceberg/spark/sql/TestUnpartitionedWrites.java | 0 .../spark/sql/TestUnpartitionedWritesToBranch.java | 0 .../spark/sql/UnpartitionedWritesTestBase.java | 0 .../decimal_dict_and_plain_encoding.parquet | Bin 574 files changed, 0 insertions(+), 0 deletions(-) rename spark/{v4.0 => v4.1}/build.gradle (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/jmh/java/org/apache/iceberg/spark/MergeCardinalityCheckBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/jmh/java/org/apache/iceberg/spark/UpdateProjectionBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/TestExtendedParser.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ProcedureUtil.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputePartitionStatsProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteWithLineage.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadMerge.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadWithLineage.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java (100%) rename spark/{v4.0 => v4.1}/spark-extensions/src/test/java/org/apache/iceberg/spark/source/PlanUtils.java (100%) rename spark/{v4.0 => v4.1}/spark-runtime/LICENSE (100%) rename spark/{v4.0 => v4.1}/spark-runtime/NOTICE (100%) rename spark/{v4.0 => v4.1}/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/SparkBenchmarkUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/Action.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceDeleteBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceFlatDataBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedDataBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedListDataBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/AvroWritersBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetEqDeleteBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/ParquetWritersBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadParquetDecimalBenchmark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/SparkDistributedDataScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/BaseFileRewriteCoordinator.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/ExtendedParser.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/IcebergSpark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/JobGroupInfo.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/JobGroupUtils.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/OrcBatchReadConf.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/PathIdentifier.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/PositionDeletesRewriteCoordinator.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/RollbackStagedTable.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SortOrderToSpark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkAggregates.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkCachedTableCatalog.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkCompressionUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkDeleteFile.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkExceptionUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkExecutorCache.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTypes.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkFunctionCatalog.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkTableCache.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/SupportsReplaceView.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSnapshotUpdateSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/BaseTableCreationSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/ComputePartitionStatsSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteReachableFilesSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/FileInfo.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/SetAccumulator.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackFileRewriteRunner.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/SparkDataFileRewriteRunner.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewritePositionDeleteRunner.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewriteRunner.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewritePlanner.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortFileRewriteRunner.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderFileRewriteRunner.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderUDF.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/AvroWithSparkSchemaVisitor.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroWriter.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriters.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessorFactory.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorBuilder.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorWithFilter.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/RowPositionColumnVector.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/functions/SparkFunctions.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/functions/UnaryUnboundFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/AncestorsOfProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/CherrypickSnapshotProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputePartitionStatsProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/FastForwardBranchProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/PublishChangesProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/RegisterTableProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteDataFilesProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/RewritePositionDeleteFilesProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteTablePathProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToSnapshotProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToTimestampProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/SetCurrentSnapshotProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/DVIterator.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/ExtractRowLineage.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/HasIcebergCatalog.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SerializableTableWithSize.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogTable.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnarReaderFactory.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkInputPartition.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkLocalScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkMetadataColumn.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkPlanningUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowReaderFactory.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkView.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/StagedSparkTable.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/EqualityDeleteFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/IndexedDeleteFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/PositionalDeleteFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDeleteFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDeleteManifests.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteManifests.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskEqualityDeleteFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskIndexedDeleteFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumDeletes.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumSplits.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskPositionalDeleteFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDeleteFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDeleteManifests.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteManifests.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataFileSize.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataManifests.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteFileSize.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteManifests.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataFileSize.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataManifests.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteFileSize.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteManifests.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/IcebergAnalysisException.java (100%) rename spark/{v4.0 => v4.1}/spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister (100%) rename spark/{v4.0 => v4.1}/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala (100%) rename spark/{v4.0 => v4.1}/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala (100%) rename spark/{v4.0 => v4.1}/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala (100%) rename spark/{v4.0 => v4.1}/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala (100%) rename spark/{v4.0 => v4.1}/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala (100%) rename spark/{v4.0 => v4.1}/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala (100%) rename spark/{v4.0 => v4.1}/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/KryoHelpers.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/TestFileIOSerialization.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/TestHadoopMetricsContextSerialization.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/TestTableSerialization.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/Employee.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/SystemFunctionPushDownHelper.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestBase.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestSparkCachedTableCatalog.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestSparkCompressionUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestSparkSessionCatalog.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestSparkValueConverter.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriteRunners.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkShufflingDataRewritePlanner.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestOrcWrite.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroWriter.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkDateTimes.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkVariants.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/TestVectorizedOrcDataReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/TestColumnarBatchUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryFallbackToPlainEncodingVectorizedReads.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/ComplexRecord.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/FilePathLastModifiedRecord.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/FourColumnRecord.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/NestedRecord.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/SimpleRecord.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/SparkSQLExecutionHelper.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroDataFrameWrite.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSource.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetVectorizedScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesReader.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDVWriters.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPlanningUtil.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkWriterMetrics.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestStructInternalRowVariant.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/source/ThreeColumnRecord.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesAsSelect.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToBranch.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDefaultValues.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkVariantRead.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java (100%) rename spark/{v4.0 => v4.1}/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet (100%) diff --git a/spark/v4.0/build.gradle b/spark/v4.1/build.gradle similarity index 100% rename from spark/v4.0/build.gradle rename to spark/v4.1/build.gradle diff --git a/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java b/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java similarity index 100% rename from spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java rename to spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java diff --git a/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/MergeCardinalityCheckBenchmark.java b/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/MergeCardinalityCheckBenchmark.java similarity index 100% rename from spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/MergeCardinalityCheckBenchmark.java rename to spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/MergeCardinalityCheckBenchmark.java diff --git a/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java b/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java similarity index 100% rename from spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java rename to spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java diff --git a/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java b/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java similarity index 100% rename from spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java rename to spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java diff --git a/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/UpdateProjectionBenchmark.java b/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/UpdateProjectionBenchmark.java similarity index 100% rename from spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/UpdateProjectionBenchmark.java rename to spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/UpdateProjectionBenchmark.java diff --git a/spark/v4.0/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 b/spark/v4.1/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 similarity index 100% rename from spark/v4.0/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 rename to spark/v4.1/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala similarity index 100% rename from spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala rename to spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/TestExtendedParser.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/TestExtendedParser.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/TestExtendedParser.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/TestExtendedParser.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ProcedureUtil.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ProcedureUtil.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ProcedureUtil.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ProcedureUtil.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputePartitionStatsProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputePartitionStatsProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputePartitionStatsProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputePartitionStatsProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteWithLineage.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteWithLineage.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteWithLineage.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteWithLineage.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadMerge.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadMerge.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadMerge.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadMerge.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadWithLineage.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadWithLineage.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadWithLineage.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadWithLineage.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/source/PlanUtils.java rename to spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/source/PlanUtils.java diff --git a/spark/v4.0/spark-runtime/LICENSE b/spark/v4.1/spark-runtime/LICENSE similarity index 100% rename from spark/v4.0/spark-runtime/LICENSE rename to spark/v4.1/spark-runtime/LICENSE diff --git a/spark/v4.0/spark-runtime/NOTICE b/spark/v4.1/spark-runtime/NOTICE similarity index 100% rename from spark/v4.0/spark-runtime/NOTICE rename to spark/v4.1/spark-runtime/NOTICE diff --git a/spark/v4.0/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java b/spark/v4.1/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java similarity index 100% rename from spark/v4.0/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java rename to spark/v4.1/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/SparkBenchmarkUtil.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/SparkBenchmarkUtil.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/SparkBenchmarkUtil.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/SparkBenchmarkUtil.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/Action.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/Action.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/Action.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/Action.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceDeleteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceDeleteBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceDeleteBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceDeleteBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceFlatDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceFlatDataBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceFlatDataBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceFlatDataBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedDataBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedDataBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedDataBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedListDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedListDataBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedListDataBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedListDataBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/AvroWritersBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/AvroWritersBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/AvroWritersBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/AvroWritersBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetEqDeleteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetEqDeleteBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetEqDeleteBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetEqDeleteBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/ParquetWritersBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/ParquetWritersBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/ParquetWritersBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/ParquetWritersBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadParquetDecimalBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadParquetDecimalBenchmark.java similarity index 100% rename from spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadParquetDecimalBenchmark.java rename to spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadParquetDecimalBenchmark.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/SparkDistributedDataScan.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/SparkDistributedDataScan.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/SparkDistributedDataScan.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/SparkDistributedDataScan.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/BaseFileRewriteCoordinator.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/BaseFileRewriteCoordinator.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/BaseFileRewriteCoordinator.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/BaseFileRewriteCoordinator.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ExtendedParser.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ExtendedParser.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ExtendedParser.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ExtendedParser.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/IcebergSpark.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/IcebergSpark.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/IcebergSpark.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/IcebergSpark.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/JobGroupInfo.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/JobGroupInfo.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/JobGroupInfo.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/JobGroupInfo.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/JobGroupUtils.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/JobGroupUtils.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/JobGroupUtils.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/JobGroupUtils.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/OrcBatchReadConf.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/OrcBatchReadConf.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/OrcBatchReadConf.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/OrcBatchReadConf.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PathIdentifier.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/PathIdentifier.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PathIdentifier.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/PathIdentifier.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PositionDeletesRewriteCoordinator.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/PositionDeletesRewriteCoordinator.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PositionDeletesRewriteCoordinator.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/PositionDeletesRewriteCoordinator.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RollbackStagedTable.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/RollbackStagedTable.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RollbackStagedTable.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/RollbackStagedTable.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SortOrderToSpark.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SortOrderToSpark.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SortOrderToSpark.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SortOrderToSpark.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkAggregates.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkAggregates.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkAggregates.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkAggregates.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCachedTableCatalog.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkCachedTableCatalog.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCachedTableCatalog.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkCachedTableCatalog.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCompressionUtil.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkCompressionUtil.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCompressionUtil.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkCompressionUtil.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkDeleteFile.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkDeleteFile.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkDeleteFile.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkDeleteFile.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkExceptionUtil.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkExceptionUtil.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkExceptionUtil.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkExceptionUtil.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkExecutorCache.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkExecutorCache.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkExecutorCache.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkExecutorCache.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTypes.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTypes.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTypes.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTypes.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFunctionCatalog.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkFunctionCatalog.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFunctionCatalog.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkFunctionCatalog.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTableCache.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkTableCache.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTableCache.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkTableCache.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SupportsReplaceView.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SupportsReplaceView.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SupportsReplaceView.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SupportsReplaceView.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSnapshotUpdateSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSnapshotUpdateSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSnapshotUpdateSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSnapshotUpdateSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseTableCreationSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseTableCreationSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseTableCreationSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseTableCreationSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ComputePartitionStatsSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/ComputePartitionStatsSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ComputePartitionStatsSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/ComputePartitionStatsSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteReachableFilesSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteReachableFilesSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteReachableFilesSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteReachableFilesSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/FileInfo.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/FileInfo.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/FileInfo.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/FileInfo.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SetAccumulator.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SetAccumulator.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SetAccumulator.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SetAccumulator.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackFileRewriteRunner.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackFileRewriteRunner.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackFileRewriteRunner.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackFileRewriteRunner.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkDataFileRewriteRunner.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkDataFileRewriteRunner.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkDataFileRewriteRunner.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkDataFileRewriteRunner.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewritePositionDeleteRunner.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewritePositionDeleteRunner.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewritePositionDeleteRunner.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewritePositionDeleteRunner.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewriteRunner.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewriteRunner.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewriteRunner.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewriteRunner.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewritePlanner.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewritePlanner.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewritePlanner.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewritePlanner.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortFileRewriteRunner.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortFileRewriteRunner.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortFileRewriteRunner.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortFileRewriteRunner.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderFileRewriteRunner.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderFileRewriteRunner.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderFileRewriteRunner.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderFileRewriteRunner.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderUDF.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderUDF.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderUDF.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderUDF.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/AvroWithSparkSchemaVisitor.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/AvroWithSparkSchemaVisitor.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/AvroWithSparkSchemaVisitor.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/AvroWithSparkSchemaVisitor.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroWriter.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroWriter.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroWriter.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroWriter.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriters.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriters.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriters.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriters.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessorFactory.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessorFactory.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessorFactory.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessorFactory.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorBuilder.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorBuilder.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorBuilder.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorBuilder.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorWithFilter.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorWithFilter.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorWithFilter.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorWithFilter.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchUtil.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchUtil.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchUtil.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchUtil.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/RowPositionColumnVector.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/RowPositionColumnVector.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/RowPositionColumnVector.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/RowPositionColumnVector.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/SparkFunctions.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/SparkFunctions.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/SparkFunctions.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/SparkFunctions.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/UnaryUnboundFunction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/UnaryUnboundFunction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/UnaryUnboundFunction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/UnaryUnboundFunction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/AncestorsOfProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/AncestorsOfProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/AncestorsOfProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/AncestorsOfProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/CherrypickSnapshotProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/CherrypickSnapshotProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/CherrypickSnapshotProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/CherrypickSnapshotProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputePartitionStatsProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputePartitionStatsProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputePartitionStatsProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputePartitionStatsProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/FastForwardBranchProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/FastForwardBranchProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/FastForwardBranchProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/FastForwardBranchProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/PublishChangesProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/PublishChangesProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/PublishChangesProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/PublishChangesProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RegisterTableProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RegisterTableProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RegisterTableProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RegisterTableProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteDataFilesProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteDataFilesProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteDataFilesProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteDataFilesProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewritePositionDeleteFilesProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RewritePositionDeleteFilesProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewritePositionDeleteFilesProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RewritePositionDeleteFilesProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteTablePathProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteTablePathProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteTablePathProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteTablePathProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToSnapshotProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToSnapshotProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToSnapshotProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToSnapshotProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToTimestampProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToTimestampProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToTimestampProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToTimestampProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SetCurrentSnapshotProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/SetCurrentSnapshotProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SetCurrentSnapshotProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/SetCurrentSnapshotProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/DVIterator.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/DVIterator.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/DVIterator.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/DVIterator.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/ExtractRowLineage.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/ExtractRowLineage.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/ExtractRowLineage.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/ExtractRowLineage.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/HasIcebergCatalog.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/HasIcebergCatalog.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/HasIcebergCatalog.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/HasIcebergCatalog.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SerializableTableWithSize.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SerializableTableWithSize.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SerializableTableWithSize.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SerializableTableWithSize.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogTable.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogTable.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnarReaderFactory.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnarReaderFactory.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnarReaderFactory.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnarReaderFactory.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkInputPartition.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkInputPartition.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkLocalScan.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkLocalScan.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkLocalScan.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkLocalScan.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkMetadataColumn.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkMetadataColumn.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkMetadataColumn.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkMetadataColumn.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPlanningUtil.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPlanningUtil.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPlanningUtil.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPlanningUtil.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowReaderFactory.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowReaderFactory.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowReaderFactory.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowReaderFactory.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkView.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkView.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkView.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkView.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StagedSparkTable.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/StagedSparkTable.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/EqualityDeleteFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/EqualityDeleteFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/EqualityDeleteFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/EqualityDeleteFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/IndexedDeleteFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/IndexedDeleteFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/IndexedDeleteFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/IndexedDeleteFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/PositionalDeleteFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/PositionalDeleteFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/PositionalDeleteFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/PositionalDeleteFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDeleteFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDeleteFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDeleteFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDeleteFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDeleteManifests.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDeleteManifests.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDeleteManifests.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDeleteManifests.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteManifests.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteManifests.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteManifests.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteManifests.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskEqualityDeleteFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskEqualityDeleteFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskEqualityDeleteFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskEqualityDeleteFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskIndexedDeleteFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskIndexedDeleteFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskIndexedDeleteFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskIndexedDeleteFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumDeletes.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumDeletes.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumDeletes.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumDeletes.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumSplits.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumSplits.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumSplits.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumSplits.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskPositionalDeleteFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskPositionalDeleteFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskPositionalDeleteFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskPositionalDeleteFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDeleteFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDeleteFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDeleteFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDeleteFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDeleteManifests.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDeleteManifests.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDeleteManifests.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDeleteManifests.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteFiles.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteFiles.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteFiles.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteFiles.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteManifests.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteManifests.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteManifests.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteManifests.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataFileSize.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataFileSize.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataFileSize.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataFileSize.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataManifests.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataManifests.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataManifests.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataManifests.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteFileSize.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteFileSize.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteFileSize.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteFileSize.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteManifests.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteManifests.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteManifests.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteManifests.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataFileSize.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataFileSize.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataFileSize.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataFileSize.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataManifests.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataManifests.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataManifests.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataManifests.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteFileSize.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteFileSize.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteFileSize.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteFileSize.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteManifests.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteManifests.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteManifests.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteManifests.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java rename to spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java diff --git a/spark/v4.0/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/IcebergAnalysisException.java b/spark/v4.1/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/IcebergAnalysisException.java similarity index 100% rename from spark/v4.0/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/IcebergAnalysisException.java rename to spark/v4.1/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/IcebergAnalysisException.java diff --git a/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/spark/v4.1/spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister similarity index 100% rename from spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister rename to spark/v4.1/spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala b/spark/v4.1/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala similarity index 100% rename from spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala rename to spark/v4.1/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala b/spark/v4.1/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala similarity index 100% rename from spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala rename to spark/v4.1/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala b/spark/v4.1/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala similarity index 100% rename from spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala rename to spark/v4.1/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala b/spark/v4.1/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala similarity index 100% rename from spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala rename to spark/v4.1/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala b/spark/v4.1/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala similarity index 100% rename from spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala rename to spark/v4.1/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala b/spark/v4.1/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala similarity index 100% rename from spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala rename to spark/v4.1/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala b/spark/v4.1/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala similarity index 100% rename from spark/v4.0/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala rename to spark/v4.1/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/KryoHelpers.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/KryoHelpers.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/KryoHelpers.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/KryoHelpers.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestFileIOSerialization.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/TestFileIOSerialization.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/TestFileIOSerialization.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/TestFileIOSerialization.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestHadoopMetricsContextSerialization.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/TestHadoopMetricsContextSerialization.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/TestHadoopMetricsContextSerialization.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/TestHadoopMetricsContextSerialization.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestTableSerialization.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/TestTableSerialization.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/TestTableSerialization.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/TestTableSerialization.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/Employee.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/Employee.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/Employee.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/Employee.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SystemFunctionPushDownHelper.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/SystemFunctionPushDownHelper.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SystemFunctionPushDownHelper.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/SystemFunctionPushDownHelper.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestBase.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestBase.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestBase.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestBase.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCachedTableCatalog.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkCachedTableCatalog.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCachedTableCatalog.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkCachedTableCatalog.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCompressionUtil.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkCompressionUtil.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCompressionUtil.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkCompressionUtil.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkSessionCatalog.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkSessionCatalog.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkSessionCatalog.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkSessionCatalog.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkValueConverter.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkValueConverter.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkValueConverter.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkValueConverter.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriteRunners.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriteRunners.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriteRunners.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriteRunners.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkShufflingDataRewritePlanner.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkShufflingDataRewritePlanner.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkShufflingDataRewritePlanner.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkShufflingDataRewritePlanner.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestOrcWrite.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestOrcWrite.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestOrcWrite.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestOrcWrite.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroReader.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroReader.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroReader.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroReader.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroWriter.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroWriter.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroWriter.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroWriter.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkDateTimes.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkDateTimes.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkDateTimes.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkDateTimes.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkVariants.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkVariants.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkVariants.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkVariants.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestVectorizedOrcDataReader.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestVectorizedOrcDataReader.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestVectorizedOrcDataReader.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestVectorizedOrcDataReader.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/TestColumnarBatchUtil.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/TestColumnarBatchUtil.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/TestColumnarBatchUtil.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/TestColumnarBatchUtil.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryFallbackToPlainEncodingVectorizedReads.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryFallbackToPlainEncodingVectorizedReads.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryFallbackToPlainEncodingVectorizedReads.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryFallbackToPlainEncodingVectorizedReads.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ComplexRecord.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/ComplexRecord.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ComplexRecord.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/ComplexRecord.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/FilePathLastModifiedRecord.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/FilePathLastModifiedRecord.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/FilePathLastModifiedRecord.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/FilePathLastModifiedRecord.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/FourColumnRecord.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/FourColumnRecord.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/FourColumnRecord.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/FourColumnRecord.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/NestedRecord.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/NestedRecord.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/NestedRecord.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/NestedRecord.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/SimpleRecord.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/SimpleRecord.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/SimpleRecord.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/SimpleRecord.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/SparkSQLExecutionHelper.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/SparkSQLExecutionHelper.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/SparkSQLExecutionHelper.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/SparkSQLExecutionHelper.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroDataFrameWrite.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroDataFrameWrite.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroDataFrameWrite.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroDataFrameWrite.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSource.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSource.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSource.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSource.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetVectorizedScan.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetVectorizedScan.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetVectorizedScan.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetVectorizedScan.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesReader.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesReader.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesReader.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesReader.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDVWriters.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDVWriters.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDVWriters.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDVWriters.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPlanningUtil.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPlanningUtil.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPlanningUtil.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPlanningUtil.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkWriterMetrics.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkWriterMetrics.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkWriterMetrics.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkWriterMetrics.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructInternalRowVariant.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructInternalRowVariant.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructInternalRowVariant.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructInternalRowVariant.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ThreeColumnRecord.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/ThreeColumnRecord.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ThreeColumnRecord.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/ThreeColumnRecord.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesAsSelect.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesAsSelect.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesAsSelect.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesAsSelect.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToBranch.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToBranch.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToBranch.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToBranch.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDefaultValues.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDefaultValues.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDefaultValues.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDefaultValues.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkVariantRead.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkVariantRead.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkVariantRead.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkVariantRead.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java diff --git a/spark/v4.0/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 similarity index 100% rename from spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java rename to spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java diff --git a/spark/v4.0/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet b/spark/v4.1/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet similarity index 100% rename from spark/v4.0/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet rename to spark/v4.1/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet From ddea5658ece8ba5aff6f22d8e3477a05064175d1 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Tue, 23 Dec 2025 09:36:43 +0800 Subject: [PATCH 128/201] Spark: Copy back 4.1 as 4.0 --- spark/v4.0/build.gradle | 349 ++ .../iceberg/DeleteFileIndexBenchmark.java | 279 ++ .../spark/MergeCardinalityCheckBenchmark.java | 231 ++ .../iceberg/spark/PlanningBenchmark.java | 409 +++ .../spark/TaskGroupPlanningBenchmark.java | 273 ++ .../spark/UpdateProjectionBenchmark.java | 213 ++ .../IcebergSqlExtensions.g4 | 367 ++ .../IcebergSparkSessionExtensions.scala | 44 + .../sql/catalyst/analysis/CheckViews.scala | 135 + .../sql/catalyst/analysis/ResolveViews.scala | 185 + .../analysis/RewriteViewCommands.scala | 210 ++ .../sql/catalyst/analysis/ViewUtil.scala | 50 + .../optimizer/ReplaceStaticInvoke.scala | 120 + .../IcebergSparkSqlExtensionsParser.scala | 342 ++ .../IcebergSqlExtensionsAstBuilder.scala | 385 +++ .../plans/logical/AddPartitionField.scala | 33 + .../plans/logical/BranchOptions.scala | 25 + .../plans/logical/CreateOrReplaceBranch.scala | 39 + .../plans/logical/CreateOrReplaceTag.scala | 39 + .../catalyst/plans/logical/DropBranch.scala | 32 + .../plans/logical/DropIdentifierFields.scala | 31 + .../plans/logical/DropPartitionField.scala | 32 + .../sql/catalyst/plans/logical/DropTag.scala | 32 + .../plans/logical/ReplacePartitionField.scala | 38 + .../plans/logical/SetIdentifierFields.scala | 31 + .../catalyst/plans/logical/TagOptions.scala | 21 + .../logical/views/CreateIcebergView.scala | 53 + .../plans/logical/views/DropIcebergView.scala | 27 + .../plans/logical/views/ResolvedV2View.scala | 29 + .../logical/views/ShowIcebergViews.scala | 35 + .../v2/AddPartitionFieldExec.scala | 58 + .../v2/AlterV2ViewSetPropertiesExec.scala | 48 + .../v2/AlterV2ViewUnsetPropertiesExec.scala | 53 + .../v2/CreateOrReplaceBranchExec.scala | 103 + .../v2/CreateOrReplaceTagExec.scala | 86 + .../datasources/v2/CreateV2ViewExec.scala | 128 + .../datasources/v2/DescribeV2ViewExec.scala | 79 + .../datasources/v2/DropBranchExec.scala | 56 + .../v2/DropIdentifierFieldsExec.scala | 68 + .../v2/DropPartitionFieldExec.scala | 68 + .../datasources/v2/DropTagExec.scala | 52 + .../datasources/v2/DropV2ViewExec.scala | 44 + .../v2/ExtendedDataSourceV2Strategy.scala | 192 ++ .../datasources/v2/RenameV2ViewExec.scala | 40 + .../v2/ReplacePartitionFieldExec.scala | 76 + .../v2/SetIdentifierFieldsExec.scala | 52 + .../SetWriteDistributionAndOrderingExec.scala | 85 + .../datasources/v2/ShowCreateV2ViewExec.scala | 78 + .../v2/ShowV2ViewPropertiesExec.scala | 51 + .../datasources/v2/ShowV2ViewsExec.scala | 70 + .../iceberg/spark/TestExtendedParser.java | 231 ++ .../iceberg/spark/extensions/Employee.java | 66 + .../spark/extensions/ExtensionsTestBase.java | 72 + .../spark/extensions/ProcedureUtil.java | 54 + .../spark/extensions/SparkPlanUtil.java | 107 + .../SparkRowLevelOperationsTestBase.java | 461 +++ .../extensions/TestAddFilesProcedure.java | 1479 ++++++++ .../TestAlterTablePartitionFields.java | 650 ++++ .../extensions/TestAlterTableSchema.java | 148 + .../extensions/TestAncestorsOfProcedure.java | 163 + .../spark/extensions/TestBranchDDL.java | 432 +++ .../spark/extensions/TestChangelogTable.java | 453 +++ .../TestCherrypickSnapshotProcedure.java | 194 ++ .../TestComputePartitionStatsProcedure.java | 121 + .../TestComputeTableStatsProcedure.java | 137 + .../extensions/TestConflictValidation.java | 394 +++ .../extensions/TestCopyOnWriteDelete.java | 227 ++ .../extensions/TestCopyOnWriteMerge.java | 195 ++ .../extensions/TestCopyOnWriteUpdate.java | 183 + .../TestCopyOnWriteWithLineage.java | 35 + .../TestCreateChangelogViewProcedure.java | 694 ++++ .../iceberg/spark/extensions/TestDelete.java | 1486 ++++++++ .../TestExpireSnapshotsProcedure.java | 624 ++++ .../TestFastForwardBranchProcedure.java | 264 ++ .../iceberg/spark/extensions/TestMerge.java | 3046 +++++++++++++++++ .../extensions/TestMergeOnReadDelete.java | 352 ++ .../extensions/TestMergeOnReadMerge.java | 173 + .../extensions/TestMergeOnReadUpdate.java | 226 ++ .../TestMergeOnReadWithLineage.java | 35 + ...TestMetaColumnProjectionWithStageScan.java | 125 + .../spark/extensions/TestMetadataTables.java | 993 ++++++ .../extensions/TestMigrateTableProcedure.java | 291 ++ .../TestPublishChangesProcedure.java | 187 + .../TestRegisterTableProcedure.java | 85 + .../extensions/TestRemoteScanPlanning.java | 58 + .../TestRemoveOrphanFilesProcedure.java | 752 ++++ .../spark/extensions/TestReplaceBranch.java | 270 ++ .../TestRequiredDistributionAndOrdering.java | 313 ++ .../TestRewriteDataFilesProcedure.java | 1108 ++++++ .../TestRewriteManifestsProcedure.java | 448 +++ .../TestRewritePositionDeleteFiles.java | 418 +++ ...stRewritePositionDeleteFilesProcedure.java | 250 ++ .../TestRewriteTablePathProcedure.java | 270 ++ .../TestRollbackToSnapshotProcedure.java | 286 ++ .../TestRollbackToTimestampProcedure.java | 339 ++ .../TestRowLevelOperationsWithLineage.java | 631 ++++ .../TestSetCurrentSnapshotProcedure.java | 291 ++ .../TestSetWriteDistributionAndOrdering.java | 339 ++ .../TestSnapshotTableProcedure.java | 378 ++ ...ePartitionedJoinsInRowLevelOperations.java | 316 ++ .../TestSystemFunctionPushDownDQL.java | 313 ++ ...mFunctionPushDownInRowLevelOperations.java | 365 ++ .../iceberg/spark/extensions/TestTagDDL.java | 365 ++ .../iceberg/spark/extensions/TestUpdate.java | 1518 ++++++++ .../iceberg/spark/extensions/TestViews.java | 2121 ++++++++++++ .../spark/extensions/TestWriteAborts.java | 185 + .../iceberg/spark/source/PlanUtils.java | 99 + spark/v4.0/spark-runtime/LICENSE | 530 +++ spark/v4.0/spark-runtime/NOTICE | 403 +++ .../apache/iceberg/spark/TestRoundTrip.java | 182 + .../iceberg/spark/SparkBenchmarkUtil.java | 58 + .../action/DeleteOrphanFilesBenchmark.java | 189 + .../IcebergSortCompactionBenchmark.java | 404 +++ .../spark/action/RandomGeneratingUDF.java | 50 + .../SparkParquetReadersFlatDataBenchmark.java | 229 ++ ...parkParquetReadersNestedDataBenchmark.java | 227 ++ .../SparkParquetWritersFlatDataBenchmark.java | 130 + ...parkParquetWritersNestedDataBenchmark.java | 130 + .../apache/iceberg/spark/source/Action.java | 24 + .../spark/source/DVReaderBenchmark.java | 267 ++ .../spark/source/DVWriterBenchmark.java | 251 ++ .../spark/source/IcebergSourceBenchmark.java | 209 ++ .../source/IcebergSourceDeleteBenchmark.java | 340 ++ .../IcebergSourceFlatDataBenchmark.java | 59 + .../IcebergSourceNestedDataBenchmark.java | 59 + .../IcebergSourceNestedListDataBenchmark.java | 62 + .../spark/source/WritersBenchmark.java | 393 +++ .../source/avro/AvroWritersBenchmark.java | 39 + ...cebergSourceFlatAvroDataReadBenchmark.java | 142 + ...bergSourceNestedAvroDataReadBenchmark.java | 142 + .../IcebergSourceFlatORCDataBenchmark.java | 68 + ...IcebergSourceFlatORCDataReadBenchmark.java | 210 ++ ...SourceNestedListORCDataWriteBenchmark.java | 109 + ...ebergSourceNestedORCDataReadBenchmark.java | 183 + ...gSourceFlatParquetDataFilterBenchmark.java | 129 + ...ergSourceFlatParquetDataReadBenchmark.java | 165 + ...rgSourceFlatParquetDataWriteBenchmark.java | 89 + ...ceNestedListParquetDataWriteBenchmark.java | 89 + ...ourceNestedParquetDataFilterBenchmark.java | 128 + ...gSourceNestedParquetDataReadBenchmark.java | 166 + ...SourceNestedParquetDataWriteBenchmark.java | 88 + ...IcebergSourceParquetEqDeleteBenchmark.java | 57 + ...SourceParquetMultiDeleteFileBenchmark.java | 57 + ...cebergSourceParquetPosDeleteBenchmark.java | 60 + ...ceParquetWithUnrelatedDeleteBenchmark.java | 64 + .../parquet/ParquetWritersBenchmark.java | 39 + ...ionaryEncodedFlatParquetDataBenchmark.java | 127 + ...ectorizedReadFlatParquetDataBenchmark.java | 355 ++ ...VectorizedReadParquetDecimalBenchmark.java | 207 ++ .../iceberg/SparkDistributedDataScan.java | 272 ++ .../org/apache/iceberg/spark/BaseCatalog.java | 94 + .../spark/BaseFileRewriteCoordinator.java | 81 + .../iceberg/spark/ChangelogIterator.java | 127 + .../apache/iceberg/spark/CommitMetadata.java | 66 + .../iceberg/spark/ComputeUpdateIterator.java | 140 + .../apache/iceberg/spark/ExtendedParser.java | 110 + .../iceberg/spark/FileRewriteCoordinator.java | 32 + .../apache/iceberg/spark/IcebergSpark.java | 56 + .../apache/iceberg/spark/JobGroupInfo.java | 48 + .../apache/iceberg/spark/JobGroupUtils.java | 64 + .../iceberg/spark/OrcBatchReadConf.java | 27 + .../iceberg/spark/ParquetBatchReadConf.java | 29 + .../iceberg/spark/ParquetReaderType.java | 47 + .../apache/iceberg/spark/PathIdentifier.java | 57 + .../PositionDeletesRewriteCoordinator.java | 33 + .../spark/PruneColumnsWithoutReordering.java | 249 ++ .../spark/RemoveCarryoverIterator.java | 148 + .../spark/RemoveNetCarryoverIterator.java | 129 + .../iceberg/spark/RollbackStagedTable.java | 142 + .../iceberg/spark/ScanTaskSetManager.java | 73 + .../iceberg/spark/SortOrderToSpark.java | 101 + .../org/apache/iceberg/spark/Spark3Util.java | 1079 ++++++ .../apache/iceberg/spark/SparkAggregates.java | 84 + .../spark/SparkCachedTableCatalog.java | 257 ++ .../apache/iceberg/spark/SparkCatalog.java | 1049 ++++++ .../iceberg/spark/SparkCompressionUtil.java | 148 + .../apache/iceberg/spark/SparkConfParser.java | 320 ++ .../iceberg/spark/SparkContentFile.java | 277 ++ .../apache/iceberg/spark/SparkDataFile.java | 46 + .../apache/iceberg/spark/SparkDeleteFile.java | 40 + .../iceberg/spark/SparkExceptionUtil.java | 64 + .../iceberg/spark/SparkExecutorCache.java | 228 ++ .../apache/iceberg/spark/SparkFilters.java | 280 ++ .../apache/iceberg/spark/SparkFixupTypes.java | 63 + .../iceberg/spark/SparkFunctionCatalog.java | 45 + .../apache/iceberg/spark/SparkReadConf.java | 378 ++ .../iceberg/spark/SparkReadOptions.java | 98 + .../iceberg/spark/SparkSQLProperties.java | 112 + .../apache/iceberg/spark/SparkSchemaUtil.java | 372 ++ .../iceberg/spark/SparkSessionCatalog.java | 542 +++ .../apache/iceberg/spark/SparkStructLike.java | 54 + .../apache/iceberg/spark/SparkTableCache.java | 54 + .../apache/iceberg/spark/SparkTableUtil.java | 1311 +++++++ .../apache/iceberg/spark/SparkTypeToType.java | 171 + .../iceberg/spark/SparkTypeVisitor.java | 86 + .../org/apache/iceberg/spark/SparkUtil.java | 339 ++ .../apache/iceberg/spark/SparkV2Filters.java | 484 +++ .../iceberg/spark/SparkValueConverter.java | 157 + .../apache/iceberg/spark/SparkWriteConf.java | 732 ++++ .../iceberg/spark/SparkWriteOptions.java | 88 + .../iceberg/spark/SparkWriteRequirements.java | 59 + .../apache/iceberg/spark/SparkWriteUtil.java | 259 ++ .../iceberg/spark/SupportsFunctions.java | 65 + .../iceberg/spark/SupportsReplaceView.java | 56 + .../apache/iceberg/spark/TypeToSparkType.java | 166 + .../BaseSnapshotUpdateSparkAction.java | 47 + .../spark/actions/BaseSparkAction.java | 443 +++ .../actions/BaseTableCreationSparkAction.java | 182 + .../ComputePartitionStatsSparkAction.java | 103 + .../actions/ComputeTableStatsSparkAction.java | 179 + .../actions/DeleteOrphanFilesSparkAction.java | 620 ++++ .../DeleteReachableFilesSparkAction.java | 156 + .../actions/ExpireSnapshotsSparkAction.java | 286 ++ .../iceberg/spark/actions/FileInfo.java | 52 + .../spark/actions/ManifestFileBean.java | 191 ++ .../actions/MigrateTableSparkAction.java | 259 ++ .../iceberg/spark/actions/NDVSketchUtil.java | 91 + .../RemoveDanglingDeletesSparkAction.java | 206 ++ .../actions/RewriteDataFilesSparkAction.java | 448 +++ .../actions/RewriteManifestsSparkAction.java | 606 ++++ ...RewritePositionDeleteFilesSparkAction.java | 380 ++ .../actions/RewriteTablePathSparkAction.java | 821 +++++ .../iceberg/spark/actions/SetAccumulator.java | 61 + .../actions/SnapshotTableSparkAction.java | 225 ++ .../iceberg/spark/actions/SparkActions.java | 122 + .../SparkBinPackFileRewriteRunner.java | 71 + .../actions/SparkDataFileRewriteRunner.java | 61 + .../SparkRewritePositionDeleteRunner.java | 150 + .../spark/actions/SparkRewriteRunner.java | 82 + .../SparkShufflingDataRewritePlanner.java | 82 + .../SparkShufflingFileRewriteRunner.java | 228 ++ .../actions/SparkSortFileRewriteRunner.java | 64 + .../actions/SparkZOrderFileRewriteRunner.java | 204 ++ .../iceberg/spark/actions/SparkZOrderUDF.java | 325 ++ .../data/AvroWithSparkSchemaVisitor.java | 81 + .../data/ParquetWithSparkSchemaVisitor.java | 248 ++ .../iceberg/spark/data/SparkAvroWriter.java | 170 + .../iceberg/spark/data/SparkOrcReader.java | 135 + .../spark/data/SparkOrcValueReaders.java | 261 ++ .../spark/data/SparkOrcValueWriters.java | 221 ++ .../iceberg/spark/data/SparkOrcWriter.java | 247 ++ .../spark/data/SparkParquetReaders.java | 782 +++++ .../spark/data/SparkParquetWriters.java | 650 ++++ .../spark/data/SparkPlannedAvroReader.java | 198 ++ .../iceberg/spark/data/SparkValueReaders.java | 352 ++ .../iceberg/spark/data/SparkValueWriters.java | 280 ++ .../ArrowVectorAccessorFactory.java | 140 + .../data/vectorized/ArrowVectorAccessors.java | 38 + .../data/vectorized/ColumnVectorBuilder.java | 45 + .../vectorized/ColumnVectorWithFilter.java | 155 + .../data/vectorized/ColumnarBatchReader.java | 94 + .../data/vectorized/ColumnarBatchUtil.java | 181 + .../data/vectorized/CometColumnReader.java | 140 + .../vectorized/CometColumnarBatchReader.java | 137 + .../vectorized/CometConstantColumnReader.java | 65 + .../vectorized/CometDeleteColumnReader.java | 74 + .../vectorized/CometDeletedColumnVector.java | 155 + .../vectorized/CometPositionColumnReader.java | 62 + .../CometVectorizedReaderBuilder.java | 138 + .../data/vectorized/ConstantColumnVector.java | 141 + .../data/vectorized/DeletedColumnVector.java | 123 + .../vectorized/IcebergArrowColumnVector.java | 164 + .../vectorized/RowPositionColumnVector.java | 120 + .../UpdatableDeletedColumnVector.java | 23 + .../vectorized/VectorizedSparkOrcReaders.java | 492 +++ .../VectorizedSparkParquetReaders.java | 143 + .../spark/functions/BaseScalarFunction.java | 40 + .../spark/functions/BucketFunction.java | 366 ++ .../iceberg/spark/functions/DaysFunction.java | 152 + .../spark/functions/HoursFunction.java | 146 + .../functions/IcebergVersionFunction.java | 95 + .../spark/functions/MonthsFunction.java | 143 + .../spark/functions/SparkFunctions.java | 75 + .../spark/functions/TruncateFunction.java | 357 ++ .../spark/functions/UnaryUnboundFunction.java | 44 + .../spark/functions/YearsFunction.java | 143 + .../spark/procedures/AddFilesProcedure.java | 278 ++ .../procedures/AncestorsOfProcedure.java | 125 + .../spark/procedures/BaseProcedure.java | 289 ++ .../CherrypickSnapshotProcedure.java | 116 + .../ComputePartitionStatsProcedure.java | 127 + .../ComputeTableStatsProcedure.java | 131 + .../CreateChangelogViewProcedure.java | 329 ++ .../procedures/ExpireSnapshotsProcedure.java | 204 ++ .../FastForwardBranchProcedure.java | 109 + .../procedures/MigrateTableProcedure.java | 133 + .../spark/procedures/ProcedureInput.java | 274 ++ .../procedures/PublishChangesProcedure.java | 129 + .../procedures/RegisterTableProcedure.java | 126 + .../RemoveOrphanFilesProcedure.java | 252 ++ .../procedures/RewriteDataFilesProcedure.java | 229 ++ .../procedures/RewriteManifestsProcedure.java | 135 + .../RewritePositionDeleteFilesProcedure.java | 131 + .../procedures/RewriteTablePathProcedure.java | 157 + .../RollbackToSnapshotProcedure.java | 112 + .../RollbackToTimestampProcedure.java | 116 + .../SetCurrentSnapshotProcedure.java | 130 + .../procedures/SnapshotTableProcedure.java | 132 + .../spark/procedures/SparkProcedures.java | 71 + .../iceberg/spark/source/BaseBatchReader.java | 221 ++ .../iceberg/spark/source/BaseReader.java | 256 ++ .../iceberg/spark/source/BaseRowReader.java | 127 + .../iceberg/spark/source/BatchDataReader.java | 126 + .../spark/source/ChangelogRowReader.java | 176 + .../iceberg/spark/source/DVIterator.java | 109 + .../spark/source/EqualityDeleteRowReader.java | 57 + .../spark/source/ExtractRowLineage.java | 92 + .../spark/source/HasIcebergCatalog.java | 30 + .../iceberg/spark/source/IcebergSource.java | 265 ++ .../spark/source/InternalRowWrapper.java | 109 + .../source/PositionDeletesRowReader.java | 120 + .../iceberg/spark/source/RowDataReader.java | 125 + .../source/SerializableTableWithSize.java | 108 + .../iceberg/spark/source/SparkBatch.java | 232 ++ .../spark/source/SparkBatchQueryScan.java | 293 ++ .../spark/source/SparkChangelogScan.java | 174 + .../spark/source/SparkChangelogTable.java | 140 + .../spark/source/SparkCleanupUtil.java | 141 + .../spark/source/SparkColumnStatistics.java | 88 + .../source/SparkColumnarReaderFactory.java | 71 + .../source/SparkCopyOnWriteOperation.java | 115 + .../spark/source/SparkCopyOnWriteScan.java | 191 ++ .../spark/source/SparkFileWriterFactory.java | 343 ++ .../spark/source/SparkInputPartition.java | 106 + .../iceberg/spark/source/SparkLocalScan.java | 66 + .../spark/source/SparkMetadataColumn.java | 130 + .../spark/source/SparkMicroBatchStream.java | 593 ++++ .../source/SparkPartitioningAwareScan.java | 253 ++ .../spark/source/SparkPlanningUtil.java | 93 + .../source/SparkPositionDeletesRewrite.java | 522 +++ .../SparkPositionDeletesRewriteBuilder.java | 108 + .../source/SparkPositionDeltaOperation.java | 125 + .../spark/source/SparkPositionDeltaWrite.java | 937 +++++ .../SparkPositionDeltaWriteBuilder.java | 135 + .../source/SparkRowLevelOperationBuilder.java | 114 + .../spark/source/SparkRowReaderFactory.java | 68 + .../iceberg/spark/source/SparkScan.java | 354 ++ .../spark/source/SparkScanBuilder.java | 778 +++++ .../iceberg/spark/source/SparkStagedScan.java | 98 + .../spark/source/SparkStagedScanBuilder.java | 92 + .../iceberg/spark/source/SparkTable.java | 502 +++ .../iceberg/spark/source/SparkView.java | 151 + .../iceberg/spark/source/SparkWrite.java | 874 +++++ .../spark/source/SparkWriteBuilder.java | 238 ++ .../spark/source/StagedSparkTable.java | 41 + .../apache/iceberg/spark/source/Stats.java | 52 + .../iceberg/spark/source/StreamingOffset.java | 161 + .../spark/source/StructInternalRow.java | 416 +++ .../source/metrics/EqualityDeleteFiles.java | 36 + .../source/metrics/IndexedDeleteFiles.java | 36 + .../spark/source/metrics/NumDeletes.java | 48 + .../spark/source/metrics/NumSplits.java | 46 + .../source/metrics/PositionalDeleteFiles.java | 36 + .../spark/source/metrics/ResultDataFiles.java | 36 + .../source/metrics/ResultDeleteFiles.java | 36 + .../source/metrics/ScannedDataManifests.java | 36 + .../metrics/ScannedDeleteManifests.java | 36 + .../source/metrics/SkippedDataFiles.java | 36 + .../source/metrics/SkippedDataManifests.java | 36 + .../source/metrics/SkippedDeleteFiles.java | 36 + .../metrics/SkippedDeleteManifests.java | 36 + .../metrics/TaskEqualityDeleteFiles.java | 47 + .../metrics/TaskIndexedDeleteFiles.java | 47 + .../spark/source/metrics/TaskNumDeletes.java | 39 + .../spark/source/metrics/TaskNumSplits.java | 39 + .../metrics/TaskPositionalDeleteFiles.java | 47 + .../source/metrics/TaskResultDataFiles.java | 47 + .../source/metrics/TaskResultDeleteFiles.java | 47 + .../metrics/TaskScannedDataManifests.java | 47 + .../metrics/TaskScannedDeleteManifests.java | 47 + .../source/metrics/TaskSkippedDataFiles.java | 47 + .../metrics/TaskSkippedDataManifests.java | 47 + .../metrics/TaskSkippedDeleteFiles.java | 47 + .../metrics/TaskSkippedDeleteManifests.java | 47 + .../source/metrics/TaskTotalDataFileSize.java | 48 + .../metrics/TaskTotalDataManifests.java | 47 + .../metrics/TaskTotalDeleteFileSize.java | 48 + .../metrics/TaskTotalDeleteManifests.java | 47 + .../metrics/TaskTotalPlanningDuration.java | 48 + .../source/metrics/TotalDataFileSize.java | 36 + .../source/metrics/TotalDataManifests.java | 36 + .../source/metrics/TotalDeleteFileSize.java | 36 + .../source/metrics/TotalDeleteManifests.java | 36 + .../source/metrics/TotalPlanningDuration.java | 36 + .../analysis/IcebergAnalysisException.java | 37 + ...pache.spark.sql.sources.DataSourceRegister | 20 + .../plans/logical/OrderAwareCoalesce.scala | 48 + .../SetWriteDistributionAndOrdering.scala | 46 + .../plans/logical/SortOrderParserUtil.scala | 41 + .../spark/sql/catalyst/utils/PlanUtils.scala | 41 + .../execution/OrderAwareCoalesceExec.scala | 59 + .../SparkExpressionConverter.scala | 73 + .../spark/sql/stats/ThetaSketchAgg.scala | 122 + .../java/org/apache/iceberg/KryoHelpers.java | 51 + .../SparkDistributedDataScanTestBase.java | 95 + .../org/apache/iceberg/TaskCheckHelper.java | 108 + .../iceberg/TestDataFileSerialization.java | 175 + .../iceberg/TestFileIOSerialization.java | 106 + ...TestHadoopMetricsContextSerialization.java | 56 + .../TestManifestFileSerialization.java | 221 ++ .../iceberg/TestScanTaskSerialization.java | 197 ++ .../TestSparkDistributedDataScanDeletes.java | 91 + ...stSparkDistributedDataScanFilterFiles.java | 85 + ...kDistributedDataScanJavaSerialization.java | 40 + ...kDistributedDataScanKryoSerialization.java | 40 + ...TestSparkDistributedDataScanReporting.java | 86 + .../iceberg/TestTableSerialization.java | 216 ++ .../org/apache/iceberg/ValidationHelpers.java | 78 + .../apache/iceberg/spark/CatalogTestBase.java | 59 + .../org/apache/iceberg/spark/Employee.java | 66 + .../iceberg/spark/SparkCatalogConfig.java | 87 + .../iceberg/spark/SparkTestHelperBase.java | 95 + .../spark/SystemFunctionPushDownHelper.java | 127 + .../org/apache/iceberg/spark/TestBase.java | 289 ++ .../iceberg/spark/TestBaseWithCatalog.java | 196 ++ .../iceberg/spark/TestChangelogIterator.java | 359 ++ .../spark/TestFileRewriteCoordinator.java | 275 ++ .../iceberg/spark/TestFunctionCatalog.java | 136 + .../apache/iceberg/spark/TestSpark3Util.java | 186 + .../spark/TestSparkCachedTableCatalog.java | 105 + .../spark/TestSparkCatalogOperations.java | 196 ++ .../spark/TestSparkCompressionUtil.java | 162 + .../TestSparkDistributionAndOrderingUtil.java | 3029 ++++++++++++++++ .../iceberg/spark/TestSparkExecutorCache.java | 677 ++++ .../iceberg/spark/TestSparkFilters.java | 201 ++ .../iceberg/spark/TestSparkSchemaUtil.java | 272 ++ .../spark/TestSparkSessionCatalog.java | 101 + .../iceberg/spark/TestSparkTableUtil.java | 106 + .../iceberg/spark/TestSparkV2Filters.java | 853 +++++ .../spark/TestSparkValueConverter.java | 94 + .../iceberg/spark/TestSparkWriteConf.java | 643 ++++ .../TestComputePartitionStatsAction.java | 309 ++ .../actions/TestComputeTableStatsAction.java | 443 +++ .../spark/actions/TestCreateActions.java | 1056 ++++++ .../TestDeleteReachableFilesAction.java | 403 +++ .../actions/TestExpireSnapshotsAction.java | 1407 ++++++++ .../spark/actions/TestMigrateTableAction.java | 68 + .../TestRemoveDanglingDeleteAction.java | 540 +++ .../actions/TestRemoveOrphanFilesAction.java | 1308 +++++++ .../actions/TestRemoveOrphanFilesAction3.java | 194 ++ .../actions/TestRewriteDataFilesAction.java | 2660 ++++++++++++++ .../actions/TestRewriteManifestsAction.java | 1312 +++++++ .../TestRewritePositionDeleteFilesAction.java | 1307 +++++++ .../actions/TestRewriteTablePathsAction.java | 1600 +++++++++ .../actions/TestSnapshotTableAction.java | 68 + .../actions/TestSparkFileRewriteRunners.java | 139 + .../TestSparkShufflingDataRewritePlanner.java | 93 + .../iceberg/spark/data/AvroDataTestBase.java | 728 ++++ .../iceberg/spark/data/GenericsHelpers.java | 441 +++ .../apache/iceberg/spark/data/RandomData.java | 404 +++ .../iceberg/spark/data/TestHelpers.java | 957 ++++++ .../iceberg/spark/data/TestOrcWrite.java | 59 + .../spark/data/TestParquetAvroReader.java | 237 ++ .../spark/data/TestParquetAvroWriter.java | 123 + .../spark/data/TestSparkAvroEnums.java | 97 + .../spark/data/TestSparkAvroReader.java | 91 + .../spark/data/TestSparkDateTimes.java | 78 + .../data/TestSparkOrcReadMetadataColumns.java | 255 ++ .../spark/data/TestSparkOrcReader.java | 127 + .../TestSparkParquetReadMetadataColumns.java | 283 ++ .../spark/data/TestSparkParquetReader.java | 263 ++ .../spark/data/TestSparkParquetWriter.java | 182 + .../data/TestSparkRecordOrcReaderWriter.java | 170 + .../iceberg/spark/data/TestSparkVariants.java | 344 ++ .../data/TestVectorizedOrcDataReader.java | 167 + .../vectorized/TestColumnarBatchUtil.java | 302 ++ ...rquetDictionaryEncodedVectorizedReads.java | 199 ++ ...allbackToPlainEncodingVectorizedReads.java | 77 + .../parquet/TestParquetVectorizedReads.java | 548 +++ .../spark/functions/TestSparkFunctions.java | 158 + .../iceberg/spark/source/BatchReaderUtil.java | 34 + .../iceberg/spark/source/ComplexRecord.java | 74 + .../spark/source/DataFrameWriteTestBase.java | 149 + .../source/FilePathLastModifiedRecord.java | 80 + .../spark/source/FourColumnRecord.java | 106 + .../iceberg/spark/source/LogMessage.java | 119 + .../iceberg/spark/source/ManualSource.java | 75 + .../iceberg/spark/source/NestedRecord.java | 77 + .../iceberg/spark/source/ScanTestBase.java | 138 + .../iceberg/spark/source/SimpleRecord.java | 78 + .../spark/source/SparkSQLExecutionHelper.java | 70 + .../spark/source/TestAvroDataFrameWrite.java | 38 + .../iceberg/spark/source/TestAvroScan.java | 68 + .../iceberg/spark/source/TestBaseReader.java | 289 ++ .../spark/source/TestChangelogReader.java | 263 ++ .../spark/source/TestCompressionSettings.java | 276 ++ .../spark/source/TestDataFrameWriterV2.java | 390 +++ .../source/TestDataFrameWriterV2Coercion.java | 81 + .../spark/source/TestDataSourceOptions.java | 520 +++ .../spark/source/TestFilteredScan.java | 760 ++++ .../source/TestForwardCompatibility.java | 229 ++ .../spark/source/TestIcebergSource.java | 42 + .../source/TestIcebergSourceHadoopTables.java | 68 + .../source/TestIcebergSourceHiveTables.java | 86 + .../source/TestIcebergSourceTablesBase.java | 2459 +++++++++++++ .../spark/source/TestIcebergSpark.java | 295 ++ .../source/TestIdentityPartitionData.java | 251 ++ .../spark/source/TestInternalRowWrapper.java | 81 + .../TestMetadataTableReadableMetrics.java | 379 ++ ...tMetadataTablesWithPartitionEvolution.java | 725 ++++ .../spark/source/TestORCDataFrameWrite.java | 57 + .../TestParquetCometVectorizedScan.java | 33 + .../source/TestParquetDataFrameWrite.java | 62 + .../iceberg/spark/source/TestParquetScan.java | 113 + .../source/TestParquetVectorizedScan.java | 26 + .../spark/source/TestPartitionPruning.java | 489 +++ .../spark/source/TestPartitionValues.java | 511 +++ .../spark/source/TestPathIdentifier.java | 83 + .../source/TestPositionDeletesReader.java | 332 ++ .../source/TestPositionDeletesTable.java | 1779 ++++++++++ .../spark/source/TestReadProjection.java | 654 ++++ .../TestRequiredDistributionAndOrdering.java | 310 ++ .../spark/source/TestRuntimeFiltering.java | 510 +++ .../spark/source/TestSnapshotSelection.java | 590 ++++ .../spark/source/TestSparkAggregates.java | 76 + .../spark/source/TestSparkCatalog.java | 68 + .../TestSparkCatalogCacheExpiration.java | 155 + .../TestSparkCatalogHadoopOverrides.java | 143 + .../spark/source/TestSparkDVWriters.java | 69 + .../spark/source/TestSparkDataFile.java | 350 ++ .../spark/source/TestSparkDataWrite.java | 739 ++++ .../source/TestSparkFileWriterFactory.java | 69 + .../source/TestSparkMetadataColumns.java | 371 ++ .../source/TestSparkPartitioningWriters.java | 69 + .../spark/source/TestSparkPlanningUtil.java | 211 ++ .../source/TestSparkPositionDeltaWriters.java | 69 + .../spark/source/TestSparkReadMetrics.java | 279 ++ .../spark/source/TestSparkReadProjection.java | 273 ++ .../spark/source/TestSparkReaderDeletes.java | 755 ++++ .../TestSparkReaderWithBloomFilter.java | 376 ++ .../source/TestSparkRollingFileWriters.java | 55 + .../iceberg/spark/source/TestSparkScan.java | 1089 ++++++ .../spark/source/TestSparkStagedScan.java | 128 + .../iceberg/spark/source/TestSparkTable.java | 59 + .../spark/source/TestSparkWriterMetrics.java | 67 + .../spark/source/TestStreamingOffset.java | 56 + .../source/TestStructInternalRowVariant.java | 182 + .../spark/source/TestStructuredStreaming.java | 309 ++ .../source/TestStructuredStreamingRead3.java | 942 +++++ .../iceberg/spark/source/TestTables.java | 209 ++ .../source/TestTimestampWithoutZone.java | 229 ++ .../spark/source/TestWriteMetricsConfig.java | 304 ++ .../spark/source/ThreeColumnRecord.java | 83 + .../spark/sql/PartitionedWritesTestBase.java | 282 ++ .../spark/sql/TestAggregatePushDown.java | 866 +++++ .../iceberg/spark/sql/TestAlterTable.java | 333 ++ .../iceberg/spark/sql/TestCTASEncryption.java | 120 + .../iceberg/spark/sql/TestCreateTable.java | 457 +++ .../spark/sql/TestCreateTableAsSelect.java | 451 +++ .../iceberg/spark/sql/TestDeleteFrom.java | 165 + .../iceberg/spark/sql/TestDropTable.java | 158 + .../iceberg/spark/sql/TestFilterPushDown.java | 703 ++++ .../iceberg/spark/sql/TestNamespaceSQL.java | 302 ++ .../spark/sql/TestPartitionedWrites.java | 21 + .../sql/TestPartitionedWritesAsSelect.java | 158 + .../sql/TestPartitionedWritesToBranch.java | 49 + .../sql/TestPartitionedWritesToWapBranch.java | 91 + .../iceberg/spark/sql/TestRefreshTable.java | 86 + .../apache/iceberg/spark/sql/TestSelect.java | 734 ++++ .../spark/sql/TestSparkBucketFunction.java | 341 ++ .../spark/sql/TestSparkDaysFunction.java | 107 + .../spark/sql/TestSparkDefaultValues.java | 234 ++ .../spark/sql/TestSparkHoursFunction.java | 93 + .../spark/sql/TestSparkMonthsFunction.java | 125 + .../spark/sql/TestSparkTruncateFunction.java | 443 +++ .../spark/sql/TestSparkVariantRead.java | 316 ++ .../spark/sql/TestSparkYearsFunction.java | 125 + .../sql/TestStoragePartitionedJoins.java | 881 +++++ .../spark/sql/TestTableEncryption.java | 341 ++ .../spark/sql/TestTimestampWithoutZone.java | 243 ++ .../spark/sql/TestUnpartitionedWrites.java | 21 + .../sql/TestUnpartitionedWritesToBranch.java | 61 + .../sql/UnpartitionedWritesTestBase.java | 191 ++ .../decimal_dict_and_plain_encoding.parquet | Bin 0 -> 3685 bytes 574 files changed, 140378 insertions(+) create mode 100644 spark/v4.0/build.gradle create mode 100644 spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java create mode 100644 spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/MergeCardinalityCheckBenchmark.java create mode 100644 spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java create mode 100644 spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java create mode 100644 spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/UpdateProjectionBenchmark.java create mode 100644 spark/v4.0/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala create mode 100644 spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/TestExtendedParser.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ProcedureUtil.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputePartitionStatsProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteWithLineage.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadMerge.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadWithLineage.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java create mode 100644 spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/source/PlanUtils.java create mode 100644 spark/v4.0/spark-runtime/LICENSE create mode 100644 spark/v4.0/spark-runtime/NOTICE create mode 100644 spark/v4.0/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/SparkBenchmarkUtil.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/Action.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceDeleteBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceFlatDataBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedDataBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedListDataBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/AvroWritersBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetEqDeleteBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/ParquetWritersBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java create mode 100644 spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadParquetDecimalBenchmark.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/SparkDistributedDataScan.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/BaseFileRewriteCoordinator.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ExtendedParser.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/IcebergSpark.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/JobGroupInfo.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/JobGroupUtils.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/OrcBatchReadConf.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PathIdentifier.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PositionDeletesRewriteCoordinator.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RollbackStagedTable.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SortOrderToSpark.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkAggregates.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCachedTableCatalog.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCompressionUtil.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkDeleteFile.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkExceptionUtil.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkExecutorCache.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTypes.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFunctionCatalog.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTableCache.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SupportsReplaceView.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSnapshotUpdateSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseTableCreationSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ComputePartitionStatsSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteReachableFilesSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/FileInfo.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SetAccumulator.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackFileRewriteRunner.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkDataFileRewriteRunner.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewritePositionDeleteRunner.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewriteRunner.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewritePlanner.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortFileRewriteRunner.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderFileRewriteRunner.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderUDF.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/AvroWithSparkSchemaVisitor.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroWriter.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriters.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessorFactory.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorBuilder.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorWithFilter.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchUtil.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/RowPositionColumnVector.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/SparkFunctions.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/UnaryUnboundFunction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/AncestorsOfProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/CherrypickSnapshotProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputePartitionStatsProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/FastForwardBranchProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/PublishChangesProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RegisterTableProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteDataFilesProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewritePositionDeleteFilesProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteTablePathProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToSnapshotProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToTimestampProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SetCurrentSnapshotProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/DVIterator.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/ExtractRowLineage.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/HasIcebergCatalog.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SerializableTableWithSize.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogTable.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnarReaderFactory.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkInputPartition.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkLocalScan.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkMetadataColumn.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPlanningUtil.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowReaderFactory.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkView.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StagedSparkTable.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/EqualityDeleteFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/IndexedDeleteFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/PositionalDeleteFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDeleteFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDeleteManifests.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteManifests.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskEqualityDeleteFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskIndexedDeleteFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumDeletes.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumSplits.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskPositionalDeleteFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDeleteFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDeleteManifests.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteFiles.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteManifests.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataFileSize.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataManifests.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteFileSize.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteManifests.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataFileSize.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataManifests.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteFileSize.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteManifests.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/IcebergAnalysisException.java create mode 100644 spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister create mode 100644 spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala create mode 100644 spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala create mode 100644 spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala create mode 100644 spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala create mode 100644 spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala create mode 100644 spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala create mode 100644 spark/v4.0/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/KryoHelpers.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/TestFileIOSerialization.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/TestHadoopMetricsContextSerialization.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/TestTableSerialization.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/Employee.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SystemFunctionPushDownHelper.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestBase.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCachedTableCatalog.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCompressionUtil.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkSessionCatalog.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkValueConverter.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriteRunners.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkShufflingDataRewritePlanner.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestOrcWrite.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroReader.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroWriter.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkDateTimes.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkVariants.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestVectorizedOrcDataReader.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/TestColumnarBatchUtil.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryFallbackToPlainEncodingVectorizedReads.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ComplexRecord.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/FilePathLastModifiedRecord.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/FourColumnRecord.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/NestedRecord.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/SimpleRecord.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/SparkSQLExecutionHelper.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroDataFrameWrite.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSource.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetVectorizedScan.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesReader.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDVWriters.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPlanningUtil.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkWriterMetrics.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructInternalRowVariant.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ThreeColumnRecord.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesAsSelect.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToBranch.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDefaultValues.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkVariantRead.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java create mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java create mode 100644 spark/v4.0/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet diff --git a/spark/v4.0/build.gradle b/spark/v4.0/build.gradle new file mode 100644 index 000000000000..8ebed9bd439b --- /dev/null +++ b/spark/v4.0/build.gradle @@ -0,0 +1,349 @@ +/* + * 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. + */ + +String sparkMajorVersion = '4.0' +String scalaVersion = '2.13' + +JavaVersion javaVersion = JavaVersion.current() +Boolean javaVersionSupported = javaVersion == JavaVersion.VERSION_17 || javaVersion == JavaVersion.VERSION_21 +if (!javaVersionSupported) { + logger.warn("Skip Spark 4.0 build which requires JDK 17 or 21 but was executed with JDK " + javaVersion) +} + +def sparkProjects = [ + project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}"), + project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}"), + project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}"), +] + +configure(sparkProjects) { + tasks.configureEach { + onlyIf { javaVersionSupported } + } + configurations { + all { + resolutionStrategy { + force "com.fasterxml.jackson.module:jackson-module-scala_${scalaVersion}:${libs.versions.jackson215.get()}" + force "com.fasterxml.jackson.core:jackson-databind:${libs.versions.jackson215.get()}" + force "com.fasterxml.jackson.core:jackson-core:${libs.versions.jackson215.get()}" + } + } + } +} + +project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { + apply plugin: 'scala' + apply plugin: 'com.github.alisiikh.scalastyle' + + sourceSets { + main { + scala.srcDirs = ['src/main/scala', 'src/main/java'] + java.srcDirs = [] + } + } + + dependencies { + implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') + api project(':iceberg-api') + annotationProcessor libs.immutables.value + compileOnly libs.immutables.value + implementation project(':iceberg-common') + implementation project(':iceberg-core') + implementation project(':iceberg-data') + implementation project(':iceberg-orc') + implementation project(':iceberg-parquet') + implementation project(':iceberg-arrow') + implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + implementation("org.apache.datasketches:datasketches-java:${libs.versions.datasketches.get()}") + + compileOnly libs.errorprone.annotations + compileOnly libs.avro.avro + compileOnly("org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark40.get()}") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.apache.arrow' + exclude group: 'org.apache.parquet' + // to make sure netty libs only come from project(':iceberg-arrow') + exclude group: 'io.netty', module: 'netty-buffer' + exclude group: 'io.netty', module: 'netty-common' + exclude group: 'org.roaringbitmap' + } + + compileOnly "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_2.13:${libs.versions.comet.get()}" + + implementation libs.parquet.column + implementation libs.parquet.hadoop + + implementation("${libs.orc.core.get().module}:${libs.versions.orc.get()}:nohive") { + exclude group: 'org.apache.hadoop' + exclude group: 'commons-lang' + // These artifacts are shaded and included in the orc-core fat jar + exclude group: 'com.google.protobuf', module: 'protobuf-java' + exclude group: 'org.apache.hive', module: 'hive-storage-api' + } + + implementation(libs.arrow.vector) { + exclude group: 'io.netty', module: 'netty-buffer' + exclude group: 'io.netty', module: 'netty-common' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + } + + implementation libs.caffeine + + testImplementation(libs.hadoop3.minicluster) { + exclude group: 'org.apache.avro', module: 'avro' + // to make sure netty libs only come from project(':iceberg-arrow') + exclude group: 'io.netty', module: 'netty-buffer' + exclude group: 'io.netty', module: 'netty-common' + } + testImplementation project(path: ':iceberg-hive-metastore') + testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts') + testImplementation (project(path: ':iceberg-open-api', configuration: 'testFixturesRuntimeElements')) { + transitive = false + } + testImplementation libs.sqlite.jdbc + testImplementation libs.awaitility + testImplementation(testFixtures(project(':iceberg-parquet'))) + // runtime dependencies for running REST Catalog based integration test + testRuntimeOnly libs.jetty.servlet + } + + test { + useJUnitPlatform() + } + + tasks.withType(Test) { + // Vectorized reads need more memory + maxHeapSize '3160m' + } +} + +project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}") { + apply plugin: 'java-library' + apply plugin: 'scala' + apply plugin: 'com.github.alisiikh.scalastyle' + apply plugin: 'antlr' + + configurations { + /* + The Gradle Antlr plugin erroneously adds both antlr-build and runtime dependencies to the runtime path. This + bug https://github.com/gradle/gradle/issues/820 exists because older versions of Antlr do not have separate + runtime and implementation dependencies and they do not want to break backwards compatibility. So to only end up with + the runtime dependency on the runtime classpath we remove the dependencies added by the plugin here. Then add + the runtime dependency back to only the runtime configuration manually. + */ + implementation { + extendsFrom = extendsFrom.findAll { it != configurations.antlr } + } + } + + dependencies { + implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + implementation libs.roaringbitmap + + compileOnly "org.scala-lang:scala-library" + compileOnly project(path: ':iceberg-bundled-guava', configuration: 'shadow') + compileOnly project(':iceberg-api') + compileOnly project(':iceberg-core') + compileOnly project(':iceberg-common') + compileOnly project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") + compileOnly("org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark40.get()}") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.apache.arrow' + exclude group: 'org.apache.parquet' + // to make sure netty libs only come from project(':iceberg-arrow') + exclude group: 'io.netty', module: 'netty-buffer' + exclude group: 'io.netty', module: 'netty-common' + exclude group: 'org.roaringbitmap' + } + compileOnly libs.errorprone.annotations + + testImplementation project(path: ':iceberg-data') + testImplementation project(path: ':iceberg-parquet') + testImplementation project(path: ':iceberg-hive-metastore') + testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') + testImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') + testImplementation (project(path: ':iceberg-open-api', configuration: 'testFixturesRuntimeElements')) { + transitive = false + } + // runtime dependencies for running REST Catalog based integration test + testRuntimeOnly libs.jetty.servlet + testRuntimeOnly libs.sqlite.jdbc + + testImplementation libs.avro.avro + testImplementation libs.parquet.hadoop + testImplementation libs.awaitility + testImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_2.13:${libs.versions.comet.get()}" + testImplementation(testFixtures(project(':iceberg-parquet'))) + + // Required because we remove antlr plugin dependencies from the compile configuration, see note above + runtimeOnly libs.antlr.runtime413 + antlr libs.antlr.antlr413 + } + + test { + useJUnitPlatform() + } + + generateGrammarSource { + maxHeapSize = "64m" + arguments += ['-visitor', '-package', 'org.apache.spark.sql.catalyst.parser.extensions'] + } +} + +project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}") { + apply plugin: 'com.gradleup.shadow' + + tasks.jar.dependsOn tasks.shadowJar + + sourceSets { + integration { + java.srcDir "$projectDir/src/integration/java" + resources.srcDir "$projectDir/src/integration/resources" + } + } + + configurations { + implementation { + exclude group: 'org.apache.spark' + // included in Spark + exclude group: 'org.slf4j' + exclude group: 'org.apache.commons' + exclude group: 'commons-pool' + exclude group: 'commons-codec' + exclude group: 'org.xerial.snappy' + exclude group: 'javax.xml.bind' + exclude group: 'javax.annotation' + exclude group: 'com.github.luben' + exclude group: 'com.ibm.icu' + exclude group: 'org.glassfish' + exclude group: 'org.abego.treelayout' + exclude group: 'org.antlr' + exclude group: 'org.scala-lang' + exclude group: 'org.scala-lang.modules' + } + } + + dependencies { + api project(':iceberg-api') + implementation project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") + implementation project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}") + implementation project(':iceberg-aws') + implementation project(':iceberg-azure') + implementation(project(':iceberg-aliyun')) { + exclude group: 'edu.umd.cs.findbugs', module: 'findbugs' + exclude group: 'org.apache.httpcomponents', module: 'httpclient' + exclude group: 'commons-logging', module: 'commons-logging' + } + implementation project(':iceberg-gcp') + implementation project(':iceberg-bigquery') + implementation project(':iceberg-hive-metastore') + implementation(project(':iceberg-nessie')) { + exclude group: 'com.google.code.findbugs', module: 'jsr305' + } + implementation (project(':iceberg-snowflake')) { + exclude group: 'net.snowflake' , module: 'snowflake-jdbc' + } + + integrationImplementation "org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}" + integrationImplementation "org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark40.get()}" + integrationImplementation libs.junit.jupiter + integrationImplementation libs.junit.platform.launcher + integrationImplementation libs.slf4j.simple + integrationImplementation libs.assertj.core + integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') + integrationImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') + integrationImplementation project(path: ":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') + + // runtime dependencies for running Hive Catalog based integration test + integrationRuntimeOnly project(':iceberg-hive-metastore') + // runtime dependencies for running REST Catalog based integration test + integrationRuntimeOnly project(path: ':iceberg-core', configuration: 'testArtifacts') + integrationRuntimeOnly (project(path: ':iceberg-open-api', configuration: 'testFixturesRuntimeElements')) { + transitive = false + } + integrationRuntimeOnly libs.jetty.servlet + integrationRuntimeOnly libs.sqlite.jdbc + + // Not allowed on our classpath, only the runtime jar is allowed + integrationCompileOnly project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}") + integrationCompileOnly project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") + integrationCompileOnly project(':iceberg-api') + } + + shadowJar { + configurations = [project.configurations.runtimeClasspath] + + zip64 true + + // include the LICENSE and NOTICE files for the shaded Jar + from(projectDir) { + include 'LICENSE' + include 'NOTICE' + } + + // Relocate dependencies to avoid conflicts + relocate 'com.google.errorprone', 'org.apache.iceberg.shaded.com.google.errorprone' + relocate 'com.google.flatbuffers', 'org.apache.iceberg.shaded.com.google.flatbuffers' + relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml' + relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes' + relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework' + relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro' + relocate 'avro.shaded', 'org.apache.iceberg.shaded.org.apache.avro.shaded' + relocate 'com.thoughtworks.paranamer', 'org.apache.iceberg.shaded.com.thoughtworks.paranamer' + relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' + relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' + relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc' + relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift' + relocate 'org.apache.hc.client5', 'org.apache.iceberg.shaded.org.apache.hc.client5' + relocate 'org.apache.hc.core5', 'org.apache.iceberg.shaded.org.apache.hc.core5' + // relocate Arrow and related deps to shade Iceberg specific version + relocate 'io.netty', 'org.apache.iceberg.shaded.io.netty' + relocate 'org.apache.arrow', 'org.apache.iceberg.shaded.org.apache.arrow' + relocate 'com.carrotsearch', 'org.apache.iceberg.shaded.com.carrotsearch' + relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' + relocate 'org.roaringbitmap', 'org.apache.iceberg.shaded.org.roaringbitmap' + relocate 'org.apache.datasketches', 'org.apache.iceberg.shaded.org.apache.datasketches' + + archiveClassifier.set(null) + } + + task integrationTest(type: Test) { + useJUnitPlatform() + description = "Test Spark3 Runtime Jar against Spark ${sparkMajorVersion}" + group = "verification" + jvmArgs += project.property('extraJvmArgs') + testClassesDirs = sourceSets.integration.output.classesDirs + classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) + inputs.file(shadowJar.archiveFile.get().asFile.path) + } + integrationTest.dependsOn shadowJar + check.dependsOn integrationTest + + jar { + enabled = false + } +} + diff --git a/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java b/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java new file mode 100644 index 000000000000..9375ca3a4f46 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java @@ -0,0 +1,279 @@ +/* + * 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; + +import com.google.errorprone.annotations.FormatMethod; +import com.google.errorprone.annotations.FormatString; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; +import org.apache.iceberg.util.ThreadPools; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * A benchmark that evaluates the delete file index build and lookup performance. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-extensions-4.0_2.13:jmh + * -PjmhIncludeRegex=DeleteFileIndexBenchmark + * -PjmhOutputPath=benchmark/iceberg-delete-file-index-benchmark.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 10) +@Timeout(time = 20, timeUnit = TimeUnit.MINUTES) +@BenchmarkMode(Mode.SingleShotTime) +public class DeleteFileIndexBenchmark { + + private static final String TABLE_NAME = "test_table"; + private static final String PARTITION_COLUMN = "ss_ticket_number"; + + private static final int NUM_PARTITIONS = 50; + private static final int NUM_DATA_FILES_PER_PARTITION = 50_000; + private static final int NUM_DELETE_FILES_PER_PARTITION = 100; + + private final Configuration hadoopConf = new Configuration(); + private SparkSession spark; + private Table table; + + private List dataFiles; + + @Param({"partition", "file", "dv"}) + private String type; + + @Setup + public void setupBenchmark() throws NoSuchTableException, ParseException { + setupSpark(); + initTable(); + initDataAndDeletes(); + loadDataFiles(); + } + + private void initDataAndDeletes() { + if (type.equals("partition")) { + initDataAndPartitionScopedDeletes(); + } else if (type.equals("file")) { + initDataAndFileScopedDeletes(); + } else { + initDataAndDVs(); + } + } + + @TearDown + public void tearDownBenchmark() { + dropTable(); + tearDownSpark(); + } + + @Benchmark + @Threads(1) + public void buildIndexAndLookup(Blackhole blackhole) { + DeleteFileIndex deletes = buildDeletes(); + for (DataFile dataFile : dataFiles) { + DeleteFile[] deleteFiles = deletes.forDataFile(dataFile.dataSequenceNumber(), dataFile); + blackhole.consume(deleteFiles); + } + } + + private void loadDataFiles() { + table.refresh(); + + Snapshot snapshot = table.currentSnapshot(); + + ManifestGroup manifestGroup = + new ManifestGroup(table.io(), snapshot.dataManifests(table.io()), ImmutableList.of()); + + try (CloseableIterable> entries = manifestGroup.entries()) { + List files = Lists.newArrayList(); + for (ManifestEntry entry : entries) { + files.add(entry.file().copyWithoutStats()); + } + this.dataFiles = files; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private DeleteFileIndex buildDeletes() { + table.refresh(); + + List deleteManifests = table.currentSnapshot().deleteManifests(table.io()); + + return DeleteFileIndex.builderFor(table.io(), deleteManifests) + .specsById(table.specs()) + .planWith(ThreadPools.getWorkerPool()) + .build(); + } + + private void initDataAndPartitionScopedDeletes() { + for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { + StructLike partition = TestHelpers.Row.of(partitionOrdinal); + + RowDelta rowDelta = table.newRowDelta(); + + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = FileGenerationUtil.generateDataFile(table, partition); + rowDelta.addRows(dataFile); + } + + for (int fileOrdinal = 0; fileOrdinal < NUM_DELETE_FILES_PER_PARTITION; fileOrdinal++) { + DeleteFile deleteFile = FileGenerationUtil.generatePositionDeleteFile(table, partition); + rowDelta.addDeletes(deleteFile); + } + + rowDelta.commit(); + } + } + + private void initDataAndFileScopedDeletes() { + for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { + StructLike partition = TestHelpers.Row.of(partitionOrdinal); + + RowDelta rowDelta = table.newRowDelta(); + + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = FileGenerationUtil.generateDataFile(table, partition); + DeleteFile deleteFile = FileGenerationUtil.generatePositionDeleteFile(table, dataFile); + rowDelta.addRows(dataFile); + rowDelta.addDeletes(deleteFile); + } + + rowDelta.commit(); + } + } + + private void initDataAndDVs() { + for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { + StructLike partition = TestHelpers.Row.of(partitionOrdinal); + + RowDelta rowDelta = table.newRowDelta(); + + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = FileGenerationUtil.generateDataFile(table, partition); + DeleteFile dv = FileGenerationUtil.generateDV(table, dataFile); + rowDelta.addRows(dataFile); + rowDelta.addDeletes(dv); + } + + rowDelta.commit(); + } + } + + private void setupSpark() { + this.spark = + SparkSession.builder() + .config("spark.ui.enabled", false) + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName()) + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) + .master("local[*]") + .getOrCreate(); + } + + private void tearDownSpark() { + spark.stop(); + } + + private void initTable() throws NoSuchTableException, ParseException { + sql( + "CREATE TABLE %s ( " + + " `ss_sold_date_sk` INT, " + + " `ss_sold_time_sk` INT, " + + " `ss_item_sk` INT, " + + " `ss_customer_sk` STRING, " + + " `ss_cdemo_sk` STRING, " + + " `ss_hdemo_sk` STRING, " + + " `ss_addr_sk` STRING, " + + " `ss_store_sk` STRING, " + + " `ss_promo_sk` STRING, " + + " `ss_ticket_number` INT, " + + " `ss_quantity` STRING, " + + " `ss_wholesale_cost` STRING, " + + " `ss_list_price` STRING, " + + " `ss_sales_price` STRING, " + + " `ss_ext_discount_amt` STRING, " + + " `ss_ext_sales_price` STRING, " + + " `ss_ext_wholesale_cost` STRING, " + + " `ss_ext_list_price` STRING, " + + " `ss_ext_tax` STRING, " + + " `ss_coupon_amt` STRING, " + + " `ss_net_paid` STRING, " + + " `ss_net_paid_inc_tax` STRING, " + + " `ss_net_profit` STRING " + + ")" + + "USING iceberg " + + "PARTITIONED BY (%s) " + + "TBLPROPERTIES (" + + " '%s' '%b'," + + " '%s' '%s'," + + " '%s' '%d')", + TABLE_NAME, + PARTITION_COLUMN, + TableProperties.MANIFEST_MERGE_ENABLED, + false, + TableProperties.DELETE_MODE, + RowLevelOperationMode.MERGE_ON_READ.modeName(), + TableProperties.FORMAT_VERSION, + type.equals("dv") ? 3 : 2); + + this.table = Spark3Util.loadIcebergTable(spark, TABLE_NAME); + } + + private void dropTable() { + sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME); + } + + private String newWarehouseDir() { + return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID(); + } + + @FormatMethod + private void sql(@FormatString String query, Object... args) { + spark.sql(String.format(query, args)); + } +} diff --git a/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/MergeCardinalityCheckBenchmark.java b/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/MergeCardinalityCheckBenchmark.java new file mode 100644 index 000000000000..963daa2c364c --- /dev/null +++ b/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/MergeCardinalityCheckBenchmark.java @@ -0,0 +1,231 @@ +/* + * 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 static org.apache.spark.sql.functions.current_date; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +import com.google.errorprone.annotations.FormatMethod; +import com.google.errorprone.annotations.FormatString; +import java.util.UUID; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.StructType; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; + +/** + * A benchmark that evaluates the performance of the cardinality check in MERGE operations. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-extensions-4.0_2.13:jmh + * -PjmhIncludeRegex=MergeCardinalityCheckBenchmark + * -PjmhOutputPath=benchmark/iceberg-merge-cardinality-check-benchmark.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class MergeCardinalityCheckBenchmark { + + private static final String TABLE_NAME = "test_table"; + private static final int NUM_FILES = 5; + private static final int NUM_ROWS_PER_FILE = 1_000_000; + private static final int NUM_UNMATCHED_RECORDS_PER_MERGE = 100_000; + + private final Configuration hadoopConf = new Configuration(); + private SparkSession spark; + private long originalSnapshotId; + + @Setup + public void setupBenchmark() throws NoSuchTableException, ParseException { + setupSpark(); + initTable(); + appendData(); + + Table table = Spark3Util.loadIcebergTable(spark, TABLE_NAME); + this.originalSnapshotId = table.currentSnapshot().snapshotId(); + } + + @TearDown + public void tearDownBenchmark() { + tearDownSpark(); + dropTable(); + } + + @Benchmark + @Threads(1) + public void copyOnWriteMergeCardinalityCheck10PercentUpdates() { + runBenchmark(RowLevelOperationMode.COPY_ON_WRITE, 0.1); + } + + @Benchmark + @Threads(1) + public void copyOnWriteMergeCardinalityCheck30PercentUpdates() { + runBenchmark(RowLevelOperationMode.COPY_ON_WRITE, 0.3); + } + + @Benchmark + @Threads(1) + public void copyOnWriteMergeCardinalityCheck90PercentUpdates() { + runBenchmark(RowLevelOperationMode.COPY_ON_WRITE, 0.9); + } + + @Benchmark + @Threads(1) + public void mergeOnReadMergeCardinalityCheck10PercentUpdates() { + runBenchmark(RowLevelOperationMode.MERGE_ON_READ, 0.1); + } + + @Benchmark + @Threads(1) + public void mergeOnReadMergeCardinalityCheck30PercentUpdates() { + runBenchmark(RowLevelOperationMode.MERGE_ON_READ, 0.3); + } + + @Benchmark + @Threads(1) + public void mergeOnReadMergeCardinalityCheck90PercentUpdates() { + runBenchmark(RowLevelOperationMode.MERGE_ON_READ, 0.9); + } + + private void runBenchmark(RowLevelOperationMode mode, double updatePercentage) { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", + TABLE_NAME, TableProperties.MERGE_MODE, mode.modeName()); + + Dataset insertDataDF = spark.range(-NUM_UNMATCHED_RECORDS_PER_MERGE, 0, 1); + Dataset updateDataDF = spark.range((long) (updatePercentage * NUM_ROWS_PER_FILE)); + Dataset sourceDF = updateDataDF.union(insertDataDF); + sourceDF.createOrReplaceTempView("source"); + + sql( + "MERGE INTO %s t USING source s " + + "ON t.id = s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET stringCol = 'invalid' " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, intCol, floatCol, doubleCol, decimalCol, dateCol, timestampCol, stringCol) " + + " VALUES (s.id, null, null, null, null, null, null, 'new')", + TABLE_NAME); + + sql( + "CALL system.rollback_to_snapshot(table => '%s', snapshot_id => %dL)", + TABLE_NAME, originalSnapshotId); + } + + private void setupSpark() { + this.spark = + SparkSession.builder() + .config("spark.ui.enabled", false) + .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName()) + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) + .config(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED().key(), "false") + .config(SQLConf.RUNTIME_ROW_LEVEL_OPERATION_GROUP_FILTER_ENABLED().key(), "false") + .config(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), "false") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "2") + .master("local") + .getOrCreate(); + } + + private void tearDownSpark() { + spark.stop(); + } + + private void initTable() { + sql( + "CREATE TABLE %s ( " + + " id LONG, intCol INT, floatCol FLOAT, doubleCol DOUBLE, " + + " decimalCol DECIMAL(20, 5), dateCol DATE, timestampCol TIMESTAMP, " + + " stringCol STRING)" + + "USING iceberg " + + "TBLPROPERTIES (" + + " '%s' '%s'," + + " '%s' '%d'," + + " '%s' '%d')", + TABLE_NAME, + TableProperties.MERGE_DISTRIBUTION_MODE, + DistributionMode.NONE.modeName(), + TableProperties.SPLIT_OPEN_FILE_COST, + Integer.MAX_VALUE, + TableProperties.FORMAT_VERSION, + 2); + + sql("ALTER TABLE %s WRITE ORDERED BY id", TABLE_NAME); + } + + private void dropTable() { + sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME); + } + + private void appendData() throws NoSuchTableException { + for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) { + Dataset inputDF = + spark + .range(NUM_ROWS_PER_FILE) + .withColumn("intCol", expr("CAST(id AS INT)")) + .withColumn("floatCol", expr("CAST(id AS FLOAT)")) + .withColumn("doubleCol", expr("CAST(id AS DOUBLE)")) + .withColumn("decimalCol", expr("CAST(id AS DECIMAL(20, 5))")) + .withColumn("dateCol", date_add(current_date(), fileNum)) + .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)")) + .withColumn("stringCol", expr("CAST(dateCol AS STRING)")); + appendAsFile(inputDF); + } + } + + private void appendAsFile(Dataset df) throws NoSuchTableException { + // ensure the schema is precise (including nullability) + StructType sparkSchema = spark.table(TABLE_NAME).schema(); + spark.createDataFrame(df.rdd(), sparkSchema).coalesce(1).writeTo(TABLE_NAME).append(); + } + + private String newWarehouseDir() { + return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID(); + } + + @FormatMethod + private void sql(@FormatString String query, Object... args) { + spark.sql(String.format(query, args)); + } +} diff --git a/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java b/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java new file mode 100644 index 000000000000..34d9d70e6ccb --- /dev/null +++ b/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java @@ -0,0 +1,409 @@ +/* + * 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 static org.apache.iceberg.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; + +import com.google.errorprone.annotations.FormatMethod; +import com.google.errorprone.annotations.FormatString; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BatchScan; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileGenerationUtil; +import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.SparkDistributedDataScan; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * A benchmark that evaluates the job planning performance. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-extensions-4.0_2.12:jmh + * -PjmhIncludeRegex=PlanningBenchmark + * -PjmhOutputPath=benchmark/iceberg-planning-benchmark.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@Timeout(time = 20, timeUnit = TimeUnit.MINUTES) +@BenchmarkMode(Mode.SingleShotTime) +public class PlanningBenchmark { + + private static final String TABLE_NAME = "test_table"; + private static final String PARTITION_COLUMN = "ss_ticket_number"; + private static final int PARTITION_VALUE = 10; + private static final String SORT_KEY_COLUMN = "ss_sold_date_sk"; + private static final int SORT_KEY_VALUE = 5; + + private static final Expression SORT_KEY_PREDICATE = + Expressions.equal(SORT_KEY_COLUMN, SORT_KEY_VALUE); + private static final Expression PARTITION_PREDICATE = + Expressions.equal(PARTITION_COLUMN, PARTITION_VALUE); + private static final Expression PARTITION_AND_SORT_KEY_PREDICATE = + Expressions.and(PARTITION_PREDICATE, SORT_KEY_PREDICATE); + + private static final int NUM_PARTITIONS = 30; + private static final int NUM_DATA_FILES_PER_PARTITION = 50_000; + private static final int NUM_DELETE_FILES_PER_PARTITION = 50; + + private final Configuration hadoopConf = new Configuration(); + private SparkSession spark; + private Table table; + + @Param({"partition", "file", "dv"}) + private String type; + + @Setup + public void setupBenchmark() throws NoSuchTableException, ParseException { + setupSpark(); + initTable(); + initDataAndDeletes(); + } + + @TearDown + public void tearDownBenchmark() { + dropTable(); + tearDownSpark(); + } + + @Benchmark + @Threads(1) + public void localPlanningWithPartitionAndMinMaxFilter(Blackhole blackhole) { + BatchScan scan = table.newBatchScan(); + List fileTasks = planFilesWithoutColumnStats(scan, PARTITION_AND_SORT_KEY_PREDICATE); + blackhole.consume(fileTasks); + } + + @Benchmark + @Threads(1) + public void distributedPlanningWithPartitionAndMinMaxFilter(Blackhole blackhole) { + BatchScan scan = newDistributedScan(DISTRIBUTED, DISTRIBUTED); + List fileTasks = planFilesWithoutColumnStats(scan, PARTITION_AND_SORT_KEY_PREDICATE); + blackhole.consume(fileTasks); + } + + @Benchmark + @Threads(1) + public void localPlanningWithMinMaxFilter(Blackhole blackhole) { + BatchScan scan = table.newBatchScan(); + List fileTasks = planFilesWithoutColumnStats(scan, SORT_KEY_PREDICATE); + blackhole.consume(fileTasks); + } + + @Benchmark + @Threads(1) + public void distributedPlanningWithMinMaxFilter(Blackhole blackhole) { + BatchScan scan = newDistributedScan(DISTRIBUTED, DISTRIBUTED); + List fileTasks = planFilesWithoutColumnStats(scan, SORT_KEY_PREDICATE); + blackhole.consume(fileTasks); + } + + @Benchmark + @Threads(1) + public void localPlanningWithoutFilter(Blackhole blackhole) { + BatchScan scan = table.newBatchScan(); + List fileTasks = planFilesWithoutColumnStats(scan, Expressions.alwaysTrue()); + blackhole.consume(fileTasks); + } + + @Benchmark + @Threads(1) + public void distributedPlanningWithoutFilter(Blackhole blackhole) { + BatchScan scan = newDistributedScan(DISTRIBUTED, DISTRIBUTED); + List fileTasks = planFilesWithoutColumnStats(scan, Expressions.alwaysTrue()); + blackhole.consume(fileTasks); + } + + @Benchmark + @Threads(1) + public void localPlanningWithoutFilterWithStats(Blackhole blackhole) { + BatchScan scan = table.newBatchScan(); + List fileTasks = planFilesWithColumnStats(scan, Expressions.alwaysTrue()); + blackhole.consume(fileTasks); + } + + @Benchmark + @Threads(1) + public void distributedPlanningWithoutFilterWithStats(Blackhole blackhole) { + BatchScan scan = newDistributedScan(DISTRIBUTED, DISTRIBUTED); + List fileTasks = planFilesWithColumnStats(scan, Expressions.alwaysTrue()); + blackhole.consume(fileTasks); + } + + @Benchmark + @Threads(1) + public void distributedDataLocalDeletesPlanningWithoutFilterWithStats(Blackhole blackhole) { + BatchScan scan = newDistributedScan(DISTRIBUTED, LOCAL); + List fileTasks = planFilesWithColumnStats(scan, Expressions.alwaysTrue()); + blackhole.consume(fileTasks); + } + + @Benchmark + @Threads(1) + public void localDataDistributedDeletesPlanningWithoutFilterWithStats(Blackhole blackhole) { + BatchScan scan = newDistributedScan(LOCAL, DISTRIBUTED); + List fileTasks = planFilesWithColumnStats(scan, Expressions.alwaysTrue()); + blackhole.consume(fileTasks); + } + + @Benchmark + @Threads(1) + public void localPlanningViaDistributedScanWithoutFilterWithStats(Blackhole blackhole) { + BatchScan scan = newDistributedScan(LOCAL, LOCAL); + List fileTasks = planFilesWithColumnStats(scan, Expressions.alwaysTrue()); + blackhole.consume(fileTasks); + } + + private void setupSpark() { + this.spark = + SparkSession.builder() + .config("spark.ui.enabled", false) + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config("spark.driver.maxResultSize", "8G") + .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName()) + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) + .master("local[*]") + .getOrCreate(); + } + + private void tearDownSpark() { + spark.stop(); + } + + private void initTable() throws NoSuchTableException, ParseException { + sql( + "CREATE TABLE %s ( " + + " `ss_sold_date_sk` INT, " + + " `ss_sold_time_sk` INT, " + + " `ss_item_sk` INT, " + + " `ss_customer_sk` STRING, " + + " `ss_cdemo_sk` STRING, " + + " `ss_hdemo_sk` STRING, " + + " `ss_addr_sk` STRING, " + + " `ss_store_sk` STRING, " + + " `ss_promo_sk` STRING, " + + " `ss_ticket_number` INT, " + + " `ss_quantity` STRING, " + + " `ss_wholesale_cost` STRING, " + + " `ss_list_price` STRING, " + + " `ss_sales_price` STRING, " + + " `ss_ext_discount_amt` STRING, " + + " `ss_ext_sales_price` STRING, " + + " `ss_ext_wholesale_cost` STRING, " + + " `ss_ext_list_price` STRING, " + + " `ss_ext_tax` STRING, " + + " `ss_coupon_amt` STRING, " + + " `ss_net_paid` STRING, " + + " `ss_net_paid_inc_tax` STRING, " + + " `ss_net_profit` STRING " + + ")" + + "USING iceberg " + + "PARTITIONED BY (%s) " + + "TBLPROPERTIES (" + + " '%s' '%b'," + + " '%s' '%s'," + + " '%s' '%d')", + TABLE_NAME, + PARTITION_COLUMN, + TableProperties.MANIFEST_MERGE_ENABLED, + false, + TableProperties.DELETE_MODE, + RowLevelOperationMode.MERGE_ON_READ.modeName(), + TableProperties.FORMAT_VERSION, + type.equals("dv") ? 3 : 2); + + this.table = Spark3Util.loadIcebergTable(spark, TABLE_NAME); + } + + private void dropTable() { + sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME); + } + + private void initDataAndDeletes() { + if (type.equals("partition")) { + initDataAndPartitionScopedDeletes(); + } else if (type.equals("file")) { + initDataAndFileScopedDeletes(); + } else { + initDataAndDVs(); + } + } + + private void initDataAndPartitionScopedDeletes() { + for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { + StructLike partition = TestHelpers.Row.of(partitionOrdinal); + + RowDelta rowDelta = table.newRowDelta(); + + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = generateDataFile(partition, Integer.MIN_VALUE, Integer.MIN_VALUE); + rowDelta.addRows(dataFile); + } + + // add one data file that would match the sort key predicate + DataFile sortKeyDataFile = generateDataFile(partition, SORT_KEY_VALUE, SORT_KEY_VALUE); + rowDelta.addRows(sortKeyDataFile); + + for (int fileOrdinal = 0; fileOrdinal < NUM_DELETE_FILES_PER_PARTITION; fileOrdinal++) { + DeleteFile deleteFile = FileGenerationUtil.generatePositionDeleteFile(table, partition); + rowDelta.addDeletes(deleteFile); + } + + rowDelta.commit(); + } + } + + private void initDataAndFileScopedDeletes() { + for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { + StructLike partition = TestHelpers.Row.of(partitionOrdinal); + + RowDelta rowDelta = table.newRowDelta(); + + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = generateDataFile(partition, Integer.MIN_VALUE, Integer.MIN_VALUE); + DeleteFile deleteFile = FileGenerationUtil.generatePositionDeleteFile(table, dataFile); + rowDelta.addRows(dataFile); + rowDelta.addDeletes(deleteFile); + } + + // add one data file that would match the sort key predicate + DataFile sortKeyDataFile = generateDataFile(partition, SORT_KEY_VALUE, SORT_KEY_VALUE); + rowDelta.addRows(sortKeyDataFile); + + rowDelta.commit(); + } + } + + private void initDataAndDVs() { + for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { + StructLike partition = TestHelpers.Row.of(partitionOrdinal); + + RowDelta rowDelta = table.newRowDelta(); + + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = generateDataFile(partition, Integer.MIN_VALUE, Integer.MIN_VALUE); + DeleteFile dv = FileGenerationUtil.generateDV(table, dataFile); + rowDelta.addRows(dataFile); + rowDelta.addDeletes(dv); + } + + // add one data file that would match the sort key predicate + DataFile sortKeyDataFile = generateDataFile(partition, SORT_KEY_VALUE, SORT_KEY_VALUE); + rowDelta.addRows(sortKeyDataFile); + + rowDelta.commit(); + } + } + + private DataFile generateDataFile(StructLike partition, int sortKeyMin, int sortKeyMax) { + int sortKeyFieldId = table.schema().findField(SORT_KEY_COLUMN).fieldId(); + ByteBuffer lower = Conversions.toByteBuffer(Types.IntegerType.get(), sortKeyMin); + Map lowerBounds = ImmutableMap.of(sortKeyFieldId, lower); + ByteBuffer upper = Conversions.toByteBuffer(Types.IntegerType.get(), sortKeyMax); + Map upperBounds = ImmutableMap.of(sortKeyFieldId, upper); + return FileGenerationUtil.generateDataFile(table, partition, lowerBounds, upperBounds); + } + + private String newWarehouseDir() { + return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID(); + } + + private List planFilesWithoutColumnStats(BatchScan scan, Expression predicate) { + return planFiles(scan, predicate, false); + } + + private List planFilesWithColumnStats(BatchScan scan, Expression predicate) { + return planFiles(scan, predicate, true); + } + + private List planFiles(BatchScan scan, Expression predicate, boolean withColumnStats) { + table.refresh(); + + BatchScan configuredScan = scan.filter(predicate); + + if (withColumnStats) { + configuredScan = scan.includeColumnStats(); + } + + try (CloseableIterable fileTasks = configuredScan.planFiles()) { + return Lists.newArrayList(fileTasks); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private BatchScan newDistributedScan(PlanningMode dataMode, PlanningMode deleteMode) { + table + .updateProperties() + .set(TableProperties.DATA_PLANNING_MODE, dataMode.modeName()) + .set(TableProperties.DELETE_PLANNING_MODE, deleteMode.modeName()) + .commit(); + SparkReadConf readConf = new SparkReadConf(spark, table, ImmutableMap.of()); + return new SparkDistributedDataScan(spark, table, readConf); + } + + @FormatMethod + private void sql(@FormatString String query, Object... args) { + spark.sql(String.format(query, args)); + } +} diff --git a/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java b/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java new file mode 100644 index 000000000000..7c2def237874 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java @@ -0,0 +1,273 @@ +/* + * 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 com.google.errorprone.annotations.FormatMethod; +import com.google.errorprone.annotations.FormatString; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileGenerationUtil; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; +import org.apache.iceberg.util.TableScanUtil; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * A benchmark that evaluates the task group planning performance. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-extensions-4.0_2.13:jmh + * -PjmhIncludeRegex=TaskGroupPlanningBenchmark + * -PjmhOutputPath=benchmark/iceberg-task-group-planning-benchmark.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@Timeout(time = 30, timeUnit = TimeUnit.MINUTES) +@BenchmarkMode(Mode.SingleShotTime) +public class TaskGroupPlanningBenchmark { + + private static final String TABLE_NAME = "test_table"; + private static final String PARTITION_COLUMN = "ss_ticket_number"; + + private static final int NUM_PARTITIONS = 150; + private static final int NUM_DATA_FILES_PER_PARTITION = 50_000; + private static final int NUM_DELETE_FILES_PER_PARTITION = 25; + + private final Configuration hadoopConf = new Configuration(); + private SparkSession spark; + private Table table; + + private List fileTasks; + + @Setup + public void setupBenchmark() throws NoSuchTableException, ParseException { + setupSpark(); + initTable(); + initDataAndDeletes(); + loadFileTasks(); + } + + @TearDown + public void tearDownBenchmark() { + dropTable(); + tearDownSpark(); + } + + @Benchmark + @Threads(1) + public void planTaskGroups(Blackhole blackhole) { + SparkReadConf readConf = new SparkReadConf(spark, table, ImmutableMap.of()); + List> taskGroups = + TableScanUtil.planTaskGroups( + fileTasks, + readConf.splitSize(), + readConf.splitLookback(), + readConf.splitOpenFileCost()); + + long rowsCount = 0L; + for (ScanTaskGroup taskGroup : taskGroups) { + rowsCount += taskGroup.estimatedRowsCount(); + } + blackhole.consume(rowsCount); + + long filesCount = 0L; + for (ScanTaskGroup taskGroup : taskGroups) { + filesCount += taskGroup.filesCount(); + } + blackhole.consume(filesCount); + + long sizeBytes = 0L; + for (ScanTaskGroup taskGroup : taskGroups) { + sizeBytes += taskGroup.sizeBytes(); + } + blackhole.consume(sizeBytes); + } + + @Benchmark + @Threads(1) + public void planTaskGroupsWithGrouping(Blackhole blackhole) { + SparkReadConf readConf = new SparkReadConf(spark, table, ImmutableMap.of()); + + List> taskGroups = + TableScanUtil.planTaskGroups( + fileTasks, + readConf.splitSize(), + readConf.splitLookback(), + readConf.splitOpenFileCost(), + Partitioning.groupingKeyType(table.schema(), table.specs().values())); + + long rowsCount = 0L; + for (ScanTaskGroup taskGroup : taskGroups) { + rowsCount += taskGroup.estimatedRowsCount(); + } + blackhole.consume(rowsCount); + + long filesCount = 0L; + for (ScanTaskGroup taskGroup : taskGroups) { + filesCount += taskGroup.filesCount(); + } + blackhole.consume(filesCount); + + long sizeBytes = 0L; + for (ScanTaskGroup taskGroup : taskGroups) { + sizeBytes += taskGroup.sizeBytes(); + } + blackhole.consume(sizeBytes); + } + + private void loadFileTasks() { + table.refresh(); + + try (CloseableIterable fileTasksIterable = table.newScan().planFiles()) { + this.fileTasks = Lists.newArrayList(fileTasksIterable); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private void initDataAndDeletes() { + for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { + StructLike partition = TestHelpers.Row.of(partitionOrdinal); + + RowDelta rowDelta = table.newRowDelta(); + + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = FileGenerationUtil.generateDataFile(table, partition); + rowDelta.addRows(dataFile); + } + + for (int fileOrdinal = 0; fileOrdinal < NUM_DELETE_FILES_PER_PARTITION; fileOrdinal++) { + DeleteFile deleteFile = FileGenerationUtil.generatePositionDeleteFile(table, partition); + rowDelta.addDeletes(deleteFile); + } + + rowDelta.commit(); + } + } + + private void setupSpark() { + this.spark = + SparkSession.builder() + .config("spark.ui.enabled", false) + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName()) + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) + .master("local[*]") + .getOrCreate(); + } + + private void tearDownSpark() { + spark.stop(); + } + + private void initTable() throws NoSuchTableException, ParseException { + sql( + "CREATE TABLE %s ( " + + " `ss_sold_date_sk` INT, " + + " `ss_sold_time_sk` INT, " + + " `ss_item_sk` INT, " + + " `ss_customer_sk` STRING, " + + " `ss_cdemo_sk` STRING, " + + " `ss_hdemo_sk` STRING, " + + " `ss_addr_sk` STRING, " + + " `ss_store_sk` STRING, " + + " `ss_promo_sk` STRING, " + + " `ss_ticket_number` INT, " + + " `ss_quantity` STRING, " + + " `ss_wholesale_cost` STRING, " + + " `ss_list_price` STRING, " + + " `ss_sales_price` STRING, " + + " `ss_ext_discount_amt` STRING, " + + " `ss_ext_sales_price` STRING, " + + " `ss_ext_wholesale_cost` STRING, " + + " `ss_ext_list_price` STRING, " + + " `ss_ext_tax` STRING, " + + " `ss_coupon_amt` STRING, " + + " `ss_net_paid` STRING, " + + " `ss_net_paid_inc_tax` STRING, " + + " `ss_net_profit` STRING " + + ")" + + "USING iceberg " + + "PARTITIONED BY (%s) " + + "TBLPROPERTIES (" + + " '%s' '%b'," + + " '%s' '%s'," + + " '%s' '%d')", + TABLE_NAME, + PARTITION_COLUMN, + TableProperties.MANIFEST_MERGE_ENABLED, + false, + TableProperties.DELETE_MODE, + RowLevelOperationMode.MERGE_ON_READ.modeName(), + TableProperties.FORMAT_VERSION, + 2); + + this.table = Spark3Util.loadIcebergTable(spark, TABLE_NAME); + } + + private void dropTable() { + sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME); + } + + private String newWarehouseDir() { + return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID(); + } + + @FormatMethod + private void sql(@FormatString String query, Object... args) { + spark.sql(String.format(query, args)); + } +} diff --git a/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/UpdateProjectionBenchmark.java b/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/UpdateProjectionBenchmark.java new file mode 100644 index 000000000000..d917eae5eb0f --- /dev/null +++ b/spark/v4.0/spark-extensions/src/jmh/java/org/apache/iceberg/spark/UpdateProjectionBenchmark.java @@ -0,0 +1,213 @@ +/* + * 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 static org.apache.spark.sql.functions.current_date; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +import com.google.errorprone.annotations.FormatMethod; +import com.google.errorprone.annotations.FormatString; +import java.util.UUID; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.StructType; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; + +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class UpdateProjectionBenchmark { + + private static final String TABLE_NAME = "test_table"; + private static final int NUM_FILES = 5; + private static final int NUM_ROWS_PER_FILE = 1_000_000; + + private final Configuration hadoopConf = new Configuration(); + private SparkSession spark; + private long originalSnapshotId; + + @Setup + public void setupBenchmark() throws NoSuchTableException, ParseException { + setupSpark(); + initTable(); + appendData(); + + Table table = Spark3Util.loadIcebergTable(spark, TABLE_NAME); + this.originalSnapshotId = table.currentSnapshot().snapshotId(); + } + + @TearDown + public void tearDownBenchmark() { + tearDownSpark(); + dropTable(); + } + + @Benchmark + @Threads(1) + public void copyOnWriteUpdate10Percent() { + runBenchmark(RowLevelOperationMode.COPY_ON_WRITE, 0.1); + } + + @Benchmark + @Threads(1) + public void copyOnWriteUpdate30Percent() { + runBenchmark(RowLevelOperationMode.COPY_ON_WRITE, 0.3); + } + + @Benchmark + @Threads(1) + public void copyOnWriteUpdate75Percent() { + runBenchmark(RowLevelOperationMode.COPY_ON_WRITE, 0.75); + } + + @Benchmark + @Threads(1) + public void mergeOnRead10Percent() { + runBenchmark(RowLevelOperationMode.MERGE_ON_READ, 0.1); + } + + @Benchmark + @Threads(1) + public void mergeOnReadUpdate30Percent() { + runBenchmark(RowLevelOperationMode.MERGE_ON_READ, 0.3); + } + + @Benchmark + @Threads(1) + public void mergeOnReadUpdate75Percent() { + runBenchmark(RowLevelOperationMode.MERGE_ON_READ, 0.75); + } + + private void runBenchmark(RowLevelOperationMode mode, double updatePercentage) { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", + TABLE_NAME, TableProperties.UPDATE_MODE, mode.modeName()); + + int mod = (int) (NUM_ROWS_PER_FILE / (NUM_ROWS_PER_FILE * updatePercentage)); + + sql( + "UPDATE %s " + + "SET intCol = intCol + 10, dateCol = date_add(dateCol, 1) " + + "WHERE mod(id, %d) = 0", + TABLE_NAME, mod); + + sql( + "CALL system.rollback_to_snapshot(table => '%s', snapshot_id => %dL)", + TABLE_NAME, originalSnapshotId); + } + + private void setupSpark() { + this.spark = + SparkSession.builder() + .config("spark.ui.enabled", false) + .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName()) + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) + .config(SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED().key(), "false") + .config(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), "false") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "2") + .master("local") + .getOrCreate(); + } + + private void tearDownSpark() { + spark.stop(); + } + + private void initTable() { + sql( + "CREATE TABLE %s ( " + + " id LONG, intCol INT, floatCol FLOAT, doubleCol DOUBLE, " + + " decimalCol DECIMAL(20, 5), dateCol DATE, timestampCol TIMESTAMP, " + + " stringCol STRING)" + + "USING iceberg " + + "TBLPROPERTIES (" + + " '%s' '%s'," + + " '%s' '%d'," + + " '%s' '%d')", + TABLE_NAME, + TableProperties.UPDATE_DISTRIBUTION_MODE, + DistributionMode.NONE.modeName(), + TableProperties.SPLIT_OPEN_FILE_COST, + Integer.MAX_VALUE, + TableProperties.FORMAT_VERSION, + 2); + + sql("ALTER TABLE %s WRITE ORDERED BY id", TABLE_NAME); + } + + private void dropTable() { + sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME); + } + + private void appendData() throws NoSuchTableException { + for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) { + Dataset inputDF = + spark + .range(NUM_ROWS_PER_FILE) + .withColumn("intCol", expr("CAST(id AS INT)")) + .withColumn("floatCol", expr("CAST(id AS FLOAT)")) + .withColumn("doubleCol", expr("CAST(id AS DOUBLE)")) + .withColumn("decimalCol", expr("CAST(id AS DECIMAL(20, 5))")) + .withColumn("dateCol", date_add(current_date(), fileNum)) + .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)")) + .withColumn("stringCol", expr("CAST(dateCol AS STRING)")); + appendAsFile(inputDF); + } + } + + private void appendAsFile(Dataset df) throws NoSuchTableException { + // ensure the schema is precise (including nullability) + StructType sparkSchema = spark.table(TABLE_NAME).schema(); + spark.createDataFrame(df.rdd(), sparkSchema).coalesce(1).writeTo(TABLE_NAME).append(); + } + + private String newWarehouseDir() { + return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID(); + } + + @FormatMethod + private void sql(@FormatString String query, Object... args) { + spark.sql(String.format(query, args)); + } +} diff --git a/spark/v4.0/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 b/spark/v4.0/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 new file mode 100644 index 000000000000..4c2a16d7b19a --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4 @@ -0,0 +1,367 @@ +/* + * 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. + * + * This file is an adaptation of Presto's and Spark's grammar files. + */ + +grammar IcebergSqlExtensions; + +@lexer::members { + /** + * Verify whether current token is a valid decimal token (which contains dot). + * Returns true if the character that follows the token is not a digit or letter or underscore. + * + * For example: + * For char stream "2.3", "2." is not a valid decimal token, because it is followed by digit '3'. + * For char stream "2.3_", "2.3" is not a valid decimal token, because it is followed by '_'. + * For char stream "2.3W", "2.3" is not a valid decimal token, because it is followed by 'W'. + * For char stream "12.0D 34.E2+0.12 " 12.0D is a valid decimal token because it is followed + * by a space. 34.E2 is a valid decimal token because it is followed by symbol '+' + * which is not a digit or letter or underscore. + */ + public boolean isValidDecimal() { + int nextChar = _input.LA(1); + if (nextChar >= 'A' && nextChar <= 'Z' || nextChar >= '0' && nextChar <= '9' || + nextChar == '_') { + return false; + } else { + return true; + } + } + + /** + * This method will be called when we see '/*' and try to match it as a bracketed comment. + * If the next character is '+', it should be parsed as hint later, and we cannot match + * it as a bracketed comment. + * + * Returns true if the next character is '+'. + */ + public boolean isHint() { + int nextChar = _input.LA(1); + if (nextChar == '+') { + return true; + } else { + return false; + } + } +} + +singleStatement + : statement EOF + ; + +statement + : ALTER TABLE multipartIdentifier ADD PARTITION FIELD transform (AS name=identifier)? #addPartitionField + | ALTER TABLE multipartIdentifier DROP PARTITION FIELD transform #dropPartitionField + | ALTER TABLE multipartIdentifier REPLACE PARTITION FIELD transform WITH transform (AS name=identifier)? #replacePartitionField + | ALTER TABLE multipartIdentifier WRITE writeSpec #setWriteDistributionAndOrdering + | ALTER TABLE multipartIdentifier SET IDENTIFIER_KW FIELDS fieldList #setIdentifierFields + | ALTER TABLE multipartIdentifier DROP IDENTIFIER_KW FIELDS fieldList #dropIdentifierFields + | ALTER TABLE multipartIdentifier createReplaceBranchClause #createOrReplaceBranch + | ALTER TABLE multipartIdentifier createReplaceTagClause #createOrReplaceTag + | ALTER TABLE multipartIdentifier DROP BRANCH (IF EXISTS)? identifier #dropBranch + | ALTER TABLE multipartIdentifier DROP TAG (IF EXISTS)? identifier #dropTag + ; + +createReplaceTagClause + : (CREATE OR)? REPLACE TAG identifier tagOptions + | CREATE TAG (IF NOT EXISTS)? identifier tagOptions + ; + +createReplaceBranchClause + : (CREATE OR)? REPLACE BRANCH identifier branchOptions + | CREATE BRANCH (IF NOT EXISTS)? identifier branchOptions + ; + +tagOptions + : (AS OF VERSION snapshotId)? (refRetain)? + ; + +branchOptions + : (AS OF VERSION snapshotId)? (refRetain)? (snapshotRetention)? + ; + +snapshotRetention + : WITH SNAPSHOT RETENTION minSnapshotsToKeep + | WITH SNAPSHOT RETENTION maxSnapshotAge + | WITH SNAPSHOT RETENTION minSnapshotsToKeep maxSnapshotAge + ; + +refRetain + : RETAIN number timeUnit + ; + +maxSnapshotAge + : number timeUnit + ; + +minSnapshotsToKeep + : number SNAPSHOTS + ; + +writeSpec + : (writeDistributionSpec | writeOrderingSpec)* + ; + +writeDistributionSpec + : DISTRIBUTED BY PARTITION + ; + +writeOrderingSpec + : LOCALLY? ORDERED BY order + | UNORDERED + ; + +singleOrder + : order EOF + ; + +order + : fields+=orderField (',' fields+=orderField)* + | '(' fields+=orderField (',' fields+=orderField)* ')' + ; + +orderField + : transform direction=(ASC | DESC)? (NULLS nullOrder=(FIRST | LAST))? + ; + +transform + : multipartIdentifier #identityTransform + | transformName=identifier + '(' arguments+=transformArgument (',' arguments+=transformArgument)* ')' #applyTransform + ; + +transformArgument + : multipartIdentifier + | constant + ; + +expression + : constant + | stringMap + | stringArray + ; + +constant + : number #numericLiteral + | booleanValue #booleanLiteral + | STRING+ #stringLiteral + | identifier STRING #typeConstructor + ; + +stringMap + : MAP '(' constant (',' constant)* ')' + ; + +stringArray + : ARRAY '(' constant (',' constant)* ')' + ; + +booleanValue + : TRUE | FALSE + ; + +number + : MINUS? EXPONENT_VALUE #exponentLiteral + | MINUS? DECIMAL_VALUE #decimalLiteral + | MINUS? INTEGER_VALUE #integerLiteral + | MINUS? BIGINT_LITERAL #bigIntLiteral + | MINUS? SMALLINT_LITERAL #smallIntLiteral + | MINUS? TINYINT_LITERAL #tinyIntLiteral + | MINUS? DOUBLE_LITERAL #doubleLiteral + | MINUS? FLOAT_LITERAL #floatLiteral + | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral + ; + +multipartIdentifier + : parts+=identifier ('.' parts+=identifier)* + ; + +identifier + : IDENTIFIER #unquotedIdentifier + | quotedIdentifier #quotedIdentifierAlternative + | nonReserved #unquotedIdentifier + ; + +quotedIdentifier + : BACKQUOTED_IDENTIFIER + ; + +fieldList + : fields+=multipartIdentifier (',' fields+=multipartIdentifier)* + ; + +nonReserved + : ADD | ALTER | AS | ASC | BRANCH | BY | CREATE | DAYS | DESC | DROP | EXISTS | FIELD | FIRST | HOURS | IF | LAST | NOT | NULLS | OF | OR | ORDERED | PARTITION | TABLE | WRITE + | DISTRIBUTED | LOCALLY | MINUTES | MONTHS | UNORDERED | REPLACE | RETAIN | VERSION | WITH | IDENTIFIER_KW | FIELDS | SET | SNAPSHOT | SNAPSHOTS + | TAG | TRUE | FALSE + | MAP + ; + +snapshotId + : number + ; + +numSnapshots + : number + ; + +timeUnit + : DAYS + | HOURS + | MINUTES + ; + +ADD: 'ADD'; +ALTER: 'ALTER'; +AS: 'AS'; +ASC: 'ASC'; +BRANCH: 'BRANCH'; +BY: 'BY'; +DAYS: 'DAYS'; +DESC: 'DESC'; +DISTRIBUTED: 'DISTRIBUTED'; +DROP: 'DROP'; +EXISTS: 'EXISTS'; +FIELD: 'FIELD'; +FIELDS: 'FIELDS'; +FIRST: 'FIRST'; +HOURS: 'HOURS'; +IF : 'IF'; +LAST: 'LAST'; +LOCALLY: 'LOCALLY'; +MINUTES: 'MINUTES'; +MONTHS: 'MONTHS'; +CREATE: 'CREATE'; +NOT: 'NOT'; +NULLS: 'NULLS'; +OF: 'OF'; +OR: 'OR'; +ORDERED: 'ORDERED'; +PARTITION: 'PARTITION'; +REPLACE: 'REPLACE'; +RETAIN: 'RETAIN'; +RETENTION: 'RETENTION'; +IDENTIFIER_KW: 'IDENTIFIER'; +SET: 'SET'; +SNAPSHOT: 'SNAPSHOT'; +SNAPSHOTS: 'SNAPSHOTS'; +TABLE: 'TABLE'; +TAG: 'TAG'; +UNORDERED: 'UNORDERED'; +VERSION: 'VERSION'; +WITH: 'WITH'; +WRITE: 'WRITE'; + +TRUE: 'TRUE'; +FALSE: 'FALSE'; + +MAP: 'MAP'; +ARRAY: 'ARRAY'; + +PLUS: '+'; +MINUS: '-'; + +STRING + : '\'' ( ~('\''|'\\') | ('\\' .) )* '\'' + | '"' ( ~('"'|'\\') | ('\\' .) )* '"' + ; + +BIGINT_LITERAL + : DIGIT+ 'L' + ; + +SMALLINT_LITERAL + : DIGIT+ 'S' + ; + +TINYINT_LITERAL + : DIGIT+ 'Y' + ; + +INTEGER_VALUE + : DIGIT+ + ; + +EXPONENT_VALUE + : DIGIT+ EXPONENT + | DECIMAL_DIGITS EXPONENT {isValidDecimal()}? + ; + +DECIMAL_VALUE + : DECIMAL_DIGITS {isValidDecimal()}? + ; + +FLOAT_LITERAL + : DIGIT+ EXPONENT? 'F' + | DECIMAL_DIGITS EXPONENT? 'F' {isValidDecimal()}? + ; + +DOUBLE_LITERAL + : DIGIT+ EXPONENT? 'D' + | DECIMAL_DIGITS EXPONENT? 'D' {isValidDecimal()}? + ; + +BIGDECIMAL_LITERAL + : DIGIT+ EXPONENT? 'BD' + | DECIMAL_DIGITS EXPONENT? 'BD' {isValidDecimal()}? + ; + +IDENTIFIER + : (LETTER | DIGIT | '_')+ + ; + +BACKQUOTED_IDENTIFIER + : '`' ( ~'`' | '``' )* '`' + ; + +fragment DECIMAL_DIGITS + : DIGIT+ '.' DIGIT* + | '.' DIGIT+ + ; + +fragment EXPONENT + : 'E' [+-]? DIGIT+ + ; + +fragment DIGIT + : [0-9] + ; + +fragment LETTER + : [A-Z] + ; + +SIMPLE_COMMENT + : '--' ('\\\n' | ~[\r\n])* '\r'? '\n'? -> channel(HIDDEN) + ; + +BRACKETED_COMMENT + : '/*' {!isHint()}? (BRACKETED_COMMENT|.)*? '*/' -> channel(HIDDEN) + ; + +WS + : [ \r\n\t]+ -> channel(HIDDEN) + ; + +// Catch-all for anything we can't recognize. +// We use this to be able to ignore and recover all the text +// when splitting statements with DelimiterLexer +UNRECOGNIZED + : . + ; diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala new file mode 100644 index 000000000000..c4de35010c6e --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/iceberg/spark/extensions/IcebergSparkSessionExtensions.scala @@ -0,0 +1,44 @@ +/* + * 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.extensions + +import org.apache.spark.sql.SparkSessionExtensions +import org.apache.spark.sql.catalyst.analysis.CheckViews +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 +import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Strategy + +class IcebergSparkSessionExtensions extends (SparkSessionExtensions => Unit) { + + override def apply(extensions: SparkSessionExtensions): Unit = { + // parser extensions + extensions.injectParser { case (_, parser) => new IcebergSparkSqlExtensionsParser(parser) } + + // analyzer extensions + extensions.injectResolutionRule { spark => ResolveViews(spark) } + extensions.injectCheckRule(_ => CheckViews) + + // optimizer extensions + extensions.injectOptimizerRule { _ => ReplaceStaticInvoke } + + // planner extensions + extensions.injectPlannerStrategy { spark => ExtendedDataSourceV2Strategy(spark) } + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala new file mode 100644 index 000000000000..5ad4b9c01409 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -0,0 +1,135 @@ +/* + * 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.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.AlterViewAs +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias +import org.apache.spark.sql.catalyst.plans.logical.View +import org.apache.spark.sql.catalyst.plans.logical.views.CreateIcebergView +import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View +import org.apache.spark.sql.connector.catalog.ViewCatalog +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.util.SchemaUtils + +object CheckViews extends (LogicalPlan => Unit) { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override def apply(plan: LogicalPlan): Unit = { + plan foreach { + case CreateIcebergView( + resolvedIdent @ ResolvedIdentifier(_: ViewCatalog, _), + _, + query, + columnAliases, + _, + _, + _, + _, + _, + replace, + _, + _) => + verifyColumnCount(resolvedIdent, columnAliases, query) + SchemaUtils.checkColumnNameDuplication( + query.schema.fieldNames.toIndexedSeq, + SQLConf.get.resolver) + if (replace) { + val viewIdent: Seq[String] = + resolvedIdent.catalog.name() +: resolvedIdent.identifier.asMultipartIdentifier + checkCyclicViewReference(viewIdent, query, Seq(viewIdent)) + } + + case AlterViewAs(ResolvedV2View(_, _), _, _) => + throw new IcebergAnalysisException( + "ALTER VIEW AS is not supported. Use CREATE OR REPLACE VIEW instead") + + case _ => // OK + } + } + + private def verifyColumnCount( + ident: ResolvedIdentifier, + columns: Seq[String], + query: LogicalPlan): Unit = { + if (columns.nonEmpty) { + if (columns.length > query.output.length) { + throw new AnalysisException( + errorClass = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", + messageParameters = Map( + "viewName" -> String.format("%s.%s", ident.catalog.name(), ident.identifier), + "viewColumns" -> columns.mkString(", "), + "dataColumns" -> query.output.map(c => c.name).mkString(", "))) + } else if (columns.length < query.output.length) { + throw new AnalysisException( + errorClass = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", + messageParameters = Map( + "viewName" -> String.format("%s.%s", ident.catalog.name(), ident.identifier), + "viewColumns" -> columns.mkString(", "), + "dataColumns" -> query.output.map(c => c.name).mkString(", "))) + } + } + } + + private def checkCyclicViewReference( + viewIdent: Seq[String], + plan: LogicalPlan, + cyclePath: Seq[Seq[String]]): Unit = { + plan match { + case sub @ SubqueryAlias(_, Project(_, _)) => + val currentViewIdent: Seq[String] = sub.identifier.qualifier :+ sub.identifier.name + checkIfRecursiveView(viewIdent, currentViewIdent, cyclePath, sub.children) + case v1View: View => + val currentViewIdent: Seq[String] = v1View.desc.identifier.nameParts + checkIfRecursiveView(viewIdent, currentViewIdent, cyclePath, v1View.children) + case _ => + plan.children.foreach(child => checkCyclicViewReference(viewIdent, child, cyclePath)) + } + + plan.expressions.flatMap(_.flatMap { + case e: SubqueryExpression => + checkCyclicViewReference(viewIdent, e.plan, cyclePath) + None + case _ => None + }) + } + + private def checkIfRecursiveView( + viewIdent: Seq[String], + currentViewIdent: Seq[String], + cyclePath: Seq[Seq[String]], + children: Seq[LogicalPlan]): Unit = { + val newCyclePath = cyclePath :+ currentViewIdent + if (currentViewIdent == viewIdent) { + throw new IcebergAnalysisException( + String.format( + "Recursive cycle in view detected: %s (cycle: %s)", + viewIdent.asIdentifier, + newCyclePath.map(p => p.mkString(".")).mkString(" -> "))) + } else { + children.foreach { c => + checkCyclicViewReference(viewIdent, c, newCyclePath) + } + } + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala new file mode 100644 index 000000000000..ff7d20241bed --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.ViewUtil.IcebergViewHelper +import org.apache.spark.sql.catalyst.expressions.Alias +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.expressions.UpCast +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias +import org.apache.spark.sql.catalyst.plans.logical.views.CreateIcebergView +import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.CurrentOrigin +import org.apache.spark.sql.catalyst.trees.Origin +import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.connector.catalog.LookupCatalog +import org.apache.spark.sql.connector.catalog.View +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.types.MetadataBuilder + +case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with LookupCatalog { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager + + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case u @ UnresolvedRelation(nameParts, _, _) + if catalogManager.v1SessionCatalog.isTempView(nameParts) => + u + + case u @ UnresolvedRelation(parts @ CatalogAndIdentifier(catalog, ident), _, _) => + ViewUtil + .loadView(catalog, ident) + .map(createViewRelation(parts, _)) + .getOrElse(u) + + case u @ UnresolvedTableOrView(CatalogAndIdentifier(catalog, ident), _, _) => + ViewUtil + .loadView(catalog, ident) + .map(_ => ResolvedV2View(catalog.asViewCatalog, ident)) + .getOrElse(u) + + case c @ CreateIcebergView( + ResolvedIdentifier(_, _), + _, + query, + columnAliases, + columnComments, + _, + _, + _, + _, + _, + _, + _) if query.resolved && !c.rewritten => + val aliased = aliasColumns(query, columnAliases, columnComments) + c.copy( + query = aliased, + queryColumnNames = query.schema.fieldNames.toIndexedSeq, + rewritten = true) + } + + private def aliasColumns( + plan: LogicalPlan, + columnAliases: Seq[String], + columnComments: Seq[Option[String]]): LogicalPlan = { + if (columnAliases.isEmpty || columnAliases.length != plan.output.length) { + plan + } else { + val projectList = plan.output.zipWithIndex.map { case (attr, pos) => + if (columnComments.apply(pos).isDefined) { + val meta = + new MetadataBuilder().putString("comment", columnComments.apply(pos).get).build() + Alias(attr, columnAliases.apply(pos))(explicitMetadata = Some(meta)) + } else { + Alias(attr, columnAliases.apply(pos))() + } + } + Project(projectList, plan) + } + } + + private def createViewRelation(nameParts: Seq[String], view: View): LogicalPlan = { + val parsed = parseViewText(nameParts.quoted, view.query) + + // Apply any necessary rewrites to preserve correct resolution + val viewCatalogAndNamespace: Seq[String] = view.currentCatalog +: view.currentNamespace.toSeq + val rewritten = rewriteIdentifiers(parsed, viewCatalogAndNamespace); + + // Apply the field aliases and column comments + // This logic differs from how Spark handles views in SessionCatalog.fromCatalogTable. + // This is more strict because it doesn't allow resolution by field name. + val aliases = view.schema.fields.zipWithIndex.map { case (expected, pos) => + val attr = GetColumnByOrdinal(pos, expected.dataType) + Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = + Some(expected.metadata)) + }.toIndexedSeq + + SubqueryAlias(nameParts, Project(aliases, rewritten)) + } + + private def parseViewText(name: String, viewText: String): LogicalPlan = { + val origin = Origin(objectType = Some("VIEW"), objectName = Some(name)) + + try { + CurrentOrigin.withOrigin(origin) { + spark.sessionState.sqlParser.parseQuery(viewText) + } + } catch { + case _: ParseException => + throw QueryCompilationErrors.invalidViewNameError(name) + } + } + + private def rewriteIdentifiers( + plan: LogicalPlan, + catalogAndNamespace: Seq[String]): LogicalPlan = { + // Substitute CTEs and Unresolved Ordinals within the view, then rewrite unresolved functions and relations + qualifyTableIdentifiers( + qualifyFunctionIdentifiers( + SubstituteUnresolvedOrdinals.apply(CTESubstitution.apply(plan)), + catalogAndNamespace), + catalogAndNamespace) + } + + private def qualifyFunctionIdentifiers( + plan: LogicalPlan, + catalogAndNamespace: Seq[String]): LogicalPlan = plan transformExpressions { + case u @ UnresolvedFunction(Seq(name), _, _, _, _, _, _) => + if (!isBuiltinFunction(name)) { + u.copy(nameParts = catalogAndNamespace :+ name) + } else { + u + } + case u @ UnresolvedFunction(parts, _, _, _, _, _, _) if !isCatalog(parts.head) => + u.copy(nameParts = catalogAndNamespace.head +: parts) + } + + /** + * Qualify table identifiers with default catalog and namespace if necessary. + */ + private def qualifyTableIdentifiers( + child: LogicalPlan, + catalogAndNamespace: Seq[String]): LogicalPlan = + child transform { + case u @ UnresolvedRelation(Seq(table), _, _) => + u.copy(multipartIdentifier = catalogAndNamespace :+ table) + case u @ UnresolvedRelation(parts, _, _) if !isCatalog(parts.head) => + u.copy(multipartIdentifier = catalogAndNamespace.head +: parts) + case other => + other.transformExpressions { case subquery: SubqueryExpression => + subquery.withNewPlan(qualifyTableIdentifiers(subquery.plan, catalogAndNamespace)) + } + } + + private def isCatalog(name: String): Boolean = { + catalogManager.isCatalogRegistered(name) + } + + private def isBuiltinFunction(name: String): Boolean = { + catalogManager.v1SessionCatalog.isBuiltinFunction(FunctionIdentifier(name)) + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala new file mode 100644 index 000000000000..ac0f75c422d1 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -0,0 +1,210 @@ +/* + * 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.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.ViewUtil.IcebergViewHelper +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.CreateView +import org.apache.spark.sql.catalyst.plans.logical.DropView +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.ShowViews +import org.apache.spark.sql.catalyst.plans.logical.View +import org.apache.spark.sql.catalyst.plans.logical.views.CreateIcebergView +import org.apache.spark.sql.catalyst.plans.logical.views.DropIcebergView +import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View +import org.apache.spark.sql.catalyst.plans.logical.views.ShowIcebergViews +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_FUNCTION +import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.connector.catalog.LookupCatalog +import scala.collection.mutable + +/** + * ResolveSessionCatalog exits early for some v2 View commands, + * thus they are pre-substituted here and then handled in ResolveViews + */ +case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] with LookupCatalog { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + case DropView(ResolvedIdent(resolved), ifExists) => + DropIcebergView(resolved, ifExists) + + case CreateView( + ResolvedIdent(resolved), + userSpecifiedColumns, + comment, + _, + properties, + Some(queryText), + query, + allowExisting, + replace, + _) => + val q = CTESubstitution.apply(query) + verifyTemporaryObjectsDontExist(resolved, q) + CreateIcebergView( + child = resolved, + queryText = queryText, + query = q, + columnAliases = userSpecifiedColumns.map(_._1), + columnComments = userSpecifiedColumns.map(_._2.orElse(Option.empty)), + comment = comment, + properties = properties, + allowExisting = allowExisting, + replace = replace) + + case view @ ShowViews(CurrentNamespace, pattern, output) => + if (ViewUtil.isViewCatalog(catalogManager.currentCatalog)) { + ShowIcebergViews( + ResolvedNamespace( + catalogManager.currentCatalog, + catalogManager.currentNamespace.toIndexedSeq), + pattern, + output) + } else { + view + } + + case ShowViews(UnresolvedNamespace(CatalogAndNamespace(catalog, ns), _), pattern, output) + if ViewUtil.isViewCatalog(catalog) => + ShowIcebergViews(ResolvedNamespace(catalog, ns), pattern, output) + + // needs to be done here instead of in ResolveViews, so that a V2 view can be resolved before the Analyzer + // tries to resolve it, which would result in an error, saying that V2 views aren't supported + case u @ UnresolvedView(ResolvedView(resolved), _, _, _) => + ViewUtil + .loadView(resolved.catalog, resolved.identifier) + .map(_ => ResolvedV2View(resolved.catalog.asViewCatalog, resolved.identifier)) + .getOrElse(u) + } + + private def isTempView(nameParts: Seq[String]): Boolean = { + catalogManager.v1SessionCatalog.isTempView(nameParts) + } + + private def isTempFunction(nameParts: Seq[String]): Boolean = { + if (nameParts.size > 1) { + return false + } + catalogManager.v1SessionCatalog.isTemporaryFunction(nameParts.asFunctionIdentifier) + } + + private object ResolvedIdent { + def unapply(unresolved: UnresolvedIdentifier): Option[ResolvedIdentifier] = unresolved match { + case UnresolvedIdentifier(nameParts, true) if isTempView(nameParts) => + None + + case UnresolvedIdentifier(CatalogAndIdentifier(catalog, ident), _) + if ViewUtil.isViewCatalog(catalog) => + Some(ResolvedIdentifier(catalog, ident)) + + case _ => + None + } + } + + /** + * Permanent views are not allowed to reference temp objects + */ + private def verifyTemporaryObjectsDontExist( + identifier: ResolvedIdentifier, + child: LogicalPlan): Unit = { + val tempViews = collectTemporaryViews(child) + if (tempViews.nonEmpty) { + throw invalidRefToTempObject( + identifier, + tempViews.map(v => v.quoted).mkString("[", ", ", "]"), + "view") + } + + val tempFunctions = collectTemporaryFunctions(child) + if (tempFunctions.nonEmpty) { + throw invalidRefToTempObject(identifier, tempFunctions.mkString("[", ", ", "]"), "function") + } + } + + private def invalidRefToTempObject( + ident: ResolvedIdentifier, + tempObjectNames: String, + tempObjectType: String) = { + new IcebergAnalysisException( + String.format( + "Cannot create view %s.%s that references temporary %s: %s", + ident.catalog.name(), + ident.identifier, + tempObjectType, + tempObjectNames)) + } + + /** + * Collect all temporary views and return the identifiers separately + */ + private def collectTemporaryViews(child: LogicalPlan): Seq[Seq[String]] = { + def collectTempViews(child: LogicalPlan): Seq[Seq[String]] = { + child.flatMap { + case unresolved: UnresolvedRelation if isTempView(unresolved.multipartIdentifier) => + Seq(unresolved.multipartIdentifier) + case view: View if view.isTempView => Seq(view.desc.identifier.nameParts) + case plan => + plan.expressions.flatMap(_.flatMap { + case e: SubqueryExpression => collectTempViews(e.plan) + case _ => Seq.empty + }) + }.distinct + } + + collectTempViews(child) + } + + private object ResolvedView { + def unapply(identifier: Seq[String]): Option[ResolvedV2View] = identifier match { + case nameParts if isTempView(nameParts) => + None + + case CatalogAndIdentifier(catalog, ident) if ViewUtil.isViewCatalog(catalog) => + ViewUtil + .loadView(catalog, ident) + .flatMap(_ => Some(ResolvedV2View(catalog.asViewCatalog, ident))) + + case _ => + None + } + } + + /** + * Collect the names of all temporary functions. + */ + private def collectTemporaryFunctions(child: LogicalPlan): Seq[String] = { + val tempFunctions = new mutable.HashSet[String]() + child.resolveExpressionsWithPruning(_.containsAnyPattern(UNRESOLVED_FUNCTION)) { + case f @ UnresolvedFunction(nameParts, _, _, _, _, _, _) if isTempFunction(nameParts) => + tempFunctions += nameParts.head + f + case e: SubqueryExpression => + tempFunctions ++= collectTemporaryFunctions(e.plan) + e + } + tempFunctions.toSeq + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala new file mode 100644 index 000000000000..a7188837c51e --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala @@ -0,0 +1,50 @@ +/* + * 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.spark.sql.connector.catalog.CatalogPlugin +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.View +import org.apache.spark.sql.connector.catalog.ViewCatalog +import org.apache.spark.sql.errors.QueryCompilationErrors + +object ViewUtil { + def loadView(catalog: CatalogPlugin, ident: Identifier): Option[View] = catalog match { + case viewCatalog: ViewCatalog => + try { + Option(viewCatalog.loadView(ident)) + } catch { + case _: NoSuchViewException => None + } + case _ => None + } + + def isViewCatalog(catalog: CatalogPlugin): Boolean = { + catalog.isInstanceOf[ViewCatalog] + } + + implicit class IcebergViewHelper(plugin: CatalogPlugin) { + def asViewCatalog: ViewCatalog = plugin match { + case viewCatalog: ViewCatalog => + viewCatalog + case _ => + throw QueryCompilationErrors.missingCatalogAbilityError(plugin, "views") + } + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala new file mode 100644 index 000000000000..bdec4aae884d --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala @@ -0,0 +1,120 @@ +/* + * 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.optimizer + +import org.apache.iceberg.spark.functions.SparkFunctions +import org.apache.spark.sql.catalyst.expressions.ApplyFunctionExpression +import org.apache.spark.sql.catalyst.expressions.BinaryComparison +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.In +import org.apache.spark.sql.catalyst.expressions.InSet +import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke +import org.apache.spark.sql.catalyst.plans.logical.Filter +import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.ReplaceData +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.BINARY_COMPARISON +import org.apache.spark.sql.catalyst.trees.TreePattern.COMMAND +import org.apache.spark.sql.catalyst.trees.TreePattern.FILTER +import org.apache.spark.sql.catalyst.trees.TreePattern.IN +import org.apache.spark.sql.catalyst.trees.TreePattern.INSET +import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN +import org.apache.spark.sql.connector.catalog.functions.ScalarFunction +import org.apache.spark.sql.types.StructField +import org.apache.spark.sql.types.StructType + +/** + * Spark analyzes the Iceberg system function to {@link StaticInvoke} which could not be pushed + * down to datasource. This rule will replace {@link StaticInvoke} to + * {@link ApplyFunctionExpression} for Iceberg system function in a filter condition. + */ +object ReplaceStaticInvoke extends Rule[LogicalPlan] { + + override def apply(plan: LogicalPlan): LogicalPlan = + plan.transformWithPruning(_.containsAnyPattern(COMMAND, FILTER, JOIN)) { + case replace @ ReplaceData(_, cond, _, _, _, _, _) => + replaceStaticInvoke(replace, cond, newCond => replace.copy(condition = newCond)) + + case join @ Join(_, _, _, Some(cond), _) => + replaceStaticInvoke(join, cond, newCond => join.copy(condition = Some(newCond))) + + case filter @ Filter(cond, _) => + replaceStaticInvoke(filter, cond, newCond => filter.copy(condition = newCond)) + } + + private def replaceStaticInvoke[T <: LogicalPlan]( + node: T, + condition: Expression, + copy: Expression => T): T = { + val newCondition = replaceStaticInvoke(condition) + if (newCondition fastEquals condition) node else copy(newCondition) + } + + private def replaceStaticInvoke(condition: Expression): Expression = { + condition.transformWithPruning(_.containsAnyPattern(BINARY_COMPARISON, IN, INSET)) { + case in @ In(value: StaticInvoke, _) if canReplace(value) => + in.copy(value = replaceStaticInvoke(value)) + + case in @ InSet(value: StaticInvoke, _) if canReplace(value) => + in.copy(child = replaceStaticInvoke(value)) + + case c @ BinaryComparison(left: StaticInvoke, right) if canReplace(left) && right.foldable => + c.withNewChildren(Seq(replaceStaticInvoke(left), right)) + + case c @ BinaryComparison(left, right: StaticInvoke) if canReplace(right) && left.foldable => + c.withNewChildren(Seq(left, replaceStaticInvoke(right))) + } + } + + private def replaceStaticInvoke(invoke: StaticInvoke): Expression = { + // Adaptive from `resolveV2Function` in org.apache.spark.sql.catalyst.analysis.ResolveFunctions + val unbound = SparkFunctions.loadFunctionByClass(invoke.staticObject) + if (unbound == null) { + return invoke + } + + val inputType = StructType(invoke.arguments.zipWithIndex.map { case (exp, pos) => + StructField(s"_$pos", exp.dataType, exp.nullable) + }) + + val bound = + try { + unbound.bind(inputType) + } catch { + case _: Exception => + return invoke + } + + if (bound.inputTypes().length != invoke.arguments.length) { + return invoke + } + + bound match { + case scalarFunc: ScalarFunction[_] => + ApplyFunctionExpression(scalarFunc, invoke.arguments) + case _ => invoke + } + } + + @inline + private def canReplace(invoke: StaticInvoke): Boolean = { + invoke.functionName == ScalarFunction.MAGIC_METHOD_NAME && !invoke.foldable + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala new file mode 100644 index 000000000000..25e056ee2d96 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala @@ -0,0 +1,342 @@ +/* + * 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.parser.extensions + +import java.util.Locale +import org.antlr.v4.runtime._ +import org.antlr.v4.runtime.atn.PredictionMode +import org.antlr.v4.runtime.misc.Interval +import org.antlr.v4.runtime.misc.ParseCancellationException +import org.antlr.v4.runtime.tree.TerminalNodeImpl +import org.apache.iceberg.common.DynConstructors +import org.apache.iceberg.spark.ExtendedParser +import org.apache.iceberg.spark.ExtendedParser.RawOrderField +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.RewriteViewCommands +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.parser.extensions.IcebergSqlExtensionsParser.NonReservedContext +import org.apache.spark.sql.catalyst.parser.extensions.IcebergSqlExtensionsParser.QuotedIdentifierContext +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees.Origin +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.VariableSubstitution +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.StructType +import scala.jdk.CollectionConverters._ + +class IcebergSparkSqlExtensionsParser(delegate: ParserInterface) + extends ParserInterface + with ExtendedParser { + + import IcebergSparkSqlExtensionsParser._ + + private lazy val substitutor = substitutorCtor.newInstance(SQLConf.get) + private lazy val astBuilder = new IcebergSqlExtensionsAstBuilder(delegate) + + /** + * Parse a string to a DataType. + */ + override def parseDataType(sqlText: String): DataType = { + delegate.parseDataType(sqlText) + } + + /** + * Parse a string to a raw DataType without CHAR/VARCHAR replacement. + */ + def parseRawDataType(sqlText: String): DataType = throw new UnsupportedOperationException() + + /** + * Parse a string to an Expression. + */ + override def parseExpression(sqlText: String): Expression = { + delegate.parseExpression(sqlText) + } + + /** + * Parse a string to a TableIdentifier. + */ + override def parseTableIdentifier(sqlText: String): TableIdentifier = { + delegate.parseTableIdentifier(sqlText) + } + + /** + * Parse a string to a FunctionIdentifier. + */ + override def parseFunctionIdentifier(sqlText: String): FunctionIdentifier = { + delegate.parseFunctionIdentifier(sqlText) + } + + /** + * Parse a string to a multi-part identifier. + */ + override def parseMultipartIdentifier(sqlText: String): Seq[String] = { + delegate.parseMultipartIdentifier(sqlText) + } + + /** + * Creates StructType for a given SQL string, which is a comma separated list of field + * definitions which will preserve the correct Hive metadata. + */ + override def parseTableSchema(sqlText: String): StructType = { + delegate.parseTableSchema(sqlText) + } + + override def parseSortOrder(sqlText: String): java.util.List[RawOrderField] = { + val fields = parse(sqlText) { parser => astBuilder.visitSingleOrder(parser.singleOrder()) } + fields.map { field => + val (term, direction, order) = field + new RawOrderField(term, direction, order) + }.asJava + } + + override def parseRoutineParam(sqlText: String): StructType = + throw new UnsupportedOperationException() + + /** + * Parse a string to a LogicalPlan. + */ + override def parsePlan(sqlText: String): LogicalPlan = { + val sqlTextAfterSubstitution = substitutor.substitute(sqlText) + if (isIcebergCommand(sqlTextAfterSubstitution)) { + parse(sqlTextAfterSubstitution) { parser => astBuilder.visit(parser.singleStatement()) } + .asInstanceOf[LogicalPlan] + } else { + RewriteViewCommands(SparkSession.active).apply(delegate.parsePlan(sqlText)) + } + } + + private def isIcebergCommand(sqlText: String): Boolean = { + val normalized = sqlText + .toLowerCase(Locale.ROOT) + .trim() + // Strip simple SQL comments that terminate a line, e.g. comments starting with `--` . + .replaceAll("--.*?\\n", " ") + // Strip newlines. + .replaceAll("\\s+", " ") + // Strip comments of the form /* ... */. This must come after stripping newlines so that + // comments that span multiple lines are caught. + .replaceAll("/\\*.*?\\*/", " ") + // Strip backtick then `system`.`ancestors_of` changes to system.ancestors_of + .replaceAll("`", "") + .trim() + + normalized.startsWith("alter table") && (normalized.contains("add partition field") || + normalized.contains("drop partition field") || + normalized.contains("replace partition field") || + normalized.contains("write ordered by") || + normalized.contains("write locally ordered by") || + normalized.contains("write distributed by") || + normalized.contains("write unordered") || + normalized.contains("set identifier fields") || + normalized.contains("drop identifier fields") || + isSnapshotRefDdl(normalized)) + } + + private def isSnapshotRefDdl(normalized: String): Boolean = { + normalized.contains("create branch") || + normalized.contains("replace branch") || + normalized.contains("create tag") || + normalized.contains("replace tag") || + normalized.contains("drop branch") || + normalized.contains("drop tag") + } + + protected def parse[T](command: String)(toResult: IcebergSqlExtensionsParser => T): T = { + val lexer = new IcebergSqlExtensionsLexer( + new UpperCaseCharStream(CharStreams.fromString(command))) + lexer.removeErrorListeners() + lexer.addErrorListener(IcebergParseErrorListener) + + val tokenStream = new CommonTokenStream(lexer) + val parser = new IcebergSqlExtensionsParser(tokenStream) + parser.addParseListener(IcebergSqlExtensionsPostProcessor) + parser.removeErrorListeners() + parser.addErrorListener(IcebergParseErrorListener) + + // https://github.com/antlr/antlr4/issues/192#issuecomment-15238595 + // Save a great deal of time on correct inputs by using a two-stage parsing strategy. + try { + try { + // first, try parsing with potentially faster SLL mode and BailErrorStrategy + parser.setErrorHandler(new BailErrorStrategy) + parser.getInterpreter.setPredictionMode(PredictionMode.SLL) + toResult(parser) + } catch { + case _: ParseCancellationException => + // if we fail, parse with LL mode with DefaultErrorStrategy + tokenStream.seek(0) // rewind input stream + parser.reset() + + // Try Again. + parser.setErrorHandler(new DefaultErrorStrategy) + parser.getInterpreter.setPredictionMode(PredictionMode.LL) + toResult(parser) + } + } catch { + case e: IcebergParseException if e.command.isDefined => + throw e + case e: IcebergParseException => + throw e.withCommand(command) + case e: AnalysisException => + val position = Origin(e.line, e.startPosition) + throw new IcebergParseException(Option(command), e.message, position, position) + } + } + + override def parseQuery(sqlText: String): LogicalPlan = { + parsePlan(sqlText) + } +} + +object IcebergSparkSqlExtensionsParser { + private val substitutorCtor: DynConstructors.Ctor[VariableSubstitution] = + DynConstructors + .builder() + .impl(classOf[VariableSubstitution]) + .impl(classOf[VariableSubstitution], classOf[SQLConf]) + .build() +} + +/* Copied from Apache Spark's to avoid dependency on Spark Internals */ +class UpperCaseCharStream(wrapped: CodePointCharStream) extends CharStream { + override def consume(): Unit = wrapped.consume + override def getSourceName(): String = wrapped.getSourceName + override def index(): Int = wrapped.index + override def mark(): Int = wrapped.mark + override def release(marker: Int): Unit = wrapped.release(marker) + override def seek(where: Int): Unit = wrapped.seek(where) + override def size(): Int = wrapped.size + + override def getText(interval: Interval): String = wrapped.getText(interval) + + // scalastyle:off + override def LA(i: Int): Int = { + val la = wrapped.LA(i) + if (la == 0 || la == IntStream.EOF) la + else Character.toUpperCase(la) + } + // scalastyle:on +} + +/** + * The post-processor validates & cleans-up the parse tree during the parse process. + */ +case object IcebergSqlExtensionsPostProcessor extends IcebergSqlExtensionsBaseListener { + + /** Remove the back ticks from an Identifier. */ + override def exitQuotedIdentifier(ctx: QuotedIdentifierContext): Unit = { + replaceTokenByIdentifier(ctx, 1) { token => + // Remove the double back ticks in the string. + token.setText(token.getText.replace("``", "`")) + token + } + } + + /** Treat non-reserved keywords as Identifiers. */ + override def exitNonReserved(ctx: NonReservedContext): Unit = { + replaceTokenByIdentifier(ctx, 0)(identity) + } + + private def replaceTokenByIdentifier(ctx: ParserRuleContext, stripMargins: Int)( + f: CommonToken => CommonToken = identity): Unit = { + val parent = ctx.getParent + parent.removeLastChild() + val token = ctx.getChild(0).getPayload.asInstanceOf[Token] + val newToken = new CommonToken( + new org.antlr.v4.runtime.misc.Pair(token.getTokenSource, token.getInputStream), + IcebergSqlExtensionsParser.IDENTIFIER, + token.getChannel, + token.getStartIndex + stripMargins, + token.getStopIndex - stripMargins) + parent.addChild(new TerminalNodeImpl(f(newToken))) + } +} + +/* Partially copied from Apache Spark's Parser to avoid dependency on Spark Internals */ +case object IcebergParseErrorListener extends BaseErrorListener { + override def syntaxError( + recognizer: Recognizer[_, _], + offendingSymbol: scala.Any, + line: Int, + charPositionInLine: Int, + msg: String, + e: RecognitionException): Unit = { + val (start, stop) = offendingSymbol match { + case token: CommonToken => + val start = Origin(Some(line), Some(token.getCharPositionInLine)) + val length = token.getStopIndex - token.getStartIndex + 1 + val stop = Origin(Some(line), Some(token.getCharPositionInLine + length)) + (start, stop) + case _ => + val start = Origin(Some(line), Some(charPositionInLine)) + (start, start) + } + throw new IcebergParseException(None, msg, start, stop) + } +} + +/** + * Copied from Apache Spark + * A [[ParseException]] is an [[AnalysisException]] that is thrown during the parse process. It + * contains fields and an extended error message that make reporting and diagnosing errors easier. + */ +class IcebergParseException( + val command: Option[String], + message: String, + val start: Origin, + val stop: Origin) + extends AnalysisException(message, start.line, start.startPosition) { + + def this(message: String, ctx: ParserRuleContext) = { + this( + Option(IcebergParserUtils.command(ctx)), + message, + IcebergParserUtils.position(ctx.getStart), + IcebergParserUtils.position(ctx.getStop)) + } + + override def getMessage: String = { + val builder = new StringBuilder + builder ++= "\n" ++= message + start match { + case Origin(Some(l), Some(p), Some(_), Some(_), Some(_), Some(_), Some(_), _, _) => + builder ++= s"(line $l, pos $p)\n" + command.foreach { cmd => + val (above, below) = cmd.split("\n").splitAt(l) + builder ++= "\n== SQL ==\n" + above.foreach(builder ++= _ += '\n') + builder ++= (0 until p).map(_ => "-").mkString("") ++= "^^^\n" + below.foreach(builder ++= _ += '\n') + } + case _ => + command.foreach { cmd => + builder ++= "\n== SQL ==\n" ++= cmd + } + } + builder.toString + } + + def withCommand(cmd: String): IcebergParseException = { + new IcebergParseException(Option(cmd), message, start, stop) + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala new file mode 100644 index 000000000000..724101cfe11d --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSqlExtensionsAstBuilder.scala @@ -0,0 +1,385 @@ +/* + * 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.parser.extensions + +import java.util.Locale +import java.util.concurrent.TimeUnit +import org.antlr.v4.runtime._ +import org.antlr.v4.runtime.misc.Interval +import org.antlr.v4.runtime.tree.ParseTree +import org.antlr.v4.runtime.tree.TerminalNode +import org.apache.iceberg.DistributionMode +import org.apache.iceberg.NullOrder +import org.apache.iceberg.SortDirection +import org.apache.iceberg.expressions.Term +import org.apache.iceberg.spark.Spark3Util +import org.apache.spark.sql.catalyst.analysis.IcebergAnalysisException +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.parser.extensions.IcebergParserUtils.withOrigin +import org.apache.spark.sql.catalyst.parser.extensions.IcebergSqlExtensionsParser._ +import org.apache.spark.sql.catalyst.plans.logical.AddPartitionField +import org.apache.spark.sql.catalyst.plans.logical.BranchOptions +import org.apache.spark.sql.catalyst.plans.logical.CreateOrReplaceBranch +import org.apache.spark.sql.catalyst.plans.logical.CreateOrReplaceTag +import org.apache.spark.sql.catalyst.plans.logical.DropBranch +import org.apache.spark.sql.catalyst.plans.logical.DropIdentifierFields +import org.apache.spark.sql.catalyst.plans.logical.DropPartitionField +import org.apache.spark.sql.catalyst.plans.logical.DropTag +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.ReplacePartitionField +import org.apache.spark.sql.catalyst.plans.logical.SetIdentifierFields +import org.apache.spark.sql.catalyst.plans.logical.SetWriteDistributionAndOrdering +import org.apache.spark.sql.catalyst.plans.logical.TagOptions +import org.apache.spark.sql.catalyst.trees.CurrentOrigin +import org.apache.spark.sql.catalyst.trees.Origin +import org.apache.spark.sql.connector.expressions +import org.apache.spark.sql.connector.expressions.ApplyTransform +import org.apache.spark.sql.connector.expressions.FieldReference +import org.apache.spark.sql.connector.expressions.IdentityTransform +import org.apache.spark.sql.connector.expressions.LiteralValue +import org.apache.spark.sql.connector.expressions.Transform +import scala.jdk.CollectionConverters._ + +class IcebergSqlExtensionsAstBuilder(delegate: ParserInterface) + extends IcebergSqlExtensionsBaseVisitor[AnyRef] { + + private def toBuffer[T](list: java.util.List[T]): scala.collection.mutable.Buffer[T] = + list.asScala + private def toSeq[T](list: java.util.List[T]): Seq[T] = toBuffer(list).toSeq + + /** + * Create an ADD PARTITION FIELD logical command. + */ + override def visitAddPartitionField(ctx: AddPartitionFieldContext): AddPartitionField = + withOrigin(ctx) { + AddPartitionField( + typedVisit[Seq[String]](ctx.multipartIdentifier), + typedVisit[Transform](ctx.transform), + Option(ctx.name).map(_.getText)) + } + + /** + * Create a DROP PARTITION FIELD logical command. + */ + override def visitDropPartitionField(ctx: DropPartitionFieldContext): DropPartitionField = + withOrigin(ctx) { + DropPartitionField( + typedVisit[Seq[String]](ctx.multipartIdentifier), + typedVisit[Transform](ctx.transform)) + } + + /** + * Create a CREATE OR REPLACE BRANCH logical command. + */ + override def visitCreateOrReplaceBranch( + ctx: CreateOrReplaceBranchContext): CreateOrReplaceBranch = withOrigin(ctx) { + val createOrReplaceBranchClause = ctx.createReplaceBranchClause() + + val branchName = createOrReplaceBranchClause.identifier() + val branchOptionsContext = Option(createOrReplaceBranchClause.branchOptions()) + val snapshotId = branchOptionsContext + .flatMap(branchOptions => Option(branchOptions.snapshotId())) + .map(_.getText.toLong) + val snapshotRetention = + branchOptionsContext.flatMap(branchOptions => Option(branchOptions.snapshotRetention())) + val minSnapshotsToKeep = snapshotRetention + .flatMap(retention => Option(retention.minSnapshotsToKeep())) + .map(minSnapshots => minSnapshots.number().getText.toLong) + val maxSnapshotAgeMs = snapshotRetention + .flatMap(retention => Option(retention.maxSnapshotAge())) + .map(retention => + TimeUnit + .valueOf(retention.timeUnit().getText.toUpperCase(Locale.ENGLISH)) + .toMillis(retention.number().getText.toLong)) + val branchRetention = + branchOptionsContext.flatMap(branchOptions => Option(branchOptions.refRetain())) + val branchRefAgeMs = branchRetention.map(retain => + TimeUnit + .valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)) + .toMillis(retain.number().getText.toLong)) + val create = createOrReplaceBranchClause.CREATE() != null + val replace = ctx.createReplaceBranchClause().REPLACE() != null + val ifNotExists = createOrReplaceBranchClause.EXISTS() != null + + val branchOptions = + BranchOptions(snapshotId, minSnapshotsToKeep, maxSnapshotAgeMs, branchRefAgeMs) + + CreateOrReplaceBranch( + typedVisit[Seq[String]](ctx.multipartIdentifier), + branchName.getText, + branchOptions, + create, + replace, + ifNotExists) + } + + /** + * Create an CREATE OR REPLACE TAG logical command. + */ + override def visitCreateOrReplaceTag(ctx: CreateOrReplaceTagContext): CreateOrReplaceTag = + withOrigin(ctx) { + val createTagClause = ctx.createReplaceTagClause() + + val tagName = createTagClause.identifier().getText + + val tagOptionsContext = Option(createTagClause.tagOptions()) + val snapshotId = tagOptionsContext + .flatMap(tagOptions => Option(tagOptions.snapshotId())) + .map(_.getText.toLong) + val tagRetain = tagOptionsContext.flatMap(tagOptions => Option(tagOptions.refRetain())) + val tagRefAgeMs = tagRetain.map(retain => + TimeUnit + .valueOf(retain.timeUnit().getText.toUpperCase(Locale.ENGLISH)) + .toMillis(retain.number().getText.toLong)) + val tagOptions = TagOptions(snapshotId, tagRefAgeMs) + + val create = createTagClause.CREATE() != null + val replace = createTagClause.REPLACE() != null + val ifNotExists = createTagClause.EXISTS() != null + + CreateOrReplaceTag( + typedVisit[Seq[String]](ctx.multipartIdentifier), + tagName, + tagOptions, + create, + replace, + ifNotExists) + } + + /** + * Create an DROP BRANCH logical command. + */ + override def visitDropBranch(ctx: DropBranchContext): DropBranch = withOrigin(ctx) { + DropBranch( + typedVisit[Seq[String]](ctx.multipartIdentifier), + ctx.identifier().getText, + ctx.EXISTS() != null) + } + + /** + * Create an DROP TAG logical command. + */ + override def visitDropTag(ctx: DropTagContext): DropTag = withOrigin(ctx) { + DropTag( + typedVisit[Seq[String]](ctx.multipartIdentifier), + ctx.identifier().getText, + ctx.EXISTS() != null) + } + + /** + * Create an REPLACE PARTITION FIELD logical command. + */ + override def visitReplacePartitionField( + ctx: ReplacePartitionFieldContext): ReplacePartitionField = withOrigin(ctx) { + ReplacePartitionField( + typedVisit[Seq[String]](ctx.multipartIdentifier), + typedVisit[Transform](ctx.transform(0)), + typedVisit[Transform](ctx.transform(1)), + Option(ctx.name).map(_.getText)) + } + + /** + * Create an SET IDENTIFIER FIELDS logical command. + */ + override def visitSetIdentifierFields(ctx: SetIdentifierFieldsContext): SetIdentifierFields = + withOrigin(ctx) { + SetIdentifierFields( + typedVisit[Seq[String]](ctx.multipartIdentifier), + toSeq(ctx.fieldList.fields).map(_.getText)) + } + + /** + * Create an DROP IDENTIFIER FIELDS logical command. + */ + override def visitDropIdentifierFields(ctx: DropIdentifierFieldsContext): DropIdentifierFields = + withOrigin(ctx) { + DropIdentifierFields( + typedVisit[Seq[String]](ctx.multipartIdentifier), + toSeq(ctx.fieldList.fields).map(_.getText)) + } + + /** + * Create a [[SetWriteDistributionAndOrdering]] for changing the write distribution and ordering. + */ + override def visitSetWriteDistributionAndOrdering( + ctx: SetWriteDistributionAndOrderingContext): SetWriteDistributionAndOrdering = { + + val tableName = typedVisit[Seq[String]](ctx.multipartIdentifier) + + val (distributionSpec, orderingSpec) = toDistributionAndOrderingSpec(ctx.writeSpec) + + if (distributionSpec == null && orderingSpec == null) { + throw new IcebergAnalysisException( + "ALTER TABLE has no changes: missing both distribution and ordering clauses") + } + + val distributionMode = if (distributionSpec != null) { + Some(DistributionMode.HASH) + } else if (orderingSpec.UNORDERED != null) { + Some(DistributionMode.NONE) + } else if (orderingSpec.LOCALLY() != null) { + None + } else { + Some(DistributionMode.RANGE) + } + + val ordering = if (orderingSpec != null && orderingSpec.order != null) { + toSeq(orderingSpec.order.fields).map(typedVisit[(Term, SortDirection, NullOrder)]) + } else { + Seq.empty + } + + SetWriteDistributionAndOrdering(tableName, distributionMode, ordering) + } + + private def toDistributionAndOrderingSpec( + writeSpec: WriteSpecContext): (WriteDistributionSpecContext, WriteOrderingSpecContext) = { + + if (writeSpec.writeDistributionSpec.size > 1) { + throw new IcebergAnalysisException("ALTER TABLE contains multiple distribution clauses") + } + + if (writeSpec.writeOrderingSpec.size > 1) { + throw new IcebergAnalysisException("ALTER TABLE contains multiple ordering clauses") + } + + val distributionSpec = toBuffer(writeSpec.writeDistributionSpec).headOption.orNull + val orderingSpec = toBuffer(writeSpec.writeOrderingSpec).headOption.orNull + + (distributionSpec, orderingSpec) + } + + /** + * Create an order field. + */ + override def visitOrderField(ctx: OrderFieldContext): (Term, SortDirection, NullOrder) = { + val term = Spark3Util.toIcebergTerm(typedVisit[Transform](ctx.transform)) + val direction = Option(ctx.ASC) + .map(_ => SortDirection.ASC) + .orElse(Option(ctx.DESC).map(_ => SortDirection.DESC)) + .getOrElse(SortDirection.ASC) + val nullOrder = Option(ctx.FIRST) + .map(_ => NullOrder.NULLS_FIRST) + .orElse(Option(ctx.LAST).map(_ => NullOrder.NULLS_LAST)) + .getOrElse( + if (direction == SortDirection.ASC) NullOrder.NULLS_FIRST else NullOrder.NULLS_LAST) + (term, direction, nullOrder) + } + + /** + * Create an IdentityTransform for a column reference. + */ + override def visitIdentityTransform(ctx: IdentityTransformContext): Transform = withOrigin(ctx) { + IdentityTransform(FieldReference(typedVisit[Seq[String]](ctx.multipartIdentifier()))) + } + + /** + * Create a named Transform from argument expressions. + */ + override def visitApplyTransform(ctx: ApplyTransformContext): Transform = withOrigin(ctx) { + val args = toSeq(ctx.arguments).map(typedVisit[expressions.Expression]) + ApplyTransform(ctx.transformName.getText, args) + } + + /** + * Create a transform argument from a column reference or a constant. + */ + override def visitTransformArgument(ctx: TransformArgumentContext): expressions.Expression = + withOrigin(ctx) { + val reference = Option(ctx.multipartIdentifier()) + .map(typedVisit[Seq[String]]) + .map(FieldReference(_)) + val literal = Option(ctx.constant) + .map(visitConstant) + .map(lit => LiteralValue(lit.value, lit.dataType)) + reference + .orElse(literal) + .getOrElse(throw new IcebergParseException(s"Invalid transform argument", ctx)) + } + + /** + * Return a multi-part identifier as Seq[String]. + */ + override def visitMultipartIdentifier(ctx: MultipartIdentifierContext): Seq[String] = + withOrigin(ctx) { + toSeq(ctx.parts).map(_.getText) + } + + override def visitSingleOrder(ctx: SingleOrderContext): Seq[(Term, SortDirection, NullOrder)] = + withOrigin(ctx) { + toSeq(ctx.order.fields).map(typedVisit[(Term, SortDirection, NullOrder)]) + } + + override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan = withOrigin(ctx) { + visit(ctx.statement).asInstanceOf[LogicalPlan] + } + + def visitConstant(ctx: ConstantContext): Literal = { + delegate.parseExpression(ctx.getText).asInstanceOf[Literal] + } + + override def visitExpression(ctx: ExpressionContext): Expression = { + // reconstruct the SQL string and parse it using the main Spark parser + // while we can avoid the logic to build Spark expressions, we still have to parse them + // we cannot call ctx.getText directly since it will not render spaces correctly + // that's why we need to recurse down the tree in reconstructSqlString + val sqlString = reconstructSqlString(ctx) + delegate.parseExpression(sqlString) + } + + private def reconstructSqlString(ctx: ParserRuleContext): String = { + toBuffer(ctx.children) + .map { + case c: ParserRuleContext => reconstructSqlString(c) + case t: TerminalNode => t.getText + } + .mkString(" ") + } + + private def typedVisit[T](ctx: ParseTree): T = { + ctx.accept(this).asInstanceOf[T] + } +} + +/* Partially copied from Apache Spark's Parser to avoid dependency on Spark Internals */ +object IcebergParserUtils { + + private[sql] def withOrigin[T](ctx: ParserRuleContext)(f: => T): T = { + val current = CurrentOrigin.get + CurrentOrigin.set(position(ctx.getStart)) + try { + f + } finally { + CurrentOrigin.set(current) + } + } + + private[sql] def position(token: Token): Origin = { + val opt = Option(token) + Origin(opt.map(_.getLine), opt.map(_.getCharPositionInLine)) + } + + /** Get the command which created the token. */ + private[sql] def command(ctx: ParserRuleContext): String = { + val stream = ctx.getStart.getInputStream + stream.getText(Interval.of(0, stream.size() - 1)) + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala new file mode 100644 index 000000000000..0a830dbd4f6a --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/AddPartitionField.scala @@ -0,0 +1,33 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.expressions.Transform + +case class AddPartitionField(table: Seq[String], transform: Transform, name: Option[String]) + extends LeafCommand { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"AddPartitionField ${table.quoted} ${name.map(n => s"$n=").getOrElse("")}${transform.describe}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala new file mode 100644 index 000000000000..15b908300213 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BranchOptions.scala @@ -0,0 +1,25 @@ +/* + * 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.plans.logical + +case class BranchOptions( + snapshotId: Option[Long], + numSnapshots: Option[Long], + snapshotRetain: Option[Long], + snapshotRefRetain: Option[Long]) diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala new file mode 100644 index 000000000000..6900f6e8cc50 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceBranch.scala @@ -0,0 +1,39 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Attribute + +case class CreateOrReplaceBranch( + table: Seq[String], + branch: String, + branchOptions: BranchOptions, + create: Boolean, + replace: Boolean, + ifNotExists: Boolean) + extends LeafCommand { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"CreateOrReplaceBranch branch: ${branch} for table: ${table.quoted}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala new file mode 100644 index 000000000000..957c68e7a540 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CreateOrReplaceTag.scala @@ -0,0 +1,39 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Attribute + +case class CreateOrReplaceTag( + table: Seq[String], + tag: String, + tagOptions: TagOptions, + create: Boolean, + replace: Boolean, + ifNotExists: Boolean) + extends LeafCommand { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"CreateOrReplaceTag tag: ${tag} for table: ${table.quoted}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala new file mode 100644 index 000000000000..ed4f1f512b85 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropBranch.scala @@ -0,0 +1,32 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Attribute + +case class DropBranch(table: Seq[String], branch: String, ifExists: Boolean) extends LeafCommand { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"DropBranch branch: ${branch} for table: ${table.quoted}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala new file mode 100644 index 000000000000..1a91806280b3 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropIdentifierFields.scala @@ -0,0 +1,31 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Attribute + +case class DropIdentifierFields(table: Seq[String], fields: Seq[String]) extends LeafCommand { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"DropIdentifierFields ${table.quoted} (${fields.quoted})" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala new file mode 100644 index 000000000000..ec952c8c7118 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropPartitionField.scala @@ -0,0 +1,32 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.expressions.Transform + +case class DropPartitionField(table: Seq[String], transform: Transform) extends LeafCommand { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"DropPartitionField ${table.quoted} ${transform.describe}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala new file mode 100644 index 000000000000..da69ca0383a1 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/DropTag.scala @@ -0,0 +1,32 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Attribute + +case class DropTag(table: Seq[String], tag: String, ifExists: Boolean) extends LeafCommand { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"DropTag tag: ${tag} for table: ${table.quoted}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala new file mode 100644 index 000000000000..c2525369e7c7 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ReplacePartitionField.scala @@ -0,0 +1,38 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.expressions.Transform + +case class ReplacePartitionField( + table: Seq[String], + transformFrom: Transform, + transformTo: Transform, + name: Option[String]) + extends LeafCommand { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"ReplacePartitionField ${table.quoted} ${transformFrom.describe} " + + s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala new file mode 100644 index 000000000000..8cd2c0ddad05 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetIdentifierFields.scala @@ -0,0 +1,31 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.expressions.Attribute + +case class SetIdentifierFields(table: Seq[String], fields: Seq[String]) extends LeafCommand { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + s"SetIdentifierFields ${table.quoted} (${fields.quoted})" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala new file mode 100644 index 000000000000..6afe1478d747 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TagOptions.scala @@ -0,0 +1,21 @@ +/* + * 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.plans.logical + +case class TagOptions(snapshotId: Option[Long], snapshotRefRetain: Option[Long]) diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala new file mode 100644 index 000000000000..84a00a4a9a88 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/CreateIcebergView.scala @@ -0,0 +1,53 @@ +/* + * 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.plans.logical.views + +import org.apache.spark.sql.catalyst.analysis.AnalysisContext +import org.apache.spark.sql.catalyst.plans.logical.AnalysisOnlyCommand +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +// Align Iceberg's CreateIcebergView with Spark’s CreateViewCommand by extending AnalysisOnlyCommand. +// The command’s children are analyzed then hidden, so the optimizer/planner won’t traverse the view body. +case class CreateIcebergView( + child: LogicalPlan, + queryText: String, + query: LogicalPlan, + columnAliases: Seq[String], + columnComments: Seq[Option[String]], + queryColumnNames: Seq[String] = Seq.empty, + comment: Option[String], + properties: Map[String, String], + allowExisting: Boolean, + replace: Boolean, + rewritten: Boolean = false, + isAnalyzed: Boolean = false) + extends AnalysisOnlyCommand { + + override def childrenToAnalyze: Seq[LogicalPlan] = child :: query :: Nil + + override def markAsAnalyzed(analysisContext: AnalysisContext): LogicalPlan = { + copy(isAnalyzed = true) + } + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[LogicalPlan]): LogicalPlan = { + assert(!isAnalyzed) + copy(child = newChildren.head, query = newChildren.last) + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala new file mode 100644 index 000000000000..092b6b33fb0c --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/DropIcebergView.scala @@ -0,0 +1,27 @@ +/* + * 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.plans.logical.views + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.UnaryCommand + +case class DropIcebergView(child: LogicalPlan, ifExists: Boolean) extends UnaryCommand { + override protected def withNewChildInternal(newChild: LogicalPlan): DropIcebergView = + copy(child = newChild) +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala new file mode 100644 index 000000000000..4d384e857703 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ResolvedV2View.scala @@ -0,0 +1,29 @@ +/* + * 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.plans.logical.views + +import org.apache.spark.sql.catalyst.analysis.LeafNodeWithoutStats +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.ViewCatalog + +case class ResolvedV2View(catalog: ViewCatalog, identifier: Identifier) + extends LeafNodeWithoutStats { + override def output: Seq[Attribute] = Nil +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala new file mode 100644 index 000000000000..cbfe23d94cbe --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/views/ShowIcebergViews.scala @@ -0,0 +1,35 @@ +/* + * 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.plans.logical.views + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.ShowViews +import org.apache.spark.sql.catalyst.plans.logical.UnaryCommand + +case class ShowIcebergViews( + namespace: LogicalPlan, + pattern: Option[String], + override val output: Seq[Attribute] = ShowViews.getOutputAttrs) + extends UnaryCommand { + override def child: LogicalPlan = namespace + + override protected def withNewChildInternal(newChild: LogicalPlan): ShowIcebergViews = + copy(namespace = newChild) +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala new file mode 100644 index 000000000000..e28dcfb194b6 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AddPartitionFieldExec.scala @@ -0,0 +1,58 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.iceberg.spark.Spark3Util +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCatalog +import org.apache.spark.sql.connector.expressions.Transform + +case class AddPartitionFieldExec( + catalog: TableCatalog, + ident: Identifier, + transform: Transform, + name: Option[String]) + extends LeafV2CommandExec { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable => + iceberg.table + .updateSpec() + .addField(name.orNull, Spark3Util.toIcebergTerm(transform)) + .commit() + + case table => + throw new UnsupportedOperationException( + s"Cannot add partition field to non-Iceberg table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"AddPartitionField ${catalog.name}.${ident.quoted} ${name.map(n => s"$n=").getOrElse("")}${transform.describe}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala new file mode 100644 index 000000000000..d6630e51ff5a --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewSetPropertiesExec.scala @@ -0,0 +1,48 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.ViewCatalog +import org.apache.spark.sql.connector.catalog.ViewChange + +case class AlterV2ViewSetPropertiesExec( + catalog: ViewCatalog, + ident: Identifier, + properties: Map[String, String]) + extends LeafV2CommandExec { + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + val changes = properties.map { case (property, value) => + ViewChange.setProperty(property, value) + }.toSeq + + catalog.alterView(ident, changes: _*) + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"AlterV2ViewSetProperties: ${ident}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala new file mode 100644 index 000000000000..aa57842f58b5 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterV2ViewUnsetPropertiesExec.scala @@ -0,0 +1,53 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.IcebergAnalysisException +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.ViewCatalog +import org.apache.spark.sql.connector.catalog.ViewChange + +case class AlterV2ViewUnsetPropertiesExec( + catalog: ViewCatalog, + ident: Identifier, + propertyKeys: Seq[String], + ifExists: Boolean) + extends LeafV2CommandExec { + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + if (!ifExists) { + propertyKeys.filterNot(catalog.loadView(ident).properties.containsKey).foreach { property => + throw new IcebergAnalysisException(s"Cannot remove property that is not set: '$property'") + } + } + + val changes = propertyKeys.map(ViewChange.removeProperty) + catalog.alterView(ident, changes: _*) + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"AlterV2ViewUnsetProperties: ${ident}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala new file mode 100644 index 000000000000..baf985f53a22 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceBranchExec.scala @@ -0,0 +1,103 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.BranchOptions +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCatalog + +case class CreateOrReplaceBranchExec( + catalog: TableCatalog, + ident: Identifier, + branch: String, + branchOptions: BranchOptions, + create: Boolean, + replace: Boolean, + ifNotExists: Boolean) + extends LeafV2CommandExec { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable => + val snapshotId: java.lang.Long = branchOptions.snapshotId + .orElse(Option(iceberg.table.currentSnapshot()).map(_.snapshotId())) + .map(java.lang.Long.valueOf) + .orNull + + val manageSnapshots = iceberg.table().manageSnapshots() + val refExists = null != iceberg.table().refs().get(branch) + + def safeCreateBranch(): Unit = { + if (snapshotId == null) { + manageSnapshots.createBranch(branch) + } else { + manageSnapshots.createBranch(branch, snapshotId) + } + } + + if (create && replace && !refExists) { + safeCreateBranch() + } else if (replace) { + Preconditions.checkArgument( + snapshotId != null, + "Cannot complete replace branch operation on %s, main has no snapshot", + ident) + manageSnapshots.replaceBranch(branch, snapshotId) + } else { + if (refExists && ifNotExists) { + return Nil + } + + safeCreateBranch() + } + + if (branchOptions.numSnapshots.nonEmpty) { + manageSnapshots.setMinSnapshotsToKeep(branch, branchOptions.numSnapshots.get.toInt) + } + + if (branchOptions.snapshotRetain.nonEmpty) { + manageSnapshots.setMaxSnapshotAgeMs(branch, branchOptions.snapshotRetain.get) + } + + if (branchOptions.snapshotRefRetain.nonEmpty) { + manageSnapshots.setMaxRefAgeMs(branch, branchOptions.snapshotRefRetain.get) + } + + manageSnapshots.commit() + + case table => + throw new UnsupportedOperationException( + s"Cannot create or replace branch on non-Iceberg table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"CreateOrReplace branch: $branch for table: ${ident.quoted}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala new file mode 100644 index 000000000000..e486892614cb --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateOrReplaceTagExec.scala @@ -0,0 +1,86 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.iceberg.relocated.com.google.common.base.Preconditions +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.TagOptions +import org.apache.spark.sql.connector.catalog._ + +case class CreateOrReplaceTagExec( + catalog: TableCatalog, + ident: Identifier, + tag: String, + tagOptions: TagOptions, + create: Boolean, + replace: Boolean, + ifNotExists: Boolean) + extends LeafV2CommandExec { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable => + val snapshotId: java.lang.Long = tagOptions.snapshotId + .orElse(Option(iceberg.table.currentSnapshot()).map(_.snapshotId())) + .map(java.lang.Long.valueOf) + .orNull + + Preconditions.checkArgument( + snapshotId != null, + "Cannot complete create or replace tag operation on %s, main has no snapshot", + ident) + + val manageSnapshot = iceberg.table.manageSnapshots() + val refExists = null != iceberg.table().refs().get(tag) + + if (create && replace && !refExists) { + manageSnapshot.createTag(tag, snapshotId) + } else if (replace) { + manageSnapshot.replaceTag(tag, snapshotId) + } else { + if (refExists && ifNotExists) { + return Nil + } + + manageSnapshot.createTag(tag, snapshotId) + } + + if (tagOptions.snapshotRefRetain.nonEmpty) { + manageSnapshot.setMaxRefAgeMs(tag, tagOptions.snapshotRefRetain.get) + } + + manageSnapshot.commit() + + case table => + throw new UnsupportedOperationException(s"Cannot create tag to non-Iceberg table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"Create tag: $tag for table: ${ident.quoted}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala new file mode 100644 index 000000000000..04f2b2af731d --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala @@ -0,0 +1,128 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.iceberg.spark.SupportsReplaceView +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.NoSuchViewException +import org.apache.spark.sql.catalyst.analysis.ViewAlreadyExistsException +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.ViewCatalog +import org.apache.spark.sql.connector.catalog.ViewInfo +import org.apache.spark.sql.types.StructType +import scala.jdk.CollectionConverters._ + +case class CreateV2ViewExec( + catalog: ViewCatalog, + ident: Identifier, + queryText: String, + viewSchema: StructType, + columnAliases: Seq[String], + columnComments: Seq[Option[String]], + queryColumnNames: Seq[String], + comment: Option[String], + properties: Map[String, String], + allowExisting: Boolean, + replace: Boolean) + extends LeafV2CommandExec { + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + val currentCatalogName = session.sessionState.catalogManager.currentCatalog.name + val currentCatalog = + if (!catalog.name().equals(currentCatalogName)) currentCatalogName else null + val currentNamespace = session.sessionState.catalogManager.currentNamespace + + val engineVersion = "Spark " + org.apache.spark.SPARK_VERSION + val newProperties = properties ++ + comment.map(ViewCatalog.PROP_COMMENT -> _) ++ + Map( + ViewCatalog.PROP_CREATE_ENGINE_VERSION -> engineVersion, + ViewCatalog.PROP_ENGINE_VERSION -> engineVersion) + + if (replace) { + // CREATE OR REPLACE VIEW + catalog match { + case c: SupportsReplaceView => + try { + replaceView(c, currentCatalog, currentNamespace, newProperties) + } catch { + // view might have been concurrently dropped during replace + case _: NoSuchViewException => + replaceView(c, currentCatalog, currentNamespace, newProperties) + } + case _ => + if (catalog.viewExists(ident)) { + catalog.dropView(ident) + } + + createView(currentCatalog, currentNamespace, newProperties) + } + } else { + try { + // CREATE VIEW [IF NOT EXISTS] + createView(currentCatalog, currentNamespace, newProperties) + } catch { + case _: ViewAlreadyExistsException if allowExisting => // Ignore + } + } + + Nil + } + + private def replaceView( + supportsReplaceView: SupportsReplaceView, + currentCatalog: String, + currentNamespace: Array[String], + newProperties: Map[String, String]) = { + supportsReplaceView.replaceView( + ident, + queryText, + currentCatalog, + currentNamespace, + viewSchema, + queryColumnNames.toArray, + columnAliases.toArray, + columnComments.map(c => c.orNull).toArray, + newProperties.asJava) + } + + private def createView( + currentCatalog: String, + currentNamespace: Array[String], + newProperties: Map[String, String]) = { + val viewInfo: ViewInfo = new ViewInfo( + ident, + queryText, + currentCatalog, + currentNamespace, + viewSchema, + queryColumnNames.toArray, + columnAliases.toArray, + columnComments.map(c => c.orNull).toArray, + newProperties.asJava) + catalog.createView(viewInfo) + } + + override def simpleString(maxFields: Int): String = { + s"CreateV2ViewExec: ${ident}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala new file mode 100644 index 000000000000..106734d2078f --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeV2ViewExec.scala @@ -0,0 +1,79 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.util.escapeSingleQuotedString +import org.apache.spark.sql.connector.catalog.View +import org.apache.spark.sql.connector.catalog.ViewCatalog +import org.apache.spark.sql.execution.LeafExecNode +import scala.jdk.CollectionConverters._ + +case class DescribeV2ViewExec(output: Seq[Attribute], view: View, isExtended: Boolean) + extends V2CommandExec + with LeafExecNode { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override protected def run(): Seq[InternalRow] = { + if (isExtended) { + (describeSchema :+ emptyRow) ++ describeExtended + } else { + describeSchema + } + } + + private def describeSchema: Seq[InternalRow] = + view.schema().map { column => + toCatalystRow(column.name, column.dataType.simpleString, column.getComment().getOrElse("")) + } + + private def emptyRow: InternalRow = toCatalystRow("", "", "") + + private def describeExtended: Seq[InternalRow] = { + val outputColumns = view.queryColumnNames.mkString("[", ", ", "]") + val properties: Map[String, String] = + view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala + val viewCatalogAndNamespace: Seq[String] = view.name.split("\\.").take(2).toIndexedSeq + val viewProperties = properties.toSeq + .sortBy(_._1) + .map { case (key, value) => + s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" + } + .mkString("[", ", ", "]") + + // omitting view text here because it is shown as + // part of SHOW CREATE TABLE and can result in weird formatting in the DESCRIBE output + toCatalystRow("# Detailed View Information", "", "") :: + toCatalystRow("Comment", view.properties.getOrDefault(ViewCatalog.PROP_COMMENT, ""), "") :: + toCatalystRow("View Catalog and Namespace", viewCatalogAndNamespace.quoted, "") :: + toCatalystRow("View Query Output Columns", outputColumns, "") :: + toCatalystRow("View Properties", viewProperties, "") :: + toCatalystRow( + "Created By", + view.properties.getOrDefault(ViewCatalog.PROP_CREATE_ENGINE_VERSION, ""), + "") :: + Nil + } + + override def simpleString(maxFields: Int): String = { + s"DescribeV2ViewExec" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala new file mode 100644 index 000000000000..e7d9c7b70d82 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropBranchExec.scala @@ -0,0 +1,56 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCatalog + +case class DropBranchExec( + catalog: TableCatalog, + ident: Identifier, + branch: String, + ifExists: Boolean) + extends LeafV2CommandExec { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable => + val ref = iceberg.table().refs().get(branch) + if (ref != null || !ifExists) { + iceberg.table().manageSnapshots().removeBranch(branch).commit() + } + + case table => + throw new UnsupportedOperationException(s"Cannot drop branch on non-Iceberg table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"DropBranch branch: ${branch} for table: ${ident.quoted}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala new file mode 100644 index 000000000000..87b18594d573 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropIdentifierFieldsExec.scala @@ -0,0 +1,68 @@ +/* + * 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.execution.datasources.v2 + +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.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCatalog + +case class DropIdentifierFieldsExec(catalog: TableCatalog, ident: Identifier, fields: Seq[String]) + extends LeafV2CommandExec { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable => + val schema = iceberg.table.schema + val identifierFieldNames = Sets.newHashSet(schema.identifierFieldNames) + + for (name <- fields) { + Preconditions.checkArgument( + schema.findField(name) != null, + "Cannot complete drop identifier fields operation: field %s not found", + name) + Preconditions.checkArgument( + identifierFieldNames.contains(name), + "Cannot complete drop identifier fields operation: %s is not an identifier field", + name) + identifierFieldNames.remove(name) + } + + iceberg.table + .updateSchema() + .setIdentifierFields(identifierFieldNames) + .commit(); + case table => + throw new UnsupportedOperationException( + s"Cannot drop identifier fields in non-Iceberg table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"DropIdentifierFields ${catalog.name}.${ident.quoted} (${fields.quoted})"; + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala new file mode 100644 index 000000000000..db43263e0e66 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropPartitionFieldExec.scala @@ -0,0 +1,68 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.iceberg.spark.Spark3Util +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCatalog +import org.apache.spark.sql.connector.expressions.FieldReference +import org.apache.spark.sql.connector.expressions.IdentityTransform +import org.apache.spark.sql.connector.expressions.Transform + +case class DropPartitionFieldExec(catalog: TableCatalog, ident: Identifier, transform: Transform) + extends LeafV2CommandExec { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable => + val schema = iceberg.table.schema + transform match { + case IdentityTransform(FieldReference(parts)) + if parts.size == 1 && schema.findField(parts.head) == null => + // the name is not present in the Iceberg schema, so it must be a partition field name, not a column name + iceberg.table + .updateSpec() + .removeField(parts.head) + .commit() + + case _ => + iceberg.table + .updateSpec() + .removeField(Spark3Util.toIcebergTerm(transform)) + .commit() + } + + case table => + throw new UnsupportedOperationException( + s"Cannot drop partition field in non-Iceberg table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"DropPartitionField ${catalog.name}.${ident.quoted} ${transform.describe}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala new file mode 100644 index 000000000000..79b4a1525591 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTagExec.scala @@ -0,0 +1,52 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCatalog + +case class DropTagExec(catalog: TableCatalog, ident: Identifier, tag: String, ifExists: Boolean) + extends LeafV2CommandExec { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable => + val ref = iceberg.table().refs().get(tag) + if (ref != null || !ifExists) { + iceberg.table().manageSnapshots().removeTag(tag).commit() + } + + case table => + throw new UnsupportedOperationException(s"Cannot drop tag on non-Iceberg table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"DropTag tag: ${tag} for table: ${ident.quoted}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala new file mode 100644 index 000000000000..6dd1188b78e8 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropV2ViewExec.scala @@ -0,0 +1,44 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.NoSuchViewException +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.ViewCatalog + +case class DropV2ViewExec(catalog: ViewCatalog, ident: Identifier, ifExists: Boolean) + extends LeafV2CommandExec { + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + val dropped = catalog.dropView(ident) + if (!dropped && !ifExists) { + throw new NoSuchViewException(ident) + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"DropV2View: ${ident}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala new file mode 100644 index 000000000000..da540f5891b7 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ExtendedDataSourceV2Strategy.scala @@ -0,0 +1,192 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.iceberg.spark.Spark3Util +import org.apache.iceberg.spark.SparkCatalog +import org.apache.iceberg.spark.SparkSessionCatalog +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.IcebergAnalysisException +import org.apache.spark.sql.catalyst.analysis.ResolvedIdentifier +import org.apache.spark.sql.catalyst.analysis.ResolvedNamespace +import org.apache.spark.sql.catalyst.expressions.PredicateHelper +import org.apache.spark.sql.catalyst.plans.logical.AddPartitionField +import org.apache.spark.sql.catalyst.plans.logical.CreateOrReplaceBranch +import org.apache.spark.sql.catalyst.plans.logical.CreateOrReplaceTag +import org.apache.spark.sql.catalyst.plans.logical.DescribeRelation +import org.apache.spark.sql.catalyst.plans.logical.DropBranch +import org.apache.spark.sql.catalyst.plans.logical.DropIdentifierFields +import org.apache.spark.sql.catalyst.plans.logical.DropPartitionField +import org.apache.spark.sql.catalyst.plans.logical.DropTag +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.OrderAwareCoalesce +import org.apache.spark.sql.catalyst.plans.logical.RenameTable +import org.apache.spark.sql.catalyst.plans.logical.ReplacePartitionField +import org.apache.spark.sql.catalyst.plans.logical.SetIdentifierFields +import org.apache.spark.sql.catalyst.plans.logical.SetViewProperties +import org.apache.spark.sql.catalyst.plans.logical.SetWriteDistributionAndOrdering +import org.apache.spark.sql.catalyst.plans.logical.ShowCreateTable +import org.apache.spark.sql.catalyst.plans.logical.ShowTableProperties +import org.apache.spark.sql.catalyst.plans.logical.UnsetViewProperties +import org.apache.spark.sql.catalyst.plans.logical.views.CreateIcebergView +import org.apache.spark.sql.catalyst.plans.logical.views.DropIcebergView +import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View +import org.apache.spark.sql.catalyst.plans.logical.views.ShowIcebergViews +import org.apache.spark.sql.classic.Strategy +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCatalog +import org.apache.spark.sql.connector.catalog.ViewCatalog +import org.apache.spark.sql.execution.OrderAwareCoalesceExec +import org.apache.spark.sql.execution.SparkPlan +import scala.jdk.CollectionConverters._ + +case class ExtendedDataSourceV2Strategy(spark: SparkSession) extends Strategy with PredicateHelper { + + override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case AddPartitionField(IcebergCatalogAndIdentifier(catalog, ident), transform, name) => + AddPartitionFieldExec(catalog, ident, transform, name) :: Nil + + case CreateOrReplaceBranch( + IcebergCatalogAndIdentifier(catalog, ident), + branch, + branchOptions, + create, + replace, + ifNotExists) => + CreateOrReplaceBranchExec( + catalog, + ident, + branch, + branchOptions, + create, + replace, + ifNotExists) :: Nil + + case CreateOrReplaceTag( + IcebergCatalogAndIdentifier(catalog, ident), + tag, + tagOptions, + create, + replace, + ifNotExists) => + CreateOrReplaceTagExec(catalog, ident, tag, tagOptions, create, replace, ifNotExists) :: Nil + + case DropBranch(IcebergCatalogAndIdentifier(catalog, ident), branch, ifExists) => + DropBranchExec(catalog, ident, branch, ifExists) :: Nil + + case DropTag(IcebergCatalogAndIdentifier(catalog, ident), tag, ifExists) => + DropTagExec(catalog, ident, tag, ifExists) :: Nil + + case DropPartitionField(IcebergCatalogAndIdentifier(catalog, ident), transform) => + DropPartitionFieldExec(catalog, ident, transform) :: Nil + + case ReplacePartitionField( + IcebergCatalogAndIdentifier(catalog, ident), + transformFrom, + transformTo, + name) => + ReplacePartitionFieldExec(catalog, ident, transformFrom, transformTo, name) :: Nil + + case SetIdentifierFields(IcebergCatalogAndIdentifier(catalog, ident), fields) => + SetIdentifierFieldsExec(catalog, ident, fields) :: Nil + + case DropIdentifierFields(IcebergCatalogAndIdentifier(catalog, ident), fields) => + DropIdentifierFieldsExec(catalog, ident, fields) :: Nil + + case SetWriteDistributionAndOrdering( + IcebergCatalogAndIdentifier(catalog, ident), + distributionMode, + ordering) => + SetWriteDistributionAndOrderingExec(catalog, ident, distributionMode, ordering) :: Nil + + case OrderAwareCoalesce(numPartitions, coalescer, child) => + OrderAwareCoalesceExec(numPartitions, coalescer, planLater(child)) :: Nil + + case RenameTable(ResolvedV2View(oldCatalog: ViewCatalog, oldIdent), newName, isView @ true) => + val newIdent = Spark3Util.catalogAndIdentifier(spark, newName.toList.asJava) + if (oldCatalog.name != newIdent.catalog().name()) { + throw new IcebergAnalysisException( + s"Cannot move view between catalogs: from=${oldCatalog.name} and to=${newIdent.catalog().name()}") + } + RenameV2ViewExec(oldCatalog, oldIdent, newIdent.identifier()) :: Nil + + case DropIcebergView(ResolvedIdentifier(viewCatalog: ViewCatalog, ident), ifExists) => + DropV2ViewExec(viewCatalog, ident, ifExists) :: Nil + + case CreateIcebergView( + ResolvedIdentifier(viewCatalog: ViewCatalog, ident), + queryText, + query, + columnAliases, + columnComments, + queryColumnNames, + comment, + properties, + allowExisting, + replace, + _, + _) => + CreateV2ViewExec( + catalog = viewCatalog, + ident = ident, + queryText = queryText, + columnAliases = columnAliases, + columnComments = columnComments, + queryColumnNames = queryColumnNames, + viewSchema = query.schema, + comment = comment, + properties = properties, + allowExisting = allowExisting, + replace = replace) :: Nil + + case DescribeRelation(ResolvedV2View(catalog, ident), _, isExtended, output) => + DescribeV2ViewExec(output, catalog.loadView(ident), isExtended) :: Nil + + case ShowTableProperties(ResolvedV2View(catalog, ident), propertyKey, output) => + ShowV2ViewPropertiesExec(output, catalog.loadView(ident), propertyKey) :: Nil + + case ShowIcebergViews(ResolvedNamespace(catalog: ViewCatalog, namespace, _), pattern, output) => + ShowV2ViewsExec(output, catalog, namespace, pattern) :: Nil + + case ShowCreateTable(ResolvedV2View(catalog, ident), _, output) => + ShowCreateV2ViewExec(output, catalog.loadView(ident)) :: Nil + + case SetViewProperties(ResolvedV2View(catalog, ident), properties) => + AlterV2ViewSetPropertiesExec(catalog, ident, properties) :: Nil + + case UnsetViewProperties(ResolvedV2View(catalog, ident), propertyKeys, ifExists) => + AlterV2ViewUnsetPropertiesExec(catalog, ident, propertyKeys, ifExists) :: Nil + + case _ => Nil + } + + private object IcebergCatalogAndIdentifier { + def unapply(identifier: Seq[String]): Option[(TableCatalog, Identifier)] = { + val catalogAndIdentifier = Spark3Util.catalogAndIdentifier(spark, identifier.asJava) + catalogAndIdentifier.catalog match { + case icebergCatalog: SparkCatalog => + Some((icebergCatalog, catalogAndIdentifier.identifier)) + case icebergCatalog: SparkSessionCatalog[_] => + Some((icebergCatalog, catalogAndIdentifier.identifier)) + case _ => + None + } + } + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala new file mode 100644 index 000000000000..5dada1cab0bb --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/RenameV2ViewExec.scala @@ -0,0 +1,40 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.ViewCatalog + +case class RenameV2ViewExec(catalog: ViewCatalog, oldIdent: Identifier, newIdent: Identifier) + extends LeafV2CommandExec { + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.renameView(oldIdent, newIdent) + + Seq.empty + } + + override def simpleString(maxFields: Int): String = { + s"RenameV2View ${oldIdent} to {newIdent}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala new file mode 100644 index 000000000000..00b998c49e83 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ReplacePartitionFieldExec.scala @@ -0,0 +1,76 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.iceberg.spark.Spark3Util +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCatalog +import org.apache.spark.sql.connector.expressions.FieldReference +import org.apache.spark.sql.connector.expressions.IdentityTransform +import org.apache.spark.sql.connector.expressions.Transform + +case class ReplacePartitionFieldExec( + catalog: TableCatalog, + ident: Identifier, + transformFrom: Transform, + transformTo: Transform, + name: Option[String]) + extends LeafV2CommandExec { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable => + val schema = iceberg.table.schema + transformFrom match { + case IdentityTransform(FieldReference(parts)) + if parts.size == 1 && schema.findField(parts.head) == null => + // the name is not present in the Iceberg schema, so it must be a partition field name, not a column name + iceberg.table + .updateSpec() + .removeField(parts.head) + .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) + .commit() + + case _ => + iceberg.table + .updateSpec() + .removeField(Spark3Util.toIcebergTerm(transformFrom)) + .addField(name.orNull, Spark3Util.toIcebergTerm(transformTo)) + .commit() + } + + case table => + throw new UnsupportedOperationException( + s"Cannot replace partition field in non-Iceberg table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"ReplacePartitionField ${catalog.name}.${ident.quoted} ${transformFrom.describe} " + + s"with ${name.map(n => s"$n=").getOrElse("")}${transformTo.describe}" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala new file mode 100644 index 000000000000..50c53473ab60 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetIdentifierFieldsExec.scala @@ -0,0 +1,52 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCatalog +import scala.jdk.CollectionConverters._ + +case class SetIdentifierFieldsExec(catalog: TableCatalog, ident: Identifier, fields: Seq[String]) + extends LeafV2CommandExec { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable => + iceberg.table + .updateSchema() + .setIdentifierFields(fields.asJava) + .commit(); + case table => + throw new UnsupportedOperationException( + s"Cannot set identifier fields in non-Iceberg table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + s"SetIdentifierFields ${catalog.name}.${ident.quoted} (${fields.quoted})"; + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala new file mode 100644 index 000000000000..9a10949d5e9e --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/SetWriteDistributionAndOrderingExec.scala @@ -0,0 +1,85 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.iceberg.DistributionMode +import org.apache.iceberg.NullOrder +import org.apache.iceberg.SortDirection +import org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE +import org.apache.iceberg.expressions.Term +import org.apache.iceberg.spark.SparkUtil +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits +import org.apache.spark.sql.connector.catalog.Identifier +import org.apache.spark.sql.connector.catalog.TableCatalog + +case class SetWriteDistributionAndOrderingExec( + catalog: TableCatalog, + ident: Identifier, + distributionMode: Option[DistributionMode], + sortOrder: Seq[(Term, SortDirection, NullOrder)]) + extends LeafV2CommandExec { + + import CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override protected def run(): Seq[InternalRow] = { + catalog.loadTable(ident) match { + case iceberg: SparkTable => + val txn = iceberg.table.newTransaction() + + val orderBuilder = txn.replaceSortOrder().caseSensitive(SparkUtil.caseSensitive(session)) + sortOrder.foreach { + case (term, SortDirection.ASC, nullOrder) => + orderBuilder.asc(term, nullOrder) + case (term, SortDirection.DESC, nullOrder) => + orderBuilder.desc(term, nullOrder) + } + orderBuilder.commit() + + distributionMode.foreach { mode => + txn + .updateProperties() + .set(WRITE_DISTRIBUTION_MODE, mode.modeName()) + .commit() + } + + txn.commitTransaction() + + case table => + throw new UnsupportedOperationException( + s"Cannot set write order of non-Iceberg table: $table") + } + + Nil + } + + override def simpleString(maxFields: Int): String = { + val tableIdent = s"${catalog.name}.${ident.quoted}" + val order = sortOrder + .map { case (term, direction, nullOrder) => + s"$term $direction $nullOrder" + } + .mkString(", ") + s"SetWriteDistributionAndOrdering $tableIdent $distributionMode $order" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala new file mode 100644 index 000000000000..07ac4aeda8fb --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowCreateV2ViewExec.scala @@ -0,0 +1,78 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.util.escapeSingleQuotedString +import org.apache.spark.sql.connector.catalog.View +import org.apache.spark.sql.connector.catalog.ViewCatalog +import org.apache.spark.sql.execution.LeafExecNode +import scala.jdk.CollectionConverters._ + +case class ShowCreateV2ViewExec(output: Seq[Attribute], view: View) + extends V2CommandExec + with LeafExecNode { + + override protected def run(): Seq[InternalRow] = { + val builder = new StringBuilder + builder ++= s"CREATE VIEW ${view.name} " + showColumns(view, builder) + showComment(view, builder) + showProperties(view, builder) + builder ++= s"AS\n${view.query}\n" + + Seq(toCatalystRow(builder.toString)) + } + + private def showColumns(view: View, builder: StringBuilder): Unit = { + val columns = concatByMultiLines( + view + .schema() + .fields + .map(x => s"${x.name}${x.getComment().map(c => s" COMMENT '$c'").getOrElse("")}")) + builder ++= columns + } + + private def showComment(view: View, builder: StringBuilder): Unit = { + Option(view.properties.get(ViewCatalog.PROP_COMMENT)) + .map("COMMENT '" + escapeSingleQuotedString(_) + "'\n") + .foreach(builder.append) + } + + private def showProperties(view: View, builder: StringBuilder): Unit = { + val showProps = view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala + if (showProps.nonEmpty) { + val props = conf.redactOptions(showProps).toSeq.sortBy(_._1).map { case (key, value) => + s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" + } + + builder ++= "TBLPROPERTIES " + builder ++= concatByMultiLines(props) + } + } + + private def concatByMultiLines(iter: Iterable[String]): String = { + iter.mkString("(\n ", ",\n ", ")\n") + } + + override def simpleString(maxFields: Int): String = { + s"ShowCreateV2ViewExec" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala new file mode 100644 index 000000000000..ace43eb6c07b --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewPropertiesExec.scala @@ -0,0 +1,51 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.View +import org.apache.spark.sql.connector.catalog.ViewCatalog +import org.apache.spark.sql.execution.LeafExecNode +import scala.jdk.CollectionConverters._ + +case class ShowV2ViewPropertiesExec(output: Seq[Attribute], view: View, propertyKey: Option[String]) + extends V2CommandExec + with LeafExecNode { + + override protected def run(): Seq[InternalRow] = { + propertyKey match { + case Some(p) => + val propValue = properties.getOrElse(p, s"View ${view.name()} does not have property: $p") + Seq(toCatalystRow(p, propValue)) + case None => + properties.map { case (k, v) => + toCatalystRow(k, v) + }.toSeq + } + } + + private def properties = { + view.properties.asScala.toMap -- ViewCatalog.RESERVED_PROPERTIES.asScala + } + + override def simpleString(maxFields: Int): String = { + s"ShowV2ViewPropertiesExec" + } +} diff --git a/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala new file mode 100644 index 000000000000..4e7700b43978 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ShowV2ViewsExec.scala @@ -0,0 +1,70 @@ +/* + * 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.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.util.StringUtils +import org.apache.spark.sql.connector.catalog.ViewCatalog +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.internal.SQLConf +import scala.collection.mutable.ArrayBuffer + +case class ShowV2ViewsExec( + output: Seq[Attribute], + catalog: ViewCatalog, + namespace: Seq[String], + pattern: Option[String]) + extends V2CommandExec + with LeafExecNode { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + override protected def run(): Seq[InternalRow] = { + val rows = new ArrayBuffer[InternalRow]() + + // handle GLOBAL VIEWS + val globalTemp = SQLConf.get.globalTempDatabase + if (namespace.nonEmpty && globalTemp == namespace.head) { + pattern + .map(p => session.sessionState.catalog.globalTempViewManager.listViewNames(p)) + .getOrElse(session.sessionState.catalog.globalTempViewManager.listViewNames("*")) + .map(name => rows += toCatalystRow(globalTemp, name, true)) + } else { + val views = catalog.listViews(namespace: _*) + views.map { view => + if (pattern.map(StringUtils.filterPattern(Seq(view.name()), _).nonEmpty).getOrElse(true)) { + rows += toCatalystRow(view.namespace().quoted, view.name(), false) + } + } + } + + // include TEMP VIEWS + pattern + .map(p => session.sessionState.catalog.listLocalTempViews(p)) + .getOrElse(session.sessionState.catalog.listLocalTempViews("*")) + .map(v => rows += toCatalystRow(v.database.toArray.quoted, v.table, true)) + + rows.toSeq + } + + override def simpleString(maxFields: Int): String = { + s"ShowV2ViewsExec" + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/TestExtendedParser.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/TestExtendedParser.java new file mode 100644 index 000000000000..bfcb5af235d3 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/TestExtendedParser.java @@ -0,0 +1,231 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.lang.reflect.Field; +import java.util.Collections; +import java.util.List; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.expressions.Term; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.parser.AbstractSqlParser; +import org.apache.spark.sql.catalyst.parser.AstBuilder; +import org.apache.spark.sql.catalyst.parser.ParserInterface; +import org.apache.spark.sql.catalyst.parser.extensions.IcebergSparkSqlExtensionsParser; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class TestExtendedParser { + + private static SparkSession spark; + private static final String SQL_PARSER_FIELD = "sqlParser"; + private ParserInterface originalParser; + + @BeforeAll + public static void before() { + spark = SparkSession.builder().master("local").appName("TestExtendedParser").getOrCreate(); + } + + @AfterAll + public static void after() { + if (spark != null) { + spark.stop(); + } + } + + @BeforeEach + public void saveOriginalParser() throws Exception { + Class clazz = spark.sessionState().getClass(); + Field parserField = null; + while (clazz != null && parserField == null) { + try { + parserField = clazz.getDeclaredField(SQL_PARSER_FIELD); + } catch (NoSuchFieldException e) { + clazz = clazz.getSuperclass(); + } + } + parserField.setAccessible(true); + originalParser = (ParserInterface) parserField.get(spark.sessionState()); + } + + @AfterEach + public void restoreOriginalParser() throws Exception { + setSessionStateParser(spark.sessionState(), originalParser); + } + + /** + * Tests that the Iceberg extended SQL parser can correctly parse a sort order string and return + * the expected RawOrderField. + * + * @throws Exception if reflection access fails + */ + @Test + public void testParseSortOrderWithRealIcebergExtendedParser() throws Exception { + ParserInterface origParser = null; + Class clazz = spark.sessionState().getClass(); + while (clazz != null && origParser == null) { + try { + Field parserField = clazz.getDeclaredField(SQL_PARSER_FIELD); + parserField.setAccessible(true); + origParser = (ParserInterface) parserField.get(spark.sessionState()); + } catch (NoSuchFieldException e) { + clazz = clazz.getSuperclass(); + } + } + assertThat(origParser).isNotNull(); + + IcebergSparkSqlExtensionsParser icebergParser = new IcebergSparkSqlExtensionsParser(origParser); + + setSessionStateParser(spark.sessionState(), icebergParser); + + List fields = + ExtendedParser.parseSortOrder(spark, "id ASC NULLS FIRST"); + + assertThat(fields).isNotEmpty(); + ExtendedParser.RawOrderField first = fields.get(0); + assertThat(first.direction()).isEqualTo(SortDirection.ASC); + assertThat(first.nullOrder()).isEqualTo(NullOrder.NULLS_FIRST); + } + + /** + * Tests that parseSortOrder can find and use an ExtendedParser that is wrapped inside another + * ParserInterface implementation. + * + * @throws Exception if reflection access fails + */ + @Test + public void testParseSortOrderFindsNestedExtendedParser() throws Exception { + ExtendedParser icebergParser = mock(ExtendedParser.class); + + ExtendedParser.RawOrderField field = + new ExtendedParser.RawOrderField( + mock(Term.class), SortDirection.ASC, NullOrder.NULLS_FIRST); + List expected = Collections.singletonList(field); + + when(icebergParser.parseSortOrder("id ASC NULLS FIRST")).thenReturn(expected); + + ParserInterface wrapper = new WrapperParser(icebergParser); + + setSessionStateParser(spark.sessionState(), wrapper); + + List result = + ExtendedParser.parseSortOrder(spark, "id ASC NULLS FIRST"); + assertThat(result).isSameAs(expected); + + verify(icebergParser).parseSortOrder("id ASC NULLS FIRST"); + } + + /** + * Tests that parseSortOrder throws an exception if no ExtendedParser instance can be found in the + * parser chain. + * + * @throws Exception if reflection access fails + */ + @Test + public void testParseSortOrderThrowsWhenNoExtendedParserFound() throws Exception { + ParserInterface dummy = mock(ParserInterface.class); + setSessionStateParser(spark.sessionState(), dummy); + + assertThatThrownBy(() -> ExtendedParser.parseSortOrder(spark, "id ASC")) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Iceberg ExtendedParser"); + } + + /** + * Tests that parseSortOrder can find an ExtendedParser in a parent class field of the parser. + * + * @throws Exception if reflection access fails + */ + @Test + public void testParseSortOrderFindsExtendedParserInParentClassField() throws Exception { + ExtendedParser icebergParser = mock(ExtendedParser.class); + ExtendedParser.RawOrderField field = + new ExtendedParser.RawOrderField( + mock(Term.class), SortDirection.ASC, NullOrder.NULLS_FIRST); + List expected = Collections.singletonList(field); + when(icebergParser.parseSortOrder("id ASC NULLS FIRST")).thenReturn(expected); + ParserInterface parser = new GrandChildParser(icebergParser); + setSessionStateParser(spark.sessionState(), parser); + + List result = + ExtendedParser.parseSortOrder(spark, "id ASC NULLS FIRST"); + assertThat(result).isSameAs(expected); + verify(icebergParser).parseSortOrder("id ASC NULLS FIRST"); + } + + private static void setSessionStateParser(Object sessionState, ParserInterface parser) + throws Exception { + Class clazz = sessionState.getClass(); + Field targetField = null; + while (clazz != null && targetField == null) { + try { + targetField = clazz.getDeclaredField(SQL_PARSER_FIELD); + } catch (NoSuchFieldException e) { + clazz = clazz.getSuperclass(); + } + } + if (targetField == null) { + throw new IllegalStateException( + "No suitable sqlParser field found in sessionState class hierarchy!"); + } + targetField.setAccessible(true); + targetField.set(sessionState, parser); + } + + private static class WrapperParser extends AbstractSqlParser { + private final ParserInterface delegate; + private String name; + + WrapperParser(ParserInterface delegate) { + this.delegate = delegate; + this.name = "delegate"; + } + + public ParserInterface getDelegate() { + return delegate; + } + + @Override + public AstBuilder astBuilder() { + return null; + } + } + + private static class ChildParser extends WrapperParser { + ChildParser(ParserInterface parent) { + super(parent); + } + } + + private static class GrandChildParser extends ChildParser { + GrandChildParser(ParserInterface parent) { + super(parent); + } + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java new file mode 100644 index 000000000000..8918dfec6584 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/Employee.java @@ -0,0 +1,66 @@ +/* + * 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.extensions; + +import java.util.Objects; + +public class Employee { + private Integer id; + private String dep; + + public Employee() {} + + public Employee(Integer id, String dep) { + this.id = id; + this.dep = dep; + } + + public Integer getId() { + return id; + } + + public void setId(Integer id) { + this.id = id; + } + + public String getDep() { + return dep; + } + + public void setDep(String dep) { + this.dep = dep; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + Employee employee = (Employee) other; + return Objects.equals(id, employee.id) && Objects.equals(dep, employee.dep); + } + + @Override + public int hashCode() { + return Objects.hash(id, dep); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java new file mode 100644 index 000000000000..796c47b545cc --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ExtensionsTestBase.java @@ -0,0 +1,72 @@ +/* + * 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.extensions; + +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; + +import java.net.InetAddress; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.TestBase; +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.BeforeAll; + +public abstract class ExtensionsTestBase extends CatalogTestBase { + + private static final Random RANDOM = ThreadLocalRandom.current(); + + @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") + .config( + SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), String.valueOf(RANDOM.nextBoolean())) + .enableHiveSupport() + .getOrCreate(); + + TestBase.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + TestBase.catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ProcedureUtil.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ProcedureUtil.java new file mode 100644 index 000000000000..de4acd74a7ed --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/ProcedureUtil.java @@ -0,0 +1,54 @@ +/* + * 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.extensions; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.UUID; +import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile; +import org.apache.iceberg.PartitionStatisticsFile; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.PositionOutputStream; + +public class ProcedureUtil { + + private ProcedureUtil() {} + + static PartitionStatisticsFile writePartitionStatsFile( + long snapshotId, String statsLocation, FileIO fileIO) { + PositionOutputStream positionOutputStream; + try { + positionOutputStream = fileIO.newOutputFile(statsLocation).create(); + positionOutputStream.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + return ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(snapshotId) + .fileSizeInBytes(42L) + .path(statsLocation) + .build(); + } + + static String statsFileLocation(String tableLocation) { + String statsFileName = "stats-file-" + UUID.randomUUID(); + return tableLocation.replaceFirst("file:", "") + "/metadata/" + statsFileName; + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java new file mode 100644 index 000000000000..830d07d86eab --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java @@ -0,0 +1,107 @@ +/* + * 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.extensions; + +import static scala.collection.JavaConverters.seqAsJavaListConverter; + +import java.util.Collection; +import java.util.List; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.execution.CommandResultExec; +import org.apache.spark.sql.execution.SparkPlan; +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper; +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec; +import scala.PartialFunction; +import scala.collection.Seq; + +public class SparkPlanUtil { + + private static final AdaptiveSparkPlanHelper SPARK_HELPER = new AdaptiveSparkPlanHelper() {}; + + private SparkPlanUtil() {} + + public static List collectLeaves(SparkPlan plan) { + return toJavaList(SPARK_HELPER.collectLeaves(actualPlan(plan))); + } + + public static List collectBatchScans(SparkPlan plan) { + List leaves = collectLeaves(plan); + return leaves.stream() + .filter(scan -> scan instanceof BatchScanExec) + .collect(Collectors.toList()); + } + + private static SparkPlan actualPlan(SparkPlan plan) { + if (plan instanceof CommandResultExec) { + return ((CommandResultExec) plan).commandPhysicalPlan(); + } else { + return plan; + } + } + + public static List collectExprs( + SparkPlan sparkPlan, Predicate predicate) { + Seq> seq = + SPARK_HELPER.collect( + sparkPlan, + new PartialFunction>() { + @Override + public List apply(SparkPlan plan) { + List exprs = Lists.newArrayList(); + + for (Expression expr : toJavaList(plan.expressions())) { + exprs.addAll(collectExprs(expr, predicate)); + } + + return exprs; + } + + @Override + public boolean isDefinedAt(SparkPlan plan) { + return true; + } + }); + return toJavaList(seq).stream().flatMap(Collection::stream).collect(Collectors.toList()); + } + + private static List collectExprs( + Expression expression, Predicate predicate) { + Seq seq = + expression.collect( + new PartialFunction() { + @Override + public Expression apply(Expression expr) { + return expr; + } + + @Override + public boolean isDefinedAt(Expression expr) { + return predicate.test(expr); + } + }); + return toJavaList(seq); + } + + private static List toJavaList(Seq seq) { + return seqAsJavaListConverter(seq).asJava(); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java new file mode 100644 index 000000000000..b5d641576314 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -0,0 +1,461 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.DataOperations.DELETE; +import static org.apache.iceberg.DataOperations.OVERWRITE; +import static org.apache.iceberg.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.apache.iceberg.SnapshotSummary.ADDED_DELETE_FILES_PROP; +import static org.apache.iceberg.SnapshotSummary.ADDED_DVS_PROP; +import static org.apache.iceberg.SnapshotSummary.ADDED_FILES_PROP; +import static org.apache.iceberg.SnapshotSummary.ADD_POS_DELETE_FILES_PROP; +import static org.apache.iceberg.SnapshotSummary.CHANGED_PARTITION_COUNT_PROP; +import static org.apache.iceberg.SnapshotSummary.DELETED_FILES_PROP; +import static org.apache.iceberg.TableProperties.DATA_PLANNING_MODE; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DELETE_PLANNING_MODE; +import static org.apache.iceberg.TableProperties.FORMAT_VERSION; +import static org.apache.iceberg.TableProperties.ORC_VECTORIZATION_ENABLED; +import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED; +import static org.apache.iceberg.TableProperties.SPARK_WRITE_PARTITIONED_FANOUT_ENABLED; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.execution.SparkPlan; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class SparkRowLevelOperationsTestBase extends ExtensionsTestBase { + + private static final Random RANDOM = ThreadLocalRandom.current(); + + @Parameter(index = 3) + protected FileFormat fileFormat; + + @Parameter(index = 4) + protected boolean vectorized; + + @Parameter(index = 5) + protected String distributionMode; + + @Parameter(index = 6) + protected boolean fanoutEnabled; + + @Parameter(index = 7) + protected String branch; + + @Parameter(index = 8) + protected PlanningMode planningMode; + + @Parameter(index = 9) + protected int formatVersion; + + @Parameters( + name = + "catalogName = {0}, implementation = {1}, config = {2}," + + " format = {3}, vectorized = {4}, distributionMode = {5}," + + " fanout = {6}, branch = {7}, planningMode = {8}, formatVersion = {9}") + public static Object[][] parameters() { + return new Object[][] { + { + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default"), + FileFormat.ORC, + true, + WRITE_DISTRIBUTION_MODE_NONE, + true, + SnapshotRef.MAIN_BRANCH, + LOCAL, + 2 + }, + { + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default"), + FileFormat.PARQUET, + true, + WRITE_DISTRIBUTION_MODE_NONE, + false, + "test", + DISTRIBUTED, + 2 + }, + { + "testhadoop", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "hadoop"), + FileFormat.PARQUET, + RANDOM.nextBoolean(), + WRITE_DISTRIBUTION_MODE_HASH, + true, + null, + LOCAL, + 2 + }, + { + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "clients", "1", + "parquet-enabled", "false", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + ), + FileFormat.AVRO, + false, + WRITE_DISTRIBUTION_MODE_RANGE, + false, + "test", + DISTRIBUTED, + 2 + }, + { + "testhadoop", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "hadoop"), + FileFormat.PARQUET, + RANDOM.nextBoolean(), + WRITE_DISTRIBUTION_MODE_HASH, + true, + null, + LOCAL, + 3 + }, + { + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", + "hive", + "default-namespace", + "default", + "clients", + "1", + "parquet-enabled", + "false", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + ), + FileFormat.AVRO, + false, + WRITE_DISTRIBUTION_MODE_RANGE, + false, + "test", + DISTRIBUTED, + 3 + }, + }; + } + + protected abstract Map extraTableProperties(); + + protected void initTable() { + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s', '%s' '%s', '%s' '%s', '%s' '%s', '%s' '%s', '%s' '%s')", + tableName, + DEFAULT_FILE_FORMAT, + fileFormat, + WRITE_DISTRIBUTION_MODE, + distributionMode, + SPARK_WRITE_PARTITIONED_FANOUT_ENABLED, + String.valueOf(fanoutEnabled), + DATA_PLANNING_MODE, + planningMode.modeName(), + DELETE_PLANNING_MODE, + planningMode.modeName(), + FORMAT_VERSION, + formatVersion); + + switch (fileFormat) { + case PARQUET: + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%b')", + tableName, PARQUET_VECTORIZATION_ENABLED, vectorized); + break; + case ORC: + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%b')", + tableName, ORC_VECTORIZATION_ENABLED, vectorized); + break; + case AVRO: + assertThat(vectorized).isFalse(); + break; + } + + Map props = extraTableProperties(); + props.forEach( + (prop, value) -> { + sql("ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", tableName, prop, value); + }); + } + + protected void createAndInitTable(String schema) { + createAndInitTable(schema, null); + } + + protected void createAndInitTable(String schema, String jsonData) { + createAndInitTable(schema, "", jsonData); + } + + protected void createAndInitTable(String schema, String partitioning, String jsonData) { + sql("CREATE TABLE %s (%s) USING iceberg %s", tableName, schema, partitioning); + initTable(); + + if (jsonData != null) { + try { + Dataset ds = toDS(schema, jsonData); + ds.coalesce(1).writeTo(tableName).append(); + createBranchIfNeeded(); + } catch (NoSuchTableException e) { + throw new RuntimeException("Failed to write data", e); + } + } + } + + protected void append(String table, String jsonData) { + append(table, null, jsonData); + } + + protected void append(String table, String schema, String jsonData) { + try { + Dataset ds = toDS(schema, jsonData); + ds.coalesce(1).writeTo(table).append(); + } catch (NoSuchTableException e) { + throw new RuntimeException("Failed to write data", e); + } + } + + protected void createOrReplaceView(String name, String jsonData) { + createOrReplaceView(name, null, jsonData); + } + + protected void createOrReplaceView(String name, String schema, String jsonData) { + Dataset ds = toDS(schema, jsonData); + ds.createOrReplaceTempView(name); + } + + protected void createOrReplaceView(String name, List data, Encoder encoder) { + spark.createDataset(data, encoder).createOrReplaceTempView(name); + } + + private Dataset toDS(String schema, String jsonData) { + List jsonRows = + Arrays.stream(jsonData.split("\n")) + .filter(str -> !str.trim().isEmpty()) + .collect(Collectors.toList()); + Dataset jsonDS = spark.createDataset(jsonRows, Encoders.STRING()); + + if (schema != null) { + return spark.read().schema(schema).json(jsonDS); + } else { + return spark.read().json(jsonDS); + } + } + + protected void validateDelete( + Snapshot snapshot, String changedPartitionCount, String deletedDataFiles) { + validateSnapshot(snapshot, DELETE, changedPartitionCount, deletedDataFiles, null, null); + } + + protected void validateCopyOnWrite( + Snapshot snapshot, + String changedPartitionCount, + String deletedDataFiles, + String addedDataFiles) { + String operation = null == addedDataFiles && null != deletedDataFiles ? DELETE : OVERWRITE; + validateSnapshot( + snapshot, operation, changedPartitionCount, deletedDataFiles, null, addedDataFiles); + } + + protected void validateMergeOnRead( + Snapshot snapshot, + String changedPartitionCount, + String addedDeleteFiles, + String addedDataFiles) { + String operation = null == addedDataFiles && null != addedDeleteFiles ? DELETE : OVERWRITE; + validateSnapshot( + snapshot, operation, changedPartitionCount, null, addedDeleteFiles, addedDataFiles); + } + + protected void validateSnapshot( + Snapshot snapshot, + String operation, + String changedPartitionCount, + String deletedDataFiles, + String addedDeleteFiles, + String addedDataFiles) { + assertThat(snapshot.operation()).as("Operation must match").isEqualTo(operation); + validateProperty(snapshot, CHANGED_PARTITION_COUNT_PROP, changedPartitionCount); + validateProperty(snapshot, DELETED_FILES_PROP, deletedDataFiles); + validateProperty(snapshot, ADDED_DELETE_FILES_PROP, addedDeleteFiles); + validateProperty(snapshot, ADDED_FILES_PROP, addedDataFiles); + if (formatVersion >= 3) { + validateProperty(snapshot, ADDED_DVS_PROP, addedDeleteFiles); + assertThat(snapshot.summary()).doesNotContainKey(ADD_POS_DELETE_FILES_PROP); + } + } + + protected void validateProperty(Snapshot snapshot, String property, Set expectedValues) { + String actual = snapshot.summary().get(property); + assertThat(actual) + .as( + "Snapshot property " + + property + + " has unexpected value, actual = " + + actual + + ", expected one of : " + + String.join(",", expectedValues)) + .isIn(expectedValues); + } + + protected void validateProperty(Snapshot snapshot, String property, String expectedValue) { + if (null == expectedValue) { + assertThat(snapshot.summary()).doesNotContainKey(property); + } else { + assertThat(snapshot.summary()) + .as("Snapshot property " + property + " has unexpected value.") + .containsEntry(property, expectedValue); + } + } + + protected void sleep(long millis) { + try { + Thread.sleep(millis); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + protected DataFile writeDataFile(Table table, List records) { + try { + OutputFile file = + Files.localOutput( + temp.resolve(fileFormat.addExtension(UUID.randomUUID().toString())).toFile()); + + DataWriter dataWriter = + Parquet.writeData(file) + .forTable(table) + .createWriterFunc(GenericParquetWriter::create) + .overwrite() + .build(); + + try { + for (GenericRecord record : records) { + dataWriter.write(record); + } + } finally { + dataWriter.close(); + } + + return dataWriter.toDataFile(); + + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + protected String commitTarget() { + return branch == null ? tableName : String.format("%s.branch_%s", tableName, branch); + } + + @Override + protected String selectTarget() { + return branch == null ? tableName : String.format("%s VERSION AS OF '%s'", tableName, branch); + } + + protected void createBranchIfNeeded() { + if (branch != null && !branch.equals(SnapshotRef.MAIN_BRANCH)) { + sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branch); + } + } + + // ORC currently does not support vectorized reads with deletes + protected boolean supportsVectorization() { + return vectorized && (isParquet() || isCopyOnWrite()); + } + + private boolean isParquet() { + return fileFormat.equals(FileFormat.PARQUET); + } + + private boolean isCopyOnWrite() { + return extraTableProperties().containsValue(RowLevelOperationMode.COPY_ON_WRITE.modeName()); + } + + protected void assertAllBatchScansVectorized(SparkPlan plan) { + List batchScans = SparkPlanUtil.collectBatchScans(plan); + assertThat(batchScans).hasSizeGreaterThan(0).allMatch(SparkPlan::supportsColumnar); + } + + protected void createTableWithDeleteGranularity( + String schema, String partitionedBy, DeleteGranularity deleteGranularity) { + createAndInitTable(schema, partitionedBy, null /* empty */); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", + tableName, TableProperties.DELETE_GRANULARITY, deleteGranularity); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java new file mode 100644 index 000000000000..29993380b50c --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -0,0 +1,1479 @@ +/* + * 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.extensions; + +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.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.avro.Schema; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.DatumWriter; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestReader; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.joda.time.DateTime; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestAddFilesProcedure extends ExtensionsTestBase { + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, formatVersion = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties(), + 1 + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + 2 + }, + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties(), + 2 + } + }; + } + + @Parameter(index = 3) + private int formatVersion; + + private final String sourceTableName = "source_table"; + private File fileTableDir; + + @BeforeEach + public void setupTempDirs() { + fileTableDir = temp.toFile(); + } + + @AfterEach + public void dropTables() { + sql("DROP TABLE IF EXISTS %s PURGE", sourceTableName); + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void addDataUnpartitioned() { + createUnpartitionedFileTable("parquet"); + + createIcebergTable("id Integer, name String, dept String, subdept String"); + + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT * FROM %s ORDER BY id", sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void deleteAndAddBackUnpartitioned() { + createUnpartitionedFileTable("parquet"); + + createIcebergTable("id Integer, name String, dept String, subdept String"); + + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + String deleteData = "DELETE FROM %s"; + sql(deleteData, tableName); + + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT * FROM %s ORDER BY id", sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @Disabled // TODO Classpath issues prevent us from actually writing to a Spark ORC table + public void addDataUnpartitionedOrc() { + createUnpartitionedFileTable("orc"); + + String createIceberg = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg"; + + sql(createIceberg, tableName); + + Object result = + scalarSql( + "CALL %s.system.add_files('%s', '`orc`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertThat(result).isEqualTo(2L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT * FROM %s ORDER BY id", sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addAvroFile() throws Exception { + // Spark Session Catalog cannot load metadata tables + // with "The namespace in session catalog must have exactly one name part" + assumeThat(catalogName).isNotEqualTo("spark_catalog"); + + // Create an Avro file + + Schema schema = + SchemaBuilder.record("record") + .fields() + .requiredInt("id") + .requiredString("data") + .endRecord(); + GenericRecord record1 = new GenericData.Record(schema); + record1.put("id", 1L); + record1.put("data", "a"); + GenericRecord record2 = new GenericData.Record(schema); + record2.put("id", 2L); + record2.put("data", "b"); + File outputFile = temp.resolve("test.avro").toFile(); + + DatumWriter datumWriter = new GenericDatumWriter(schema); + DataFileWriter dataFileWriter = new DataFileWriter(datumWriter); + dataFileWriter.create(schema, outputFile); + dataFileWriter.append(record1); + dataFileWriter.append(record2); + dataFileWriter.close(); + + createIcebergTable("id Long, data String"); + + List result = + sql( + "CALL %s.system.add_files('%s', '`avro`.`%s`')", + catalogName, tableName, outputFile.getPath()); + assertOutput(result, 1L, 1L); + + List expected = Lists.newArrayList(new Object[] {1L, "a"}, new Object[] {2L, "b"}); + + assertEquals( + "Iceberg table contains correct data", + expected, + sql("SELECT * FROM %s ORDER BY id", tableName)); + + List actualRecordCount = + sql("select %s from %s.files", DataFile.RECORD_COUNT.name(), tableName); + List expectedRecordCount = Lists.newArrayList(); + expectedRecordCount.add(new Object[] {2L}); + assertEquals( + "Iceberg file metadata should have correct metadata count", + expectedRecordCount, + actualRecordCount); + } + + // TODO Adding spark-avro doesn't work in tests + @Disabled + public void addDataUnpartitionedAvro() { + createUnpartitionedFileTable("avro"); + + String createIceberg = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg"; + + sql(createIceberg, tableName); + + Object result = + scalarSql( + "CALL %s.system.add_files('%s', '`avro`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertThat(result).isEqualTo(2L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT * FROM %s ORDER BY id", sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addDataUnpartitionedHive() { + createUnpartitionedHiveTable(); + + createIcebergTable("id Integer, name String, dept String, subdept String"); + + List result = + sql("CALL %s.system.add_files('%s', '%s')", catalogName, tableName, sourceTableName); + + assertOutput(result, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT * FROM %s ORDER BY id", sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addDataUnpartitionedExtraCol() { + createUnpartitionedFileTable("parquet"); + + createIcebergTable("id Integer, name String, dept String, subdept String, foo string"); + + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT * FROM %s ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addDataUnpartitionedMissingCol() { + createUnpartitionedFileTable("parquet"); + + createIcebergTable("id Integer, name String, dept String"); + + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept FROM %s ORDER BY id", sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addDataPartitionedMissingCol() { + createPartitionedFileTable("parquet"); + + createIcebergTable("id Integer, name String, dept String", "PARTITIONED BY (id)"); + + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 8L, 4L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept FROM %s ORDER BY id", sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addDataPartitioned() { + createPartitionedFileTable("parquet"); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 8L, 4L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @Disabled // TODO Classpath issues prevent us from actually writing to a Spark ORC table + public void addDataPartitionedOrc() { + createPartitionedFileTable("orc"); + + String createIceberg = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)"; + + sql(createIceberg, tableName); + + Object result = + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertThat(result).isEqualTo(8L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + // TODO Adding spark-avro doesn't work in tests + @Disabled + public void addDataPartitionedAvro() { + createPartitionedFileTable("avro"); + + String createIceberg = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg PARTITIONED BY (id)"; + + sql(createIceberg, tableName); + + Object result = + scalarSql( + "CALL %s.system.add_files('%s', '`avro`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertThat(result).isEqualTo(8L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addDataPartitionedHive() { + createPartitionedHiveTable(); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + List result = + sql("CALL %s.system.add_files('%s', '%s')", catalogName, tableName, sourceTableName); + + assertOutput(result, 8L, 4L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addPartitionToPartitioned() { + createPartitionedFileTable("parquet"); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void deleteAndAddBackPartitioned() { + createPartitionedFileTable("parquet"); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + String deleteData = "DELETE FROM %s where id = 1"; + sql(deleteData, tableName); + + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addPartitionToPartitionedSnapshotIdInheritanceEnabledInTwoRuns() + throws NoSuchTableException, ParseException { + createPartitionedFileTable("parquet"); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", + tableName, TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED); + + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 2))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s WHERE id < 3 ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + + manifestSpecMatchesTableSpec(); + + verifyUUIDInPath(); + } + + @TestTemplate + public void addPartitionsFromHiveSnapshotInheritanceEnabled() + throws NoSuchTableException, ParseException { + createPartitionedHiveTable(); + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", + tableName, TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED); + + sql("CALL %s.system.add_files('%s', '%s')", catalogName, tableName, sourceTableName); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + + manifestSpecMatchesTableSpec(); + + verifyUUIDInPath(); + } + + @TestTemplate + public void addDataPartitionedByDateToPartitioned() { + createDatePartitionedFileTable("parquet"); + + createIcebergTable("id Integer, name String, date Date", "PARTITIONED BY (date)"); + + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('date', '2021-01-01'))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, date FROM %s WHERE date = '2021-01-01' ORDER BY id", sourceTableName), + sql("SELECT id, name, date FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addDataPartitionedVerifyPartitionTypeInferredCorrectly() { + createTableWithTwoPartitions("parquet"); + + createIcebergTable( + "id Integer, name String, date Date, dept String", "PARTITIONED BY (date, dept)"); + + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('date', '2021-01-01'))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + String sqlFormat = + "SELECT id, name, dept, date FROM %s WHERE date = '2021-01-01' and dept= '01' ORDER BY id"; + assertEquals( + "Iceberg table contains correct data", + sql(sqlFormat, sourceTableName), + sql(sqlFormat, tableName)); + } + + @TestTemplate + public void addFilteredPartitionsToPartitioned() { + createCompositePartitionedTable("parquet"); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id, dept)"); + + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addFilteredPartitionsToPartitioned2() { + createCompositePartitionedTable("parquet"); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id, dept)"); + + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('dept', 'hr'))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 6L, 3L); + + assertEquals( + "Iceberg table contains correct data", + sql( + "SELECT id, name, dept, subdept FROM %s WHERE dept = 'hr' ORDER BY id", + sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addFilteredPartitionsToPartitionedWithNullValueFilteringOnId() { + createCompositePartitionedTableWithNullValueInPartitionColumn("parquet"); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id, dept)"); + + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addFilteredPartitionsToPartitionedWithNullValueFilteringOnDept() { + createCompositePartitionedTableWithNullValueInPartitionColumn("parquet"); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id, dept)"); + + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('dept', 'hr'))", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 6L, 3L); + + assertEquals( + "Iceberg table contains correct data", + sql( + "SELECT id, name, dept, subdept FROM %s WHERE dept = 'hr' ORDER BY id", + sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addAllPartitionsToPartitionedWithNullValue() { + createCompositePartitionedTableWithNullValueInPartitionColumn("parquet"); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id, dept)"); + + // Add all partitions including null partitions. + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 10L, 5L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addAllPartitionsToNonStringPartitionedWithNullValue() { + createPartitionedTableWithNullValueInPartitionColumnOnId("parquet"); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + // Add all partitions including null partitions. + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 10L, 5L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addPartitionsWithNullValueShouldAddFilesToNullPartition() { + // This test is to ensure that "null" string partition is not incorrectly created. + + createPartitionedTableWithNullValueInPartitionColumnOnDept("parquet"); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (dept)"); + + // Add all partitions including null partitions. + List result = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 6L, 3L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + + // Check if correct partitions are created + List actualRows = + sql("SELECT partition from %s.partitions ORDER BY partition", tableName); + assertEquals( + "Other partitions should match", + ImmutableList.of( + row(new Object[] {new Object[] {null}}), + row(new Object[] {new Object[] {"facilities"}}), + row(new Object[] {new Object[] {"hr"}})), + actualRows); + } + + @TestTemplate + public void addFileTableOldSpecDataAfterPartitionSpecEvolved() + throws NoSuchTableException, ParseException { + createPartitionedFileTable("parquet"); + createIcebergTable( + "id Integer, name String, dept String, subdept String", + "PARTITIONED BY (id, dept, subdept)"); + sql("ALTER TABLE %s DROP PARTITION FIELD dept", tableName); + sql( + "ALTER TABLE %s DROP PARTITION FIELD subdept", + tableName); // This spec now matches the partitioning of the parquet table + sql("ALTER TABLE %s ADD PARTITION FIELD subdept", tableName); + + if (formatVersion == 1) { + // In V1, since we are dropping the partition field, it adds a void transform which will not + // match with the input spec + assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot find a partition spec in Iceberg table %s that matches the partition columns ([id]) in input table", + tableName); + return; + } + + List result = + sql( + "CALL %s.system.add_files(table => '%s', source_table => '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + + assertOutput(result, 8L, 4L); + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + // Find the spec that matches the partitioning of the parquet table + PartitionSpec compatibleSpec = + table.specs().values().stream() + .filter(spec -> spec.fields().size() == 1) + .filter(spec -> "id".equals(spec.fields().get(0).name())) + .findFirst() + .orElse(null); + + assertThat(compatibleSpec).isNotNull(); + manifestSpecMatchesGivenSpec(table, compatibleSpec); + verifyUUIDInPath(); + } + + @TestTemplate + public void addFileTableNoCompatibleSpec() { + createPartitionedFileTable("parquet"); + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (dept)"); + sql("ALTER TABLE %s ADD PARTITION FIELD subdept", tableName); + + String fullTableName = tableName; + if (implementation.equals(SparkCatalogConfig.SPARK_SESSION.implementation())) { + fullTableName = String.format("%s.%s", catalogName, tableName); + } + assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files(table => '%s', source_table => '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot find a partition spec in Iceberg table %s that matches the partition columns (%s) in input table", + fullTableName, "[id]"); + } + + @TestTemplate + public void addWeirdCaseHiveTable() { + createWeirdCaseTable(); + + createIcebergTable( + "id Integer, `naMe` String, dept String, subdept String", "PARTITIONED BY (`naMe`)"); + + List result = + sql( + "CALL %s.system.add_files('%s', '%s', map('naMe', 'John Doe'))", + catalogName, tableName, sourceTableName); + + assertOutput(result, 2L, 1L); + + /* + While we would like to use + SELECT id, `naMe`, dept, subdept FROM %s WHERE `naMe` = 'John Doe' ORDER BY id + Spark does not actually handle this pushdown correctly for hive based tables and it returns 0 records + */ + List expected = + sql("SELECT id, `naMe`, dept, subdept from %s ORDER BY id", sourceTableName).stream() + .filter(r -> r[1].equals("John Doe")) + .collect(Collectors.toList()); + + // TODO when this assert breaks Spark fixed the pushdown issue + assertThat( + sql( + "SELECT id, `naMe`, dept, subdept from %s WHERE `naMe` = 'John Doe' ORDER BY id", + sourceTableName)) + .as("If this assert breaks it means that Spark has fixed the pushdown issue") + .isEmpty(); + + // Pushdown works for iceberg + assertThat( + sql( + "SELECT id, `naMe`, dept, subdept FROM %s WHERE `naMe` = 'John Doe' ORDER BY id", + tableName)) + .as("We should be able to pushdown mixed case partition keys") + .hasSize(2); + + assertEquals( + "Iceberg table contains correct data", + expected, + sql("SELECT id, `naMe`, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void addPartitionToPartitionedHive() { + createPartitionedHiveTable(); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + List result = + sql( + "CALL %s.system.add_files('%s', '%s', map('id', 1))", + catalogName, tableName, sourceTableName); + + assertOutput(result, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void invalidDataImport() { + createPartitionedFileTable("parquet"); + + createIcebergTable("id Integer, name String, dept String, subdept String"); + + assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot find a partition spec in Iceberg table") + .hasMessageContaining("that matches the partition columns"); + + assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot find a partition spec in Iceberg table") + .hasMessageContaining("that matches the partition columns"); + } + + @TestTemplate + public void invalidDataImportPartitioned() { + createUnpartitionedFileTable("parquet"); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('x', '1', 'y', '2'))", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot find a partition spec in Iceberg table") + .hasMessageContaining("that matches the partition columns"); + + assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('dept', '2'))", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot find a partition spec in Iceberg table") + .hasMessageContaining("that matches the partition columns"); + } + + @TestTemplate + public void partitionColumnCountMismatchInFilter() { + createPartitionedHiveTable(); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '%s', map('id', '0', 'dept', '1'))", + catalogName, tableName, sourceTableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add data files to target table") + .hasMessageContaining( + "because that table is partitioned, but the number of columns in the provided partition filter (2)" + + " is greater than the number of partitioned columns in table (1)"); + } + + @TestTemplate + public void invalidPartitionColumnsInFilter() { + createPartitionedHiveTable(); + + String icebergTablePartitionNames = "id"; + createIcebergTable( + "id Integer, name String, dept String, subdept String", + String.format("PARTITIONED BY (%s)", icebergTablePartitionNames)); + assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '%s', map('dept', '1'))", + catalogName, tableName, sourceTableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot add files to target table") + .hasMessageContaining( + "specified partition filter refers to columns that are not partitioned: [dept]") + .hasMessageContaining("Valid partition columns: [%s]", icebergTablePartitionNames); + } + + @TestTemplate + public void addTwice() { + createPartitionedHiveTable(); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + List result1 = + sql( + "CALL %s.system.add_files(" + + "table => '%s', " + + "source_table => '%s', " + + "partition_filter => map('id', 1))", + catalogName, tableName, sourceTableName); + assertOutput(result1, 2L, 1L); + + List result2 = + sql( + "CALL %s.system.add_files(" + + "table => '%s', " + + "source_table => '%s', " + + "partition_filter => map('id', 2))", + catalogName, tableName, sourceTableName); + assertOutput(result2, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", tableName)); + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s WHERE id = 2 ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s WHERE id = 2 ORDER BY id", tableName)); + } + + @TestTemplate + public void duplicateDataPartitioned() { + createPartitionedHiveTable(); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + sql( + "CALL %s.system.add_files(" + + "table => '%s', " + + "source_table => '%s', " + + "partition_filter => map('id', 1))", + catalogName, tableName, sourceTableName); + + assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files(" + + "table => '%s', " + + "source_table => '%s', " + + "partition_filter => map('id', 1))", + catalogName, tableName, sourceTableName)) + .isInstanceOf(IllegalStateException.class) + .hasMessageStartingWith( + "Cannot complete import because data files to be imported already" + + " exist within the target table"); + } + + @TestTemplate + public void duplicateDataPartitionedAllowed() { + createPartitionedHiveTable(); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + List result1 = + sql( + "CALL %s.system.add_files(" + + "table => '%s', " + + "source_table => '%s', " + + "partition_filter => map('id', 1))", + catalogName, tableName, sourceTableName); + + assertOutput(result1, 2L, 1L); + + List result2 = + sql( + "CALL %s.system.add_files(" + + "table => '%s', " + + "source_table => '%s', " + + "partition_filter => map('id', 1)," + + "check_duplicate_files => false)", + catalogName, tableName, sourceTableName); + + assertOutput(result2, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql( + "SELECT id, name, dept, subdept FROM %s WHERE id = 1 UNION ALL " + + "SELECT id, name, dept, subdept FROM %s WHERE id = 1", + sourceTableName, sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s", tableName, tableName)); + } + + @TestTemplate + public void duplicateDataUnpartitioned() { + createUnpartitionedHiveTable(); + + createIcebergTable("id Integer, name String, dept String, subdept String"); + + sql("CALL %s.system.add_files('%s', '%s')", catalogName, tableName, sourceTableName); + + assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '%s')", + catalogName, tableName, sourceTableName)) + .isInstanceOf(IllegalStateException.class) + .hasMessageStartingWith( + "Cannot complete import because data files to be imported already" + + " exist within the target table"); + } + + @TestTemplate + public void duplicateDataUnpartitionedAllowed() { + createUnpartitionedHiveTable(); + + createIcebergTable("id Integer, name String, dept String, subdept String"); + + List result1 = + sql("CALL %s.system.add_files('%s', '%s')", catalogName, tableName, sourceTableName); + assertOutput(result1, 2L, 1L); + + List result2 = + sql( + "CALL %s.system.add_files(" + + "table => '%s', " + + "source_table => '%s'," + + "check_duplicate_files => false)", + catalogName, tableName, sourceTableName); + assertOutput(result2, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql( + "SELECT * FROM (SELECT * FROM %s UNION ALL " + "SELECT * from %s) ORDER BY id", + sourceTableName, sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testEmptyImportDoesNotThrow() { + createIcebergTable("id Integer, name String, dept String, subdept String"); + + // Empty path based import + List pathResult = + sql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath()); + assertOutput(pathResult, 0L, 0L); + assertEquals( + "Iceberg table contains no added data when importing from an empty path", + EMPTY_QUERY_RESULT, + sql("SELECT * FROM %s ORDER BY id", tableName)); + + // Empty table based import + String createHive = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) STORED AS parquet"; + sql(createHive, sourceTableName); + + List tableResult = + sql("CALL %s.system.add_files('%s', '%s')", catalogName, tableName, sourceTableName); + assertOutput(tableResult, 0L, 0L); + assertEquals( + "Iceberg table contains no added data when importing from an empty table", + EMPTY_QUERY_RESULT, + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testPartitionedImportFromEmptyPartitionDoesNotThrow() { + createPartitionedHiveTable(); + + final int emptyPartitionId = 999; + // Add an empty partition to the hive table + sql( + "ALTER TABLE %s ADD PARTITION (id = '%d') LOCATION '%d'", + sourceTableName, emptyPartitionId, emptyPartitionId); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + List tableResult = + sql( + "CALL %s.system.add_files(" + + "table => '%s', " + + "source_table => '%s', " + + "partition_filter => map('id', %d))", + catalogName, tableName, sourceTableName, emptyPartitionId); + + assertOutput(tableResult, 0L, 0L); + assertEquals( + "Iceberg table contains no added data when importing from an empty table", + EMPTY_QUERY_RESULT, + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testAddFilesWithParallelism() { + createUnpartitionedHiveTable(); + + String createIceberg = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg"; + + sql(createIceberg, tableName); + + List result = + sql( + "CALL %s.system.add_files(table => '%s', source_table => '%s', parallelism => 2)", + catalogName, tableName, sourceTableName); + + assertEquals("Procedure output must match", ImmutableList.of(row(2L, 1L)), result); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT * FROM %s ORDER BY id", sourceTableName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testAddFilesPartitionedWithParallelism() { + createPartitionedHiveTable(); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + List result = + sql( + "CALL %s.system.add_files(table => '%s', source_table => '%s', parallelism => 2)", + catalogName, tableName, sourceTableName); + + assertOutput(result, 8L, 4L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testAddFilesWithInvalidParallelism() { + createUnpartitionedHiveTable(); + + createIcebergTable( + "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.add_files(table => '%s', source_table => '%s', parallelism => -1)", + catalogName, tableName, sourceTableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Parallelism should be larger than 0"); + } + + @TestTemplate + public void testAddFilesToTableWithManySpecs() { + createPartitionedHiveTable(); + createIcebergTable("id Integer, name String, dept String, subdept String"); // Spec 0 + + sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName); // Spec 1 + sql("ALTER TABLE %s ADD PARTITION FIELD name", tableName); // Spec 2 + sql("ALTER TABLE %s ADD PARTITION FIELD subdept", tableName); // Spec 3 + + List result = + sql( + "CALL %s.system.add_files('%s', '%s', map('id', 1))", + catalogName, tableName, sourceTableName); + + assertOutput(result, 2L, 1L); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT id, name, dept, subdept FROM %s WHERE id = 1 ORDER BY id", sourceTableName), + sql("SELECT id, name, dept, subdept FROM %s ORDER BY id", tableName)); + } + + private static final List EMPTY_QUERY_RESULT = Lists.newArrayList(); + + private static final StructField[] STRUCT = { + new StructField("id", DataTypes.IntegerType, true, Metadata.empty()), + new StructField("name", DataTypes.StringType, true, Metadata.empty()), + new StructField("dept", DataTypes.StringType, true, Metadata.empty()), + new StructField("subdept", DataTypes.StringType, true, Metadata.empty()) + }; + + private Dataset unpartitionedDF() { + return spark + .createDataFrame( + ImmutableList.of( + RowFactory.create(1, "John Doe", "hr", "communications"), + RowFactory.create(2, "Jane Doe", "hr", "salary"), + RowFactory.create(3, "Matt Doe", "hr", "communications"), + RowFactory.create(4, "Will Doe", "facilities", "all")), + new StructType(STRUCT)) + .repartition(1); + } + + private Dataset singleNullRecordDF() { + return spark + .createDataFrame( + ImmutableList.of(RowFactory.create(null, null, null, null)), new StructType(STRUCT)) + .repartition(1); + } + + private Dataset partitionedDF() { + return unpartitionedDF().select("name", "dept", "subdept", "id"); + } + + private Dataset compositePartitionedDF() { + return unpartitionedDF().select("name", "subdept", "id", "dept"); + } + + private Dataset compositePartitionedNullRecordDF() { + return singleNullRecordDF().select("name", "subdept", "id", "dept"); + } + + private Dataset weirdColumnNamesDF() { + Dataset unpartitionedDF = unpartitionedDF(); + return unpartitionedDF.select( + unpartitionedDF.col("id"), + unpartitionedDF.col("subdept"), + unpartitionedDF.col("dept"), + unpartitionedDF.col("name").as("naMe")); + } + + private static final StructField[] DATE_STRUCT = { + new StructField("id", DataTypes.IntegerType, true, Metadata.empty()), + new StructField("name", DataTypes.StringType, true, Metadata.empty()), + new StructField("ts", DataTypes.DateType, true, Metadata.empty()), + new StructField("dept", DataTypes.StringType, true, Metadata.empty()), + }; + + private static java.sql.Date toDate(String value) { + return new java.sql.Date(DateTime.parse(value).getMillis()); + } + + private Dataset dateDF() { + return spark + .createDataFrame( + ImmutableList.of( + RowFactory.create(1, "John Doe", toDate("2021-01-01"), "01"), + RowFactory.create(2, "Jane Doe", toDate("2021-01-01"), "01"), + RowFactory.create(3, "Matt Doe", toDate("2021-01-02"), "02"), + RowFactory.create(4, "Will Doe", toDate("2021-01-02"), "02")), + new StructType(DATE_STRUCT)) + .repartition(2); + } + + private void createUnpartitionedFileTable(String format) { + String createParquet = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING %s LOCATION '%s'"; + + sql(createParquet, sourceTableName, format, fileTableDir.getAbsolutePath()); + Dataset df = unpartitionedDF(); + df.write().insertInto(sourceTableName); + df.write().insertInto(sourceTableName); + } + + private void createPartitionedFileTable(String format) { + String createParquet = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING %s PARTITIONED BY (id) " + + "LOCATION '%s'"; + + sql(createParquet, sourceTableName, format, fileTableDir.getAbsolutePath()); + + Dataset df = partitionedDF(); + df.write().insertInto(sourceTableName); + df.write().insertInto(sourceTableName); + } + + private void createCompositePartitionedTable(String format) { + String createParquet = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING %s " + + "PARTITIONED BY (id, dept) LOCATION '%s'"; + sql(createParquet, sourceTableName, format, fileTableDir.getAbsolutePath()); + + Dataset df = compositePartitionedDF(); + df.write().insertInto(sourceTableName); + df.write().insertInto(sourceTableName); + } + + private void createCompositePartitionedTableWithNullValueInPartitionColumn(String format) { + String createParquet = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING %s " + + "PARTITIONED BY (id, dept) LOCATION '%s'"; + sql(createParquet, sourceTableName, format, fileTableDir.getAbsolutePath()); + + Dataset unionedDF = + compositePartitionedDF() + .unionAll(compositePartitionedNullRecordDF()) + .select("name", "subdept", "id", "dept") + .repartition(1); + + unionedDF.write().insertInto(sourceTableName); + unionedDF.write().insertInto(sourceTableName); + } + + private void createPartitionedTableWithNullValueInPartitionColumnOnDept(String format) { + String createParquet = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING %s " + + "PARTITIONED BY (dept) LOCATION '%s'"; + sql(createParquet, sourceTableName, format, fileTableDir.getAbsolutePath()); + + Dataset unionedDF = + unpartitionedDF() + .select("id", "name", "subdept", "dept") + .unionAll(singleNullRecordDF().select("id", "name", "subdept", "dept")) + .repartition(1); + + unionedDF.write().insertInto(sourceTableName); + unionedDF.write().insertInto(sourceTableName); + } + + private void createPartitionedTableWithNullValueInPartitionColumnOnId(String format) { + String createParquet = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING %s " + + "PARTITIONED BY (id) LOCATION '%s'"; + sql(createParquet, sourceTableName, format, fileTableDir.getAbsolutePath()); + + Dataset unionedDF = + unpartitionedDF() + .select("name", "subdept", "dept", "id") + .unionAll(singleNullRecordDF().select("name", "subdept", "dept", "id")) + .repartition(1); + + unionedDF.write().insertInto(sourceTableName); + unionedDF.write().insertInto(sourceTableName); + } + + private void createWeirdCaseTable() { + String createParquet = + "CREATE TABLE %s (id Integer, subdept String, dept String) " + + "PARTITIONED BY (`naMe` String) STORED AS parquet"; + + sql(createParquet, sourceTableName); + + Dataset df = weirdColumnNamesDF(); + df.write().insertInto(sourceTableName); + df.write().insertInto(sourceTableName); + } + + private void createUnpartitionedHiveTable() { + String createHive = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) STORED AS parquet"; + + sql(createHive, sourceTableName); + + Dataset df = unpartitionedDF(); + df.write().insertInto(sourceTableName); + df.write().insertInto(sourceTableName); + } + + private void createPartitionedHiveTable() { + String createHive = + "CREATE TABLE %s (name String, dept String, subdept String) " + + "PARTITIONED BY (id Integer) STORED AS parquet"; + + sql(createHive, sourceTableName); + + Dataset df = partitionedDF(); + df.write().insertInto(sourceTableName); + df.write().insertInto(sourceTableName); + } + + private void createDatePartitionedFileTable(String format) { + String createParquet = + "CREATE TABLE %s (id Integer, name String, date Date) USING %s " + + "PARTITIONED BY (date) LOCATION '%s'"; + + sql(createParquet, sourceTableName, format, fileTableDir.getAbsolutePath()); + + dateDF().select("id", "name", "ts").write().insertInto(sourceTableName); + } + + private void createTableWithTwoPartitions(String format) { + String createParquet = + "CREATE TABLE %s (id Integer, name String, date Date, dept String) USING %s " + + "PARTITIONED BY (date, dept) LOCATION '%s'"; + + sql(createParquet, sourceTableName, format, fileTableDir.getAbsolutePath()); + + dateDF().write().insertInto(sourceTableName); + } + + private void createIcebergTable(String schema) { + createIcebergTable(schema, ""); + } + + private void createIcebergTable(String schema, String partitioning) { + sql( + "CREATE TABLE %s (%s) USING iceberg %s TBLPROPERTIES ('%s' '%d')", + tableName, schema, partitioning, TableProperties.FORMAT_VERSION, formatVersion); + } + + private void assertOutput( + List result, long expectedAddedFilesCount, long expectedChangedPartitionCount) { + Object[] output = Iterables.getOnlyElement(result); + assertThat(output[0]).isEqualTo(expectedAddedFilesCount); + if (formatVersion == 1) { + assertThat(output[1]).isEqualTo(expectedChangedPartitionCount); + } else { + // the number of changed partitions may not be populated in v2 tables + assertThat(output[1]).isIn(expectedChangedPartitionCount, null); + } + } + + private void manifestSpecMatchesTableSpec() throws NoSuchTableException, ParseException { + Table table = Spark3Util.loadIcebergTable(spark, tableName); + manifestSpecMatchesGivenSpec(table, table.spec()); + } + + private void manifestSpecMatchesGivenSpec(Table table, PartitionSpec partitionSpec) { + FileIO io = ((HasTableOperations) table).operations().io(); + // Check that the manifests have the correct partition spec + assertThat( + table.currentSnapshot().allManifests(io).stream() + .map(mf -> ManifestFiles.read(mf, io, null /* force reading spec from file*/)) + .map(ManifestReader::spec) + .collect(Collectors.toList())) + .allSatisfy(spec -> assertThat(spec).isEqualTo(partitionSpec)); + } + + private void verifyUUIDInPath() { + // verify manifest file name has uuid pattern + String manifestPath = (String) sql("select path from %s.manifests", tableName).get(0)[0]; + + Pattern uuidPattern = Pattern.compile("[a-f0-9]{8}(?:-[a-f0-9]{4}){4}[a-f0-9]{8}"); + + Matcher matcher = uuidPattern.matcher(manifestPath); + assertThat(matcher.find()).as("verify manifest path has uuid").isTrue(); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java new file mode 100644 index 000000000000..296564e20d4a --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java @@ -0,0 +1,650 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.List; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.source.SparkTable; +import org.apache.spark.sql.connector.catalog.CatalogManager; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestAlterTablePartitionFields extends ExtensionsTestBase { + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, formatVersion = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties(), + 1 + }, + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties(), + 2 + } + }; + } + + @Parameter(index = 3) + private int formatVersion; + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testAddIdentityPartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD category", tableName); + + table.refresh(); + + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).identity("category").build(); + + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } + + @TestTemplate + public void testAddBucketPartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, id)", tableName); + + table.refresh(); + + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()) + .withSpecId(1) + .bucket("id", 16, "id_bucket_16") + .build(); + + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } + + @TestTemplate + public void testAddTruncatePartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD truncate(data, 4)", tableName); + + table.refresh(); + + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()) + .withSpecId(1) + .truncate("data", 4, "data_trunc_4") + .build(); + + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } + + @TestTemplate + public void testAddYearsPartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD years(ts)", tableName); + + table.refresh(); + + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).year("ts").build(); + + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } + + @TestTemplate + public void testAddMonthsPartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD months(ts)", tableName); + + table.refresh(); + + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).month("ts").build(); + + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } + + @TestTemplate + public void testAddDaysPartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD days(ts)", tableName); + + table.refresh(); + + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).day("ts").build(); + + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } + + @TestTemplate + public void testAddHoursPartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD hours(ts)", tableName); + + table.refresh(); + + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).hour("ts").build(); + + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } + + @TestTemplate + public void testAddYearPartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD year(ts)", tableName); + + table.refresh(); + + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).year("ts").build(); + + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } + + @TestTemplate + public void testAddMonthPartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD month(ts)", tableName); + + table.refresh(); + + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).month("ts").build(); + + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } + + @TestTemplate + public void testAddDayPartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD day(ts)", tableName); + + table.refresh(); + + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).day("ts").build(); + + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } + + @TestTemplate + public void testAddHourPartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD hour(ts)", tableName); + + table.refresh(); + + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).hour("ts").build(); + + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } + + @TestTemplate + public void testAddNamedPartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, id) AS shard", tableName); + + table.refresh(); + + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).bucket("id", 16, "shard").build(); + + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } + + @TestTemplate + public void testDropIdentityPartition() { + createTable("id bigint NOT NULL, category string, data string", "category"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().fields()).as("Table should start with 1 partition field").hasSize(1); + + sql("ALTER TABLE %s DROP PARTITION FIELD category", tableName); + + table.refresh(); + + if (formatVersion == 1) { + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()) + .withSpecId(1) + .alwaysNull("category", "category") + .build(); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } else { + assertThat(table.spec().isUnpartitioned()).as("New spec must be unpartitioned").isTrue(); + } + } + + @TestTemplate + public void testDropDaysPartition() { + createTable("id bigint NOT NULL, ts timestamp, data string", "days(ts)"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().fields()).as("Table should start with 1 partition field").hasSize(1); + + sql("ALTER TABLE %s DROP PARTITION FIELD days(ts)", tableName); + + table.refresh(); + + if (formatVersion == 1) { + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).alwaysNull("ts", "ts_day").build(); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } else { + assertThat(table.spec().isUnpartitioned()).as("New spec must be unpartitioned").isTrue(); + } + } + + @TestTemplate + public void testDropBucketPartition() { + createTable("id bigint NOT NULL, data string", "bucket(16, id)"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().fields()).as("Table should start with 1 partition field").hasSize(1); + + sql("ALTER TABLE %s DROP PARTITION FIELD bucket(16, id)", tableName); + + table.refresh(); + + if (formatVersion == 1) { + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()) + .withSpecId(1) + .alwaysNull("id", "id_bucket") + .build(); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } else { + assertThat(table.spec().isUnpartitioned()).as("New spec must be unpartitioned").isTrue(); + } + } + + @TestTemplate + public void testDropPartitionByName() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, id) AS shard", tableName); + + table.refresh(); + + assertThat(table.spec().fields()).as("Table should have 1 partition field").hasSize(1); + + // Should be recognized as iceberg command even with extra white spaces + sql("ALTER TABLE %s DROP PARTITION \n FIELD shard", tableName); + + table.refresh(); + + if (formatVersion == 1) { + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(2).alwaysNull("id", "shard").build(); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + } else { + assertThat(table.spec().isUnpartitioned()).as("New spec must be unpartitioned").isTrue(); + } + } + + @TestTemplate + public void testReplacePartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD days(ts)", tableName); + table.refresh(); + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).day("ts").build(); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + + sql("ALTER TABLE %s REPLACE PARTITION FIELD days(ts) WITH hours(ts)", tableName); + table.refresh(); + if (formatVersion == 1) { + expected = + PartitionSpec.builderFor(table.schema()) + .withSpecId(2) + .alwaysNull("ts", "ts_day") + .hour("ts") + .build(); + } else { + expected = + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "ts_hour") + .build(); + } + assertThat(table.spec()) + .as("Should changed from daily to hourly partitioned field") + .isEqualTo(expected); + } + + @TestTemplate + public void testReplacePartitionAndRename() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD days(ts)", tableName); + table.refresh(); + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).day("ts").build(); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + + sql("ALTER TABLE %s REPLACE PARTITION FIELD days(ts) WITH hours(ts) AS hour_col", tableName); + table.refresh(); + if (formatVersion == 1) { + expected = + PartitionSpec.builderFor(table.schema()) + .withSpecId(2) + .alwaysNull("ts", "ts_day") + .hour("ts", "hour_col") + .build(); + } else { + expected = + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "hour_col") + .build(); + } + assertThat(table.spec()) + .as("Should changed from daily to hourly partitioned field") + .isEqualTo(expected); + } + + @TestTemplate + public void testReplaceNamedPartition() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD days(ts) AS day_col", tableName); + table.refresh(); + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).day("ts", "day_col").build(); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + + sql("ALTER TABLE %s REPLACE PARTITION FIELD day_col WITH hours(ts)", tableName); + table.refresh(); + if (formatVersion == 1) { + expected = + PartitionSpec.builderFor(table.schema()) + .withSpecId(2) + .alwaysNull("ts", "day_col") + .hour("ts") + .build(); + } else { + expected = + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "ts_hour") + .build(); + } + assertThat(table.spec()) + .as("Should changed from daily to hourly partitioned field") + .isEqualTo(expected); + } + + @TestTemplate + public void testReplaceNamedPartitionAndRenameDifferently() { + createTable("id bigint NOT NULL, category string, ts timestamp, data string"); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); + + sql("ALTER TABLE %s ADD PARTITION FIELD days(ts) AS day_col", tableName); + table.refresh(); + PartitionSpec expected = + PartitionSpec.builderFor(table.schema()).withSpecId(1).day("ts", "day_col").build(); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + + sql("ALTER TABLE %s REPLACE PARTITION FIELD day_col WITH hours(ts) AS hour_col", tableName); + table.refresh(); + if (formatVersion == 1) { + expected = + PartitionSpec.builderFor(table.schema()) + .withSpecId(2) + .alwaysNull("ts", "day_col") + .hour("ts", "hour_col") + .build(); + } else { + expected = + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "hour_col") + .build(); + } + assertThat(table.spec()) + .as("Should changed from daily to hourly partitioned field") + .isEqualTo(expected); + } + + @TestTemplate + public void testSparkTableAddDropPartitions() throws Exception { + createTable("id bigint NOT NULL, ts timestamp, data string"); + assertThat(sparkTable().partitioning()).as("spark table partition should be empty").isEmpty(); + + sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, id) AS shard", tableName); + assertPartitioningEquals(sparkTable(), 1, "bucket(16, id)"); + + sql("ALTER TABLE %s ADD PARTITION FIELD truncate(data, 4)", tableName); + assertPartitioningEquals(sparkTable(), 2, "truncate(4, data)"); + + sql("ALTER TABLE %s ADD PARTITION FIELD years(ts)", tableName); + assertPartitioningEquals(sparkTable(), 3, "years(ts)"); + + sql("ALTER TABLE %s DROP PARTITION FIELD years(ts)", tableName); + assertPartitioningEquals(sparkTable(), 2, "truncate(4, data)"); + + sql("ALTER TABLE %s DROP PARTITION FIELD truncate(4, data)", tableName); + assertPartitioningEquals(sparkTable(), 1, "bucket(16, id)"); + + sql("ALTER TABLE %s DROP PARTITION FIELD shard", tableName); + sql("DESCRIBE %s", tableName); + assertThat(sparkTable().partitioning()).as("spark table partition should be empty").isEmpty(); + } + + @TestTemplate + public void testDropColumnOfOldPartitionFieldV1() { + // default table created in v1 format + sql( + "CREATE TABLE %s (id bigint NOT NULL, ts timestamp, day_of_ts date) USING iceberg PARTITIONED BY (day_of_ts) TBLPROPERTIES('format-version' = '1')", + tableName); + + sql("ALTER TABLE %s REPLACE PARTITION FIELD day_of_ts WITH days(ts)", tableName); + + sql("ALTER TABLE %s DROP COLUMN day_of_ts", tableName); + } + + @TestTemplate + public void testDropColumnOfOldPartitionFieldV2() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, ts timestamp, day_of_ts date) USING iceberg PARTITIONED BY (day_of_ts) TBLPROPERTIES('format-version' = '2')", + tableName); + + sql("ALTER TABLE %s REPLACE PARTITION FIELD day_of_ts WITH days(ts)", tableName); + + sql("ALTER TABLE %s DROP COLUMN day_of_ts", tableName); + } + + private void assertPartitioningEquals(SparkTable table, int len, String transform) { + assertThat(table.partitioning()).as("spark table partition should be " + len).hasSize(len); + assertThat(table.partitioning()[len - 1]) + .asString() + .as("latest spark table partition transform should match") + .isEqualTo(transform); + } + + private SparkTable sparkTable() throws Exception { + validationCatalog.loadTable(tableIdent).refresh(); + CatalogManager catalogManager = spark.sessionState().catalogManager(); + TableCatalog catalog = (TableCatalog) catalogManager.catalog(catalogName); + Identifier identifier = Identifier.of(tableIdent.namespace().levels(), tableIdent.name()); + return (SparkTable) catalog.loadTable(identifier); + } + + private void createTable(String schema) { + createTable(schema, null); + } + + private void createTable(String schema, String spec) { + if (spec == null) { + sql( + "CREATE TABLE %s (%s) USING iceberg TBLPROPERTIES ('%s' '%d')", + tableName, schema, TableProperties.FORMAT_VERSION, formatVersion); + } else { + sql( + "CREATE TABLE %s (%s) USING iceberg PARTITIONED BY (%s) TBLPROPERTIES ('%s' '%d')", + tableName, schema, spec, TableProperties.FORMAT_VERSION, formatVersion); + } + } + + private void runCreateAndDropPartitionField( + String column, String partitionType, List expected, String predicate) { + sql("DROP TABLE IF EXISTS %s", tableName); + sql( + "CREATE TABLE %s (col_int INTEGER, col_ts TIMESTAMP_NTZ, col_long BIGINT) USING ICEBERG TBLPROPERTIES ('format-version' = %d)", + tableName, formatVersion); + sql("INSERT INTO %s VALUES (1000, CAST('2024-03-01 19:25:00' as TIMESTAMP), 2100)", tableName); + sql("ALTER TABLE %s ADD PARTITION FIELD %s AS col2_partition", tableName, partitionType); + sql("INSERT INTO %s VALUES (2000, CAST('2024-04-01 19:25:00' as TIMESTAMP), 2200)", tableName); + sql("ALTER TABLE %s DROP PARTITION FIELD col2_partition", tableName); + sql("INSERT INTO %s VALUES (3000, CAST('2024-05-01 19:25:00' as TIMESTAMP), 2300)", tableName); + sql("ALTER TABLE %s DROP COLUMN %s", tableName, column); + + assertThat(sql("SELECT * FROM %s WHERE %s ORDER BY col_int", tableName, predicate)) + .containsExactlyElementsOf(expected); + } + + @TestTemplate + public void testDropPartitionAndSourceColumnLong() { + String predicateTs = "col_long >= 2200"; + List expectedTs = + Lists.newArrayList(new Object[] {2000, 2200L}, new Object[] {3000, 2300L}); + runCreateAndDropPartitionField("col_ts", "col_ts", expectedTs, predicateTs); + runCreateAndDropPartitionField("col_ts", "year(col_ts)", expectedTs, predicateTs); + runCreateAndDropPartitionField("col_ts", "month(col_ts)", expectedTs, predicateTs); + runCreateAndDropPartitionField("col_ts", "day(col_ts)", expectedTs, predicateTs); + } + + @TestTemplate + public void testDropPartitionAndSourceColumnTimestamp() { + String predicate = "col_ts >= '2024-04-01 19:25:00'"; + List expected = + Lists.newArrayList( + new Object[] {2000, LocalDateTime.ofEpochSecond(1711999500, 0, ZoneOffset.UTC)}, + new Object[] {3000, LocalDateTime.ofEpochSecond(1714591500, 0, ZoneOffset.UTC)}); + runCreateAndDropPartitionField("col_long", "col_long", expected, predicate); + runCreateAndDropPartitionField("col_long", "truncate(2, col_long)", expected, predicate); + runCreateAndDropPartitionField("col_long", "bucket(16, col_long)", expected, predicate); + } + + @TestTemplate + public void deleteAfterDroppingPartitionAndSourceColumn() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql( + "CREATE TABLE %s (id INTEGER, data STRING) USING ICEBERG TBLPROPERTIES ('format-version' = %d)", + tableName, formatVersion); + sql("INSERT INTO %s VALUES (1, 'data1')", tableName); + sql("ALTER TABLE %s ADD PARTITION FIELD data", tableName); + sql("INSERT INTO %s VALUES (2, 'data2')", tableName); + sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName); + sql("INSERT INTO %s VALUES (3, 'data3')", tableName); + sql("ALTER TABLE %s DROP COLUMN data", tableName); + + assertThat(sql("SELECT * FROM %s WHERE id >= 1 ORDER BY id", tableName)) + .containsExactly(row(1), row(2), row(3)); + + sql("DELETE FROM %s WHERE id >= 1", tableName); + assertThat(sql("SELECT * FROM %s WHERE id >= 1", tableName)).isEmpty(); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java new file mode 100644 index 000000000000..f36a2e4470e3 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java @@ -0,0 +1,148 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestAlterTableSchema extends ExtensionsTestBase { + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testSetIdentifierFields() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, " + + "location struct NOT NULL) USING iceberg", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.schema().identifierFieldIds()) + .as("Table should start without identifier") + .isEmpty(); + + sql("ALTER TABLE %s SET IDENTIFIER FIELDS id", tableName); + table.refresh(); + assertThat(table.schema().identifierFieldIds()) + .containsExactly(table.schema().findField("id").fieldId()); + + sql("ALTER TABLE %s SET IDENTIFIER FIELDS id, location.lon", tableName); + table.refresh(); + assertThat(table.schema().identifierFieldIds()) + .as("Should have new identifier field") + .containsExactlyInAnyOrder( + table.schema().findField("id").fieldId(), + table.schema().findField("location.lon").fieldId()); + + sql("ALTER TABLE %s SET IDENTIFIER FIELDS location.lon", tableName); + table.refresh(); + assertThat(table.schema().identifierFieldIds()) + .as("Should have new identifier field") + .containsExactly(table.schema().findField("location.lon").fieldId()); + } + + @TestTemplate + public void testSetInvalidIdentifierFields() { + sql("CREATE TABLE %s (id bigint NOT NULL, id2 bigint) USING iceberg", tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.schema().identifierFieldIds()) + .as("Table should start without identifier") + .isEmpty(); + assertThatThrownBy(() -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS unknown", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageEndingWith("not found in current schema or added columns"); + + assertThatThrownBy(() -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS id2", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageEndingWith("not a required field"); + } + + @TestTemplate + public void testDropIdentifierFields() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, " + + "location struct NOT NULL) USING iceberg", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.schema().identifierFieldIds()) + .as("Table should start without identifier") + .isEmpty(); + + sql("ALTER TABLE %s SET IDENTIFIER FIELDS id, location.lon", tableName); + table.refresh(); + assertThat(table.schema().identifierFieldIds()) + .as("Should have new identifier fields") + .containsExactlyInAnyOrder( + table.schema().findField("id").fieldId(), + table.schema().findField("location.lon").fieldId()); + + sql("ALTER TABLE %s DROP IDENTIFIER FIELDS id", tableName); + table.refresh(); + assertThat(table.schema().identifierFieldIds()) + .as("Should removed identifier field") + .containsExactly(table.schema().findField("location.lon").fieldId()); + + sql("ALTER TABLE %s SET IDENTIFIER FIELDS id, location.lon", tableName); + table.refresh(); + assertThat(table.schema().identifierFieldIds()) + .as("Should have new identifier fields") + .containsExactlyInAnyOrder( + table.schema().findField("id").fieldId(), + table.schema().findField("location.lon").fieldId()); + + sql("ALTER TABLE %s DROP IDENTIFIER FIELDS id, location.lon", tableName); + table.refresh(); + assertThat(table.schema().identifierFieldIds()).as("Should have no identifier field").isEmpty(); + } + + @TestTemplate + public void testDropInvalidIdentifierFields() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string NOT NULL, " + + "location struct NOT NULL) USING iceberg", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.schema().identifierFieldIds()) + .as("Table should start without identifier") + .isEmpty(); + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS unknown", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot complete drop identifier fields operation: field unknown not found"); + + sql("ALTER TABLE %s SET IDENTIFIER FIELDS id", tableName); + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS data", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot complete drop identifier fields operation: data is not an identifier field"); + + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS location.lon", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot complete drop identifier fields operation: location.lon is not an identifier field"); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java new file mode 100644 index 000000000000..2408f3cfa2c0 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java @@ -0,0 +1,163 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.spark.sql.AnalysisException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestAncestorsOfProcedure extends ExtensionsTestBase { + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testAncestorOfUsingEmptyArgs() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Long currentSnapshotId = table.currentSnapshot().snapshotId(); + Long currentTimestamp = table.currentSnapshot().timestampMillis(); + Long preSnapshotId = table.currentSnapshot().parentId(); + Long preTimeStamp = table.snapshot(table.currentSnapshot().parentId()).timestampMillis(); + + List output = sql("CALL %s.system.ancestors_of('%s')", catalogName, tableIdent); + + assertEquals( + "Procedure output must match", + ImmutableList.of( + row(currentSnapshotId, currentTimestamp), row(preSnapshotId, preTimeStamp)), + output); + } + + @TestTemplate + public void testAncestorOfUsingSnapshotId() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Long currentSnapshotId = table.currentSnapshot().snapshotId(); + Long currentTimestamp = table.currentSnapshot().timestampMillis(); + Long preSnapshotId = table.currentSnapshot().parentId(); + Long preTimeStamp = table.snapshot(table.currentSnapshot().parentId()).timestampMillis(); + + assertEquals( + "Procedure output must match", + ImmutableList.of( + row(currentSnapshotId, currentTimestamp), row(preSnapshotId, preTimeStamp)), + sql("CALL %s.system.ancestors_of('%s', %dL)", catalogName, tableIdent, currentSnapshotId)); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(preSnapshotId, preTimeStamp)), + sql("CALL %s.system.ancestors_of('%s', %dL)", catalogName, tableIdent, preSnapshotId)); + } + + @TestTemplate + public void testAncestorOfWithRollBack() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + Table table = validationCatalog.loadTable(tableIdent); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + table.refresh(); + Long firstSnapshotId = table.currentSnapshot().snapshotId(); + Long firstTimestamp = table.currentSnapshot().timestampMillis(); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + table.refresh(); + Long secondSnapshotId = table.currentSnapshot().snapshotId(); + Long secondTimestamp = table.currentSnapshot().timestampMillis(); + sql("INSERT INTO TABLE %s VALUES (3, 'c')", tableName); + table.refresh(); + Long thirdSnapshotId = table.currentSnapshot().snapshotId(); + Long thirdTimestamp = table.currentSnapshot().timestampMillis(); + + // roll back + sql( + "CALL %s.system.rollback_to_snapshot('%s', %dL)", + catalogName, tableIdent, secondSnapshotId); + + sql("INSERT INTO TABLE %s VALUES (4, 'd')", tableName); + table.refresh(); + Long fourthSnapshotId = table.currentSnapshot().snapshotId(); + Long fourthTimestamp = table.currentSnapshot().timestampMillis(); + + assertEquals( + "Procedure output must match", + ImmutableList.of( + row(fourthSnapshotId, fourthTimestamp), + row(secondSnapshotId, secondTimestamp), + row(firstSnapshotId, firstTimestamp)), + sql("CALL %s.system.ancestors_of('%s', %dL)", catalogName, tableIdent, fourthSnapshotId)); + + assertEquals( + "Procedure output must match", + ImmutableList.of( + row(thirdSnapshotId, thirdTimestamp), + row(secondSnapshotId, secondTimestamp), + row(firstSnapshotId, firstTimestamp)), + sql("CALL %s.system.ancestors_of('%s', %dL)", catalogName, tableIdent, thirdSnapshotId)); + } + + @TestTemplate + public void testAncestorOfUsingNamedArgs() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Long firstSnapshotId = table.currentSnapshot().snapshotId(); + Long firstTimestamp = table.currentSnapshot().timestampMillis(); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(firstSnapshotId, firstTimestamp)), + sql( + "CALL %s.system.ancestors_of(snapshot_id => %dL, table => '%s')", + catalogName, firstSnapshotId, tableIdent)); + } + + @TestTemplate + public void testInvalidAncestorOfCases() { + assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of()", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `ancestors_of` because the parameter named `table` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of('')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for parameter 'table'"); + + assertThatThrownBy( + () -> sql("CALL %s.system.ancestors_of('%s', '1.1')", catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "[CAST_INVALID_INPUT] The value '1.1' of the type \"STRING\" cannot be cast to \"BIGINT\" because it is malformed."); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java new file mode 100644 index 000000000000..c49def204f7e --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestBranchDDL.java @@ -0,0 +1,432 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.extensions.IcebergParseException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestBranchDDL extends ExtensionsTestBase { + + @BeforeEach + public void createTable() { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + } + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties() + } + }; + } + + @TestTemplate + public void testCreateBranch() throws NoSuchTableException { + Table table = insertRows(); + long snapshotId = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + Integer minSnapshotsToKeep = 2; + long maxSnapshotAge = 2L; + long maxRefAge = 10L; + sql( + "ALTER TABLE %s CREATE BRANCH %s AS OF VERSION %d RETAIN %d DAYS WITH SNAPSHOT RETENTION %d SNAPSHOTS %d days", + tableName, branchName, snapshotId, maxRefAge, minSnapshotsToKeep, maxSnapshotAge); + table.refresh(); + assertThat(table.refs()) + .hasEntrySatisfying( + branchName, + ref -> { + assertThat(ref.snapshotId()).isEqualTo(table.currentSnapshot().snapshotId()); + assertThat(ref.minSnapshotsToKeep()).isEqualTo(minSnapshotsToKeep); + assertThat(ref.maxSnapshotAgeMs().longValue()) + .isEqualTo(TimeUnit.DAYS.toMillis(maxSnapshotAge)); + assertThat(ref.maxRefAgeMs().longValue()) + .isEqualTo(TimeUnit.DAYS.toMillis(maxRefAge)); + }); + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branchName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref b1 already exists"); + } + + @TestTemplate + public void testCreateBranchOnEmptyTable() { + String branchName = "b1"; + sql("ALTER TABLE %s CREATE BRANCH %s", tableName, "b1"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.refs()) + .doesNotContainKey(SnapshotRef.MAIN_BRANCH) + .hasEntrySatisfying( + branchName, + ref -> { + assertThat(ref.minSnapshotsToKeep()).isNull(); + assertThat(ref.maxSnapshotAgeMs()).isNull(); + assertThat(ref.maxRefAgeMs()).isNull(); + + Snapshot snapshot = table.snapshot(ref.snapshotId()); + assertThat(snapshot.parentId()).isNull(); + assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + }); + } + + @TestTemplate + public void testCreateBranchUseDefaultConfig() throws NoSuchTableException { + Table table = insertRows(); + String branchName = "b1"; + sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branchName); + table.refresh(); + assertThat(table.refs()) + .hasEntrySatisfying( + branchName, + ref -> { + assertThat(ref.snapshotId()).isEqualTo(table.currentSnapshot().snapshotId()); + assertThat(ref.minSnapshotsToKeep()).isNull(); + assertThat(ref.maxSnapshotAgeMs()).isNull(); + assertThat(ref.maxRefAgeMs()).isNull(); + }); + } + + @TestTemplate + public void testCreateBranchUseCustomMinSnapshotsToKeep() throws NoSuchTableException { + Integer minSnapshotsToKeep = 2; + Table table = insertRows(); + String branchName = "b1"; + sql( + "ALTER TABLE %s CREATE BRANCH %s WITH SNAPSHOT RETENTION %d SNAPSHOTS", + tableName, branchName, minSnapshotsToKeep); + table.refresh(); + assertThat(table.refs()) + .hasEntrySatisfying( + branchName, + ref -> { + assertThat(ref.snapshotId()).isEqualTo(table.currentSnapshot().snapshotId()); + assertThat(ref.minSnapshotsToKeep()).isEqualTo(minSnapshotsToKeep); + assertThat(ref.maxSnapshotAgeMs()).isNull(); + assertThat(ref.maxRefAgeMs()).isNull(); + }); + } + + @TestTemplate + public void testCreateBranchUseCustomMaxSnapshotAge() throws NoSuchTableException { + long maxSnapshotAge = 2L; + Table table = insertRows(); + String branchName = "b1"; + sql( + "ALTER TABLE %s CREATE BRANCH %s WITH SNAPSHOT RETENTION %d DAYS", + tableName, branchName, maxSnapshotAge); + table.refresh(); + assertThat(table.refs()) + .hasEntrySatisfying( + branchName, + ref -> { + assertThat(ref).isNotNull(); + assertThat(ref.minSnapshotsToKeep()).isNull(); + assertThat(ref.maxSnapshotAgeMs().longValue()) + .isEqualTo(TimeUnit.DAYS.toMillis(maxSnapshotAge)); + assertThat(ref.maxRefAgeMs()).isNull(); + }); + } + + @TestTemplate + public void testCreateBranchIfNotExists() throws NoSuchTableException { + long maxSnapshotAge = 2L; + Table table = insertRows(); + String branchName = "b1"; + sql( + "ALTER TABLE %s CREATE BRANCH %s WITH SNAPSHOT RETENTION %d DAYS", + tableName, branchName, maxSnapshotAge); + sql("ALTER TABLE %s CREATE BRANCH IF NOT EXISTS %s", tableName, branchName); + + table.refresh(); + assertThat(table.refs()) + .hasEntrySatisfying( + branchName, + ref -> { + assertThat(ref.snapshotId()).isEqualTo(table.currentSnapshot().snapshotId()); + assertThat(ref.minSnapshotsToKeep()).isNull(); + assertThat(ref.maxSnapshotAgeMs().longValue()) + .isEqualTo(TimeUnit.DAYS.toMillis(maxSnapshotAge)); + assertThat(ref.maxRefAgeMs()).isNull(); + }); + } + + @TestTemplate + public void testCreateBranchUseCustomMinSnapshotsToKeepAndMaxSnapshotAge() + throws NoSuchTableException { + Integer minSnapshotsToKeep = 2; + long maxSnapshotAge = 2L; + Table table = insertRows(); + String branchName = "b1"; + sql( + "ALTER TABLE %s CREATE BRANCH %s WITH SNAPSHOT RETENTION %d SNAPSHOTS %d DAYS", + tableName, branchName, minSnapshotsToKeep, maxSnapshotAge); + table.refresh(); + assertThat(table.refs()) + .hasEntrySatisfying( + branchName, + ref -> { + assertThat(ref.snapshotId()).isEqualTo(table.currentSnapshot().snapshotId()); + assertThat(ref.minSnapshotsToKeep()).isEqualTo(minSnapshotsToKeep); + assertThat(ref.maxSnapshotAgeMs().longValue()) + .isEqualTo(TimeUnit.DAYS.toMillis(maxSnapshotAge)); + assertThat(ref.maxRefAgeMs()).isNull(); + }); + + assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s CREATE BRANCH %s WITH SNAPSHOT RETENTION", + tableName, branchName)) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("no viable alternative at input 'WITH SNAPSHOT RETENTION'"); + } + + @TestTemplate + public void testCreateBranchUseCustomMaxRefAge() throws NoSuchTableException { + long maxRefAge = 10L; + Table table = insertRows(); + String branchName = "b1"; + sql("ALTER TABLE %s CREATE BRANCH %s RETAIN %d DAYS", tableName, branchName, maxRefAge); + table.refresh(); + assertThat(table.refs()) + .hasEntrySatisfying( + branchName, + ref -> { + assertThat(ref.snapshotId()).isEqualTo(table.currentSnapshot().snapshotId()); + assertThat(ref.minSnapshotsToKeep()).isNull(); + assertThat(ref.maxSnapshotAgeMs()).isNull(); + assertThat(ref.maxRefAgeMs().longValue()) + .isEqualTo(TimeUnit.DAYS.toMillis(maxRefAge)); + }); + + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("no viable alternative at input"); + + assertThatThrownBy( + () -> + sql("ALTER TABLE %s CREATE BRANCH %s RETAIN %s DAYS", tableName, branchName, "abc")) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("no viable alternative at input"); + + assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s CREATE BRANCH %s RETAIN %d SECONDS", + tableName, branchName, maxRefAge)) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input 'SECONDS' expecting {'DAYS', 'HOURS', 'MINUTES'}"); + } + + @TestTemplate + public void testDropBranch() throws NoSuchTableException { + insertRows(); + + Table table = validationCatalog.loadTable(tableIdent); + String branchName = "b1"; + table.manageSnapshots().createBranch(branchName, table.currentSnapshot().snapshotId()).commit(); + assertThat(table.refs()) + .hasEntrySatisfying( + branchName, + ref -> { + assertThat(ref.snapshotId()).isEqualTo(table.currentSnapshot().snapshotId()); + }); + + sql("ALTER TABLE %s DROP BRANCH %s", tableName, branchName); + table.refresh(); + + assertThat(table.refs()).doesNotContainKey(branchName); + } + + @TestTemplate + public void testDropBranchDoesNotExist() { + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "nonExistingBranch")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Branch does not exist: nonExistingBranch"); + } + + @TestTemplate + public void testDropBranchFailsForTag() throws NoSuchTableException { + String tagName = "b1"; + Table table = insertRows(); + table.manageSnapshots().createTag(tagName, table.currentSnapshot().snapshotId()).commit(); + + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, tagName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref b1 is a tag not a branch"); + } + + @TestTemplate + public void testDropBranchNonConformingName() { + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("no viable alternative at input '123'"); + } + + @TestTemplate + public void testDropMainBranchFails() { + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH main", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot remove main branch"); + } + + @TestTemplate + public void testDropBranchIfExists() { + String branchName = "nonExistingBranch"; + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.refs()).doesNotContainKey(branchName); + + sql("ALTER TABLE %s DROP BRANCH IF EXISTS %s", tableName, branchName); + table.refresh(); + assertThat(table.refs()).doesNotContainKey(branchName); + } + + private Table insertRows() throws NoSuchTableException { + List records = + ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.writeTo(tableName).append(); + return validationCatalog.loadTable(tableIdent); + } + + @TestTemplate + public void createOrReplace() throws NoSuchTableException { + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + insertRows(); + long second = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createBranch(branchName, second).commit(); + + sql( + "ALTER TABLE %s CREATE OR REPLACE BRANCH %s AS OF VERSION %d", + tableName, branchName, first); + table.refresh(); + assertThat(table.refs().get(branchName).snapshotId()).isEqualTo(second); + } + + @TestTemplate + public void testCreateOrReplaceBranchOnEmptyTable() { + String branchName = "b1"; + sql("ALTER TABLE %s CREATE OR REPLACE BRANCH %s", tableName, "b1"); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.refs()) + .doesNotContainKey(SnapshotRef.MAIN_BRANCH) + .hasEntrySatisfying( + branchName, + ref -> { + assertThat(ref).isNotNull(); + assertThat(ref.minSnapshotsToKeep()).isNull(); + assertThat(ref.maxSnapshotAgeMs()).isNull(); + assertThat(ref.maxRefAgeMs()).isNull(); + + Snapshot snapshot = table.snapshot(ref.snapshotId()); + assertThat(snapshot.parentId()).isNull(); + assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + }); + } + + @TestTemplate + public void createOrReplaceWithNonExistingBranch() throws NoSuchTableException { + Table table = insertRows(); + String branchName = "b1"; + insertRows(); + long snapshotId = table.currentSnapshot().snapshotId(); + + sql( + "ALTER TABLE %s CREATE OR REPLACE BRANCH %s AS OF VERSION %d", + tableName, branchName, snapshotId); + table.refresh(); + assertThat(table.refs().get(branchName).snapshotId()).isEqualTo(snapshotId); + } + + @TestTemplate + public void replaceBranch() throws NoSuchTableException { + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + long expectedMaxRefAgeMs = 1000; + table + .manageSnapshots() + .createBranch(branchName, first) + .setMaxRefAgeMs(branchName, expectedMaxRefAgeMs) + .commit(); + + insertRows(); + long second = table.currentSnapshot().snapshotId(); + + sql("ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", tableName, branchName, second); + table.refresh(); + assertThat(table.refs()) + .hasEntrySatisfying( + branchName, + ref -> { + assertThat(ref.snapshotId()).isEqualTo(second); + assertThat(ref.maxRefAgeMs()).isEqualTo(expectedMaxRefAgeMs); + }); + } + + @TestTemplate + public void replaceBranchDoesNotExist() throws NoSuchTableException { + Table table = insertRows(); + + assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", + tableName, "someBranch", table.currentSnapshot().snapshotId())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Branch does not exist: someBranch"); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java new file mode 100644 index 000000000000..742235935eb2 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java @@ -0,0 +1,453 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.TableProperties.FORMAT_VERSION; +import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; +import static org.apache.iceberg.TableProperties.MANIFEST_MIN_MERGE_COUNT; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.source.SparkChangelogTable; +import org.apache.spark.sql.DataFrameReader; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestChangelogTable extends ExtensionsTestBase { + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, formatVersion = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties(), + 1 + }, + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties(), + 2 + } + }; + } + + @Parameter(index = 3) + private int formatVersion; + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testDataFilters() { + createTableWithDefaultRows(); + + sql("INSERT INTO %s VALUES (3, 'c')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + Snapshot snap3 = table.currentSnapshot(); + + sql("DELETE FROM %s WHERE id = 3", tableName); + + table.refresh(); + + Snapshot snap4 = table.currentSnapshot(); + + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(3, "c", "INSERT", 2, snap3.snapshotId()), + row(3, "c", "DELETE", 3, snap4.snapshotId())), + sql("SELECT * FROM %s.changes WHERE id = 3 ORDER BY _change_ordinal, id", tableName)); + } + + @TestTemplate + public void testOverwrites() { + createTableWithDefaultRows(); + + Table table = validationCatalog.loadTable(tableIdent); + + Snapshot snap2 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName); + + table.refresh(); + + Snapshot snap3 = table.currentSnapshot(); + + assertEquals( + "Rows should match", + ImmutableList.of( + row(2, "b", "DELETE", 0, snap3.snapshotId()), + row(-2, "b", "INSERT", 0, snap3.snapshotId())), + changelogRecords(snap2, snap3)); + } + + @TestTemplate + public void testQueryWithTimeRange() { + createTable(); + + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap1 = table.currentSnapshot(); + long rightAfterSnap1 = waitUntilAfter(snap1.timestampMillis()); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + long rightAfterSnap2 = waitUntilAfter(snap2.timestampMillis()); + + sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName); + table.refresh(); + Snapshot snap3 = table.currentSnapshot(); + long rightAfterSnap3 = waitUntilAfter(snap3.timestampMillis()); + + assertEquals( + "Should have expected changed rows only from snapshot 3", + ImmutableList.of( + row(2, "b", "DELETE", 0, snap3.snapshotId()), + row(-2, "b", "INSERT", 0, snap3.snapshotId())), + changelogRecords(rightAfterSnap2, snap3.timestampMillis())); + + assertEquals( + "Should have expected changed rows only from snapshot 3", + ImmutableList.of( + row(2, "b", "DELETE", 0, snap3.snapshotId()), + row(-2, "b", "INSERT", 0, snap3.snapshotId())), + changelogRecords(snap2.timestampMillis(), snap3.timestampMillis())); + + assertEquals( + "Should have expected changed rows from snapshot 2 and 3", + ImmutableList.of( + row(2, "b", "INSERT", 0, snap2.snapshotId()), + row(2, "b", "DELETE", 1, snap3.snapshotId()), + row(-2, "b", "INSERT", 1, snap3.snapshotId())), + changelogRecords(rightAfterSnap1, snap3.timestampMillis())); + + assertEquals( + "Should have expected changed rows up to the current snapshot", + ImmutableList.of( + row(2, "b", "INSERT", 0, snap2.snapshotId()), + row(2, "b", "DELETE", 1, snap3.snapshotId()), + row(-2, "b", "INSERT", 1, snap3.snapshotId())), + changelogRecords(rightAfterSnap1, null)); + + assertEquals( + "Should have empty changed rows if end time is before the first snapshot", + ImmutableList.of(), + changelogRecords(null, snap1.timestampMillis() - 1)); + + assertEquals( + "Should have empty changed rows if start time is after the current snapshot", + ImmutableList.of(), + changelogRecords(rightAfterSnap3, null)); + + assertEquals( + "Should have empty changed rows if end time is before the first snapshot", + ImmutableList.of(), + changelogRecords(null, snap1.timestampMillis() - 1)); + + assertEquals( + "Should have empty changed rows if there are no snapshots between start time and end time", + ImmutableList.of(), + changelogRecords(rightAfterSnap2, snap3.timestampMillis() - 1)); + } + + @TestTemplate + public void testTimeRangeValidation() { + createTableWithDefaultRows(); + + Table table = validationCatalog.loadTable(tableIdent); + + Snapshot snap2 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName); + table.refresh(); + Snapshot snap3 = table.currentSnapshot(); + long rightAfterSnap3 = waitUntilAfter(snap3.timestampMillis()); + assertThatThrownBy(() -> changelogRecords(snap3.timestampMillis(), snap2.timestampMillis())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot set start-timestamp to be greater than end-timestamp for changelogs"); + } + + @TestTemplate + public void testMetadataDeletes() { + createTableWithDefaultRows(); + + Table table = validationCatalog.loadTable(tableIdent); + + Snapshot snap2 = table.currentSnapshot(); + + sql("DELETE FROM %s WHERE data = 'a'", tableName); + + table.refresh(); + + Snapshot snap3 = table.currentSnapshot(); + assertThat(snap3.operation()).as("Operation must match").isEqualTo(DataOperations.DELETE); + + assertEquals( + "Rows should match", + ImmutableList.of(row(1, "a", "DELETE", 0, snap3.snapshotId())), + changelogRecords(snap2, snap3)); + } + + @TestTemplate + public void testExistingEntriesInNewDataManifestsAreIgnored() { + sql( + "CREATE TABLE %s (id INT, data STRING) " + + "USING iceberg " + + "PARTITIONED BY (data) " + + "TBLPROPERTIES ( " + + " '%s' = '%d', " + + " '%s' = '1', " + + " '%s' = 'true' " + + ")", + tableName, FORMAT_VERSION, formatVersion, MANIFEST_MIN_MERGE_COUNT, MANIFEST_MERGE_ENABLED); + + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + Snapshot snap1 = table.currentSnapshot(); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + + table.refresh(); + + Snapshot snap2 = table.currentSnapshot(); + assertThat(snap2.dataManifests(table.io())).as("Manifest number must match").hasSize(1); + + assertEquals( + "Rows should match", + ImmutableList.of(row(2, "b", "INSERT", 0, snap2.snapshotId())), + changelogRecords(snap1, snap2)); + } + + @TestTemplate + public void testManifestRewritesAreIgnored() { + createTableWithDefaultRows(); + + sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Num snapshots must match").hasSize(3); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "INSERT"), row(2, "INSERT")), + sql("SELECT id, _change_type FROM %s.changes ORDER BY id", tableName)); + } + + @TestTemplate + public void testMetadataColumns() { + createTableWithDefaultRows(); + List rows = + sql( + "SELECT id, _file, _pos, _deleted, _spec_id, _partition FROM %s.changes ORDER BY id", + tableName); + + String file1 = rows.get(0)[1].toString(); + assertThat(file1).startsWith("file:/"); + String file2 = rows.get(1)[1].toString(); + + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, file1, 0L, false, 0, row("a")), row(2, file2, 0L, false, 0, row("b"))), + rows); + } + + @TestTemplate + public void testQueryWithRollback() { + createTable(); + + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap1 = table.currentSnapshot(); + long rightAfterSnap1 = waitUntilAfter(snap1.timestampMillis()); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + long rightAfterSnap2 = waitUntilAfter(snap2.timestampMillis()); + + sql( + "CALL %s.system.rollback_to_snapshot('%s', %d)", + catalogName, tableIdent, snap1.snapshotId()); + table.refresh(); + assertThat(table.currentSnapshot()).isEqualTo(snap1); + + sql("INSERT OVERWRITE %s VALUES (-2, 'a')", tableName); + table.refresh(); + Snapshot snap3 = table.currentSnapshot(); + long rightAfterSnap3 = waitUntilAfter(snap3.timestampMillis()); + + assertEquals( + "Should have expected changed rows up to snapshot 3", + ImmutableList.of( + row(1, "a", "INSERT", 0, snap1.snapshotId()), + row(1, "a", "DELETE", 1, snap3.snapshotId()), + row(-2, "a", "INSERT", 1, snap3.snapshotId())), + changelogRecords(null, rightAfterSnap3)); + + assertEquals( + "Should have expected changed rows up to snapshot 2", + ImmutableList.of(row(1, "a", "INSERT", 0, snap1.snapshotId())), + changelogRecords(null, rightAfterSnap2)); + + assertEquals( + "Should have expected changed rows from snapshot 3 only since snapshot 2 is on a different branch.", + ImmutableList.of( + row(1, "a", "DELETE", 0, snap3.snapshotId()), + row(-2, "a", "INSERT", 0, snap3.snapshotId())), + changelogRecords(rightAfterSnap1, snap3.timestampMillis())); + + assertEquals( + "Should have expected changed rows from snapshot 3", + ImmutableList.of( + row(1, "a", "DELETE", 0, snap3.snapshotId()), + row(-2, "a", "INSERT", 0, snap3.snapshotId())), + changelogRecords(rightAfterSnap2, null)); + + sql( + "CALL %s.system.set_current_snapshot('%s', %d)", + catalogName, tableIdent, snap2.snapshotId()); + table.refresh(); + assertThat(table.currentSnapshot()).isEqualTo(snap2); + assertEquals( + "Should have expected changed rows from snapshot 2 only since snapshot 3 is on a different branch.", + ImmutableList.of(row(2, "b", "INSERT", 0, snap2.snapshotId())), + changelogRecords(rightAfterSnap1, null)); + } + + private void createTableWithDefaultRows() { + createTable(); + insertDefaultRows(); + } + + private void createTable() { + sql( + "CREATE TABLE %s (id INT, data STRING) " + + "USING iceberg " + + "PARTITIONED BY (data) " + + "TBLPROPERTIES ( " + + " '%s' = '%d' " + + ")", + tableName, FORMAT_VERSION, formatVersion); + } + + private void insertDefaultRows() { + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + } + + private List changelogRecords(Snapshot startSnapshot, Snapshot endSnapshot) { + DataFrameReader reader = spark.read(); + + if (startSnapshot != null) { + reader = reader.option(SparkReadOptions.START_SNAPSHOT_ID, startSnapshot.snapshotId()); + } + + if (endSnapshot != null) { + reader = reader.option(SparkReadOptions.END_SNAPSHOT_ID, endSnapshot.snapshotId()); + } + + return rowsToJava(collect(reader)); + } + + private List changelogRecords(Long startTimestamp, Long endTimeStamp) { + DataFrameReader reader = spark.read(); + + if (startTimestamp != null) { + reader = reader.option(SparkReadOptions.START_TIMESTAMP, startTimestamp); + } + + if (endTimeStamp != null) { + reader = reader.option(SparkReadOptions.END_TIMESTAMP, endTimeStamp); + } + + return rowsToJava(collect(reader)); + } + + private List collect(DataFrameReader reader) { + return reader + .table(tableName + "." + SparkChangelogTable.TABLE_NAME) + .orderBy("_change_ordinal", "_commit_snapshot_id", "_change_type", "id") + .collectAsList(); + } + + @TestTemplate + public void testChangelogViewOutsideTimeRange() { + createTableWithDefaultRows(); + + // Insert new records + sql("INSERT INTO %s VALUES (3, 'c')", tableName); + sql("INSERT INTO %s VALUES (4, 'd')", tableName); + + // Small delay to ensure our timestamps are different + try { + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException("Test interrupted", e); + } + + long startTime = System.currentTimeMillis(); + long endTime = startTime + 1000; // 1 second window + + // Create changelog view for a time window after our inserts + sql( + "CALL %s.system.create_changelog_view(" + + " table => '%s', " + + " options => map(" + + " 'start-timestamp', '%d'," + + " 'end-timestamp', '%d'" + + " )," + + " changelog_view => 'test_changelog_view'" + + ")", + catalogName, tableName, startTime, endTime); + + // Query the changelog view + List results = + sql( + "SELECT * FROM test_changelog_view WHERE _change_type IN ('INSERT', 'DELETE') ORDER BY _change_ordinal"); + + // Verify no changes are returned since our window is after the inserts + assertThat(results).as("Num records must be zero").isEmpty(); + + // Clean up the changelog view + sql("DROP VIEW IF EXISTS test_changelog_view"); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java new file mode 100644 index 000000000000..a236f13f12bd --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCherrypickSnapshotProcedure.java @@ -0,0 +1,194 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestCherrypickSnapshotProcedure extends ExtensionsTestBase { + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testCherrypickSnapshotUsingPositionalArgs() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); + + spark.conf().set("spark.wap.id", "1"); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should not see rows from staged snapshot", + ImmutableList.of(), + sql("SELECT * FROM %s", tableName)); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot wapSnapshot = Iterables.getOnlyElement(table.snapshots()); + + List output = + sql( + "CALL %s.system.cherrypick_snapshot('%s', %dL)", + catalogName, tableIdent, wapSnapshot.snapshotId()); + + table.refresh(); + + Snapshot currentSnapshot = table.currentSnapshot(); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(wapSnapshot.snapshotId(), currentSnapshot.snapshotId())), + output); + + assertEquals( + "Cherrypick must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s", tableName)); + } + + @TestTemplate + public void testCherrypickSnapshotUsingNamedArgs() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); + + spark.conf().set("spark.wap.id", "1"); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should not see rows from staged snapshot", + ImmutableList.of(), + sql("SELECT * FROM %s", tableName)); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot wapSnapshot = Iterables.getOnlyElement(table.snapshots()); + + List output = + sql( + "CALL %s.system.cherrypick_snapshot(snapshot_id => %dL, table => '%s')", + catalogName, wapSnapshot.snapshotId(), tableIdent); + + table.refresh(); + + Snapshot currentSnapshot = table.currentSnapshot(); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(wapSnapshot.snapshotId(), currentSnapshot.snapshotId())), + output); + + assertEquals( + "Cherrypick must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s", tableName)); + } + + @TestTemplate + public void testCherrypickSnapshotRefreshesRelationCache() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); + + Dataset query = spark.sql("SELECT * FROM " + tableName + " WHERE id = 1"); + query.createOrReplaceTempView("tmp"); + + spark.sql("CACHE TABLE tmp"); + + assertEquals("View should not produce rows", ImmutableList.of(), sql("SELECT * FROM tmp")); + + spark.conf().set("spark.wap.id", "1"); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should not see rows from staged snapshot", + ImmutableList.of(), + sql("SELECT * FROM %s", tableName)); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot wapSnapshot = Iterables.getOnlyElement(table.snapshots()); + + sql( + "CALL %s.system.cherrypick_snapshot('%s', %dL)", + catalogName, tableIdent, wapSnapshot.snapshotId()); + + assertEquals( + "Cherrypick snapshot should be visible", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM tmp")); + + sql("UNCACHE TABLE tmp"); + } + + @TestTemplate + public void testCherrypickInvalidSnapshot() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + assertThatThrownBy( + () -> sql("CALL %s.system.cherrypick_snapshot('%s', -1L)", catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot cherry-pick unknown snapshot ID: -1"); + } + + @TestTemplate + public void testInvalidCherrypickSnapshotCases() { + assertThatThrownBy( + () -> sql("CALL %s.system.cherrypick_snapshot('n', table => 't', 1L)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[UNEXPECTED_POSITIONAL_ARGUMENT] Cannot invoke routine `cherrypick_snapshot` because it contains positional argument(s) following the named argument assigned to `table`; please rearrange them so the positional arguments come first and then retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.custom.cherrypick_snapshot('n', 't', 1L)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[FAILED_TO_LOAD_ROUTINE] Failed to load routine `%s`.`custom`.`cherrypick_snapshot`. SQLSTATE: 38000", + catalogName); + + assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `cherrypick_snapshot` because the parameter named `snapshot_id` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('', 1L)", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for parameter 'table'"); + + assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t', '2.2')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "[CAST_INVALID_INPUT] The value '2.2' of the type \"STRING\" cannot be cast to \"BIGINT\" because it is malformed."); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputePartitionStatsProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputePartitionStatsProcedure.java new file mode 100644 index 000000000000..37423fc14736 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputePartitionStatsProcedure.java @@ -0,0 +1,121 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionStatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestComputePartitionStatsProcedure extends ExtensionsTestBase { + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void procedureOnEmptyTable() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + List result = + sql("CALL %s.system.compute_partition_stats('%s')", catalogName, tableIdent); + assertThat(result).isEmpty(); + } + + @TestTemplate + public void procedureWithPositionalArgs() throws NoSuchTableException, ParseException { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", tableName); + List output = + sql("CALL %s.system.compute_partition_stats('%s')", catalogName, tableIdent); + assertThat(output.get(0)).isNotEmpty(); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + assertThat(table.partitionStatisticsFiles()).hasSize(1); + PartitionStatisticsFile statisticsFile = table.partitionStatisticsFiles().get(0); + assertThat(statisticsFile.path()).isEqualTo(output.get(0)[0].toString()); + assertThat(statisticsFile.snapshotId()).isEqualTo(table.currentSnapshot().snapshotId()); + assertThat(new File(statisticsFile.path().replace("file:", ""))).exists(); + } + + @TestTemplate + public void procedureWithNamedArgs() throws NoSuchTableException, ParseException { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", tableName); + sql("ALTER TABLE %s CREATE BRANCH `b1`", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + long branchSnapshotId = table.refs().get("b1").snapshotId(); + sql("INSERT INTO TABLE %s VALUES (5, 'e'), (6, 'f'), (7, 'g'), (8, 'h')", tableName); + + List output = + sql( + "CALL %s.system.compute_partition_stats(table => '%s', snapshot_id => %s)", + catalogName, tableIdent, branchSnapshotId); + table.refresh(); + assertThat(table.partitionStatisticsFiles()).hasSize(1); + PartitionStatisticsFile statisticsFile = table.partitionStatisticsFiles().get(0); + assertThat(statisticsFile.path()).isEqualTo(output.get(0)[0].toString()); + // should be from the branch's snapshot instead of latest snapshot of the table + assertThat(statisticsFile.snapshotId()).isEqualTo(branchSnapshotId); + assertThat(new File(statisticsFile.path().replace("file:", ""))).exists(); + } + + @TestTemplate + public void procedureWithInvalidSnapshotId() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.compute_partition_stats(table => '%s', snapshot_id => 42)", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Snapshot not found: 42"); + } + + @TestTemplate + public void procedureWithInvalidTable() { + assertThatThrownBy( + () -> + sql( + "CALL %s.system.compute_partition_stats(table => '%s')", + catalogName, TableIdentifier.of(Namespace.of("default"), "abcd"))) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Couldn't load table"); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java new file mode 100644 index 000000000000..c487d4b7c848 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestComputeTableStatsProcedure.java @@ -0,0 +1,137 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.iceberg.BlobMetadata; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.actions.NDVSketchUtil; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestComputeTableStatsProcedure extends ExtensionsTestBase { + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testProcedureOnEmptyTable() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + List result = + sql("CALL %s.system.compute_table_stats('%s')", catalogName, tableIdent); + assertThat(result).isEmpty(); + } + + @TestTemplate + public void testProcedureWithNamedArgs() throws NoSuchTableException, ParseException { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", tableName); + List output = + sql( + "CALL %s.system.compute_table_stats(table => '%s', columns => array('id'))", + catalogName, tableIdent); + assertThat(output.get(0)).isNotEmpty(); + Object obj = output.get(0)[0]; + assertThat(obj.toString()).endsWith(".stats"); + verifyTableStats(tableName); + } + + @TestTemplate + public void testProcedureWithPositionalArgs() throws NoSuchTableException, ParseException { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + Snapshot snapshot = table.currentSnapshot(); + List output = + sql( + "CALL %s.system.compute_table_stats('%s', %dL)", + catalogName, tableIdent, snapshot.snapshotId()); + assertThat(output.get(0)).isNotEmpty(); + Object obj = output.get(0)[0]; + assertThat(obj.toString()).endsWith(".stats"); + verifyTableStats(tableName); + } + + @TestTemplate + public void testProcedureWithInvalidColumns() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", tableName); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.compute_table_stats(table => '%s', columns => array('id1'))", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Can't find column id1"); + } + + @TestTemplate + public void testProcedureWithInvalidSnapshot() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.compute_table_stats(table => '%s', snapshot_id => %dL)", + catalogName, tableIdent, 1234L)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Snapshot not found"); + } + + @TestTemplate + public void testProcedureWithInvalidTable() { + assertThatThrownBy( + () -> + sql( + "CALL %s.system.compute_table_stats(table => '%s', snapshot_id => %dL)", + catalogName, TableIdentifier.of(Namespace.of("default"), "abcd"), 1234L)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Couldn't load table"); + } + + void verifyTableStats(String tableName) throws NoSuchTableException, ParseException { + Table table = Spark3Util.loadIcebergTable(spark, tableName); + StatisticsFile statisticsFile = table.statisticsFiles().get(0); + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + assertThat(blobMetadata.properties()) + .containsKey(NDVSketchUtil.APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java new file mode 100644 index 000000000000..b5ba7eec1b01 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java @@ -0,0 +1,394 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.iceberg.IsolationLevel; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.functions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestConflictValidation extends ExtensionsTestBase { + + @BeforeEach + public void createTables() { + sql( + "CREATE TABLE %s (id int, data string) USING iceberg " + + "PARTITIONED BY (id)" + + "TBLPROPERTIES" + + "('format-version'='2'," + + "'write.delete.mode'='merge-on-read')", + tableName); + sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testOverwriteFilterSerializableIsolation() throws Exception { + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + List records = Lists.newArrayList(new SimpleRecord(1, "a")); + spark.createDataFrame(records, SimpleRecord.class).writeTo(tableName).append(); + + // Validating from previous snapshot finds conflicts + Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); + assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwrite(functions.col("id").equalTo(1))) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found conflicting files that can contain records matching ref(name=\"id\") == 1:"); + + // Validating from latest snapshot should succeed + table.refresh(); + long newSnapshotId = table.currentSnapshot().snapshotId(); + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(newSnapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwrite(functions.col("id").equalTo(1)); + } + + @TestTemplate + public void testOverwriteFilterSerializableIsolation2() throws Exception { + List records = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(1, "b")); + spark.createDataFrame(records, SimpleRecord.class).coalesce(1).writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + // This should add a delete file + sql("DELETE FROM %s WHERE id='1' and data='b'", tableName); + table.refresh(); + + // Validating from previous snapshot finds conflicts + List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); + Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); + assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwrite(functions.col("id").equalTo(1))) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found new conflicting delete files that can apply to records matching ref(name=\"id\") == 1:"); + + // Validating from latest snapshot should succeed + table.refresh(); + long newSnapshotId = table.currentSnapshot().snapshotId(); + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(newSnapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwrite(functions.col("id").equalTo(1)); + } + + @TestTemplate + public void testOverwriteFilterSerializableIsolation3() throws Exception { + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + // This should delete a data file + sql("DELETE FROM %s WHERE id='1'", tableName); + table.refresh(); + + // Validating from previous snapshot finds conflicts + List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); + Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); + assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwrite(functions.col("id").equalTo(1))) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found conflicting deleted files that can contain records matching ref(name=\"id\") == 1:"); + + // Validating from latest snapshot should succeed + table.refresh(); + long newSnapshotId = table.currentSnapshot().snapshotId(); + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(newSnapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwrite(functions.col("id").equalTo(1)); + } + + @TestTemplate + public void testOverwriteFilterNoSnapshotIdValidation() throws Exception { + Table table = validationCatalog.loadTable(tableIdent); + + List records = Lists.newArrayList(new SimpleRecord(1, "a")); + spark.createDataFrame(records, SimpleRecord.class).writeTo(tableName).append(); + + // Validating from no snapshot id defaults to beginning snapshot id and finds conflicts + Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); + + assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwrite(functions.col("id").equalTo(1))) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found conflicting files that can contain records matching ref(name=\"id\") == 1:"); + + // Validating from latest snapshot should succeed + table.refresh(); + long newSnapshotId = table.currentSnapshot().snapshotId(); + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(newSnapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwrite(functions.col("id").equalTo(1)); + } + + @TestTemplate + public void testOverwriteFilterSnapshotIsolation() throws Exception { + List records = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(1, "b")); + spark.createDataFrame(records, SimpleRecord.class).coalesce(1).writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + // This should add a delete file + sql("DELETE FROM %s WHERE id='1' and data='b'", tableName); + table.refresh(); + + // Validating from previous snapshot finds conflicts + List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); + Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); + assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwrite(functions.col("id").equalTo(1))) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found new conflicting delete files that can apply to records matching ref(name=\"id\") == 1:"); + + // Validating from latest snapshot should succeed + table.refresh(); + long newSnapshotId = table.currentSnapshot().snapshotId(); + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(newSnapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwrite(functions.col("id").equalTo(1)); + } + + @TestTemplate + public void testOverwriteFilterSnapshotIsolation2() throws Exception { + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + List records = Lists.newArrayList(new SimpleRecord(1, "a")); + spark.createDataFrame(records, SimpleRecord.class).writeTo(tableName).append(); + + // Validation should not fail due to conflicting data file in snapshot isolation mode + Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwrite(functions.col("id").equalTo(1)); + } + + @TestTemplate + public void testOverwritePartitionSerializableIsolation() throws Exception { + Table table = validationCatalog.loadTable(tableIdent); + final long snapshotId = table.currentSnapshot().snapshotId(); + + List records = Lists.newArrayList(new SimpleRecord(1, "a")); + spark.createDataFrame(records, SimpleRecord.class).writeTo(tableName).append(); + + // Validating from previous snapshot finds conflicts + Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); + assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwritePartitions()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found conflicting files that can contain records matching partitions [id=1]"); + + // Validating from latest snapshot should succeed + table.refresh(); + long newSnapshotId = table.currentSnapshot().snapshotId(); + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(newSnapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwritePartitions(); + } + + @TestTemplate + public void testOverwritePartitionSnapshotIsolation() throws Exception { + List records = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(1, "b")); + spark.createDataFrame(records, SimpleRecord.class).coalesce(1).writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + final long snapshotId = table.currentSnapshot().snapshotId(); + + // This should generate a delete file + sql("DELETE FROM %s WHERE data='a'", tableName); + + // Validating from previous snapshot finds conflicts + Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); + assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwritePartitions()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found new conflicting delete files that can apply to records matching [id=1]"); + + // Validating from latest snapshot should succeed + table.refresh(); + long newSnapshotId = table.currentSnapshot().snapshotId(); + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(newSnapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwritePartitions(); + } + + @TestTemplate + public void testOverwritePartitionSnapshotIsolation2() throws Exception { + Table table = validationCatalog.loadTable(tableIdent); + final long snapshotId = table.currentSnapshot().snapshotId(); + + // This should delete a data file + sql("DELETE FROM %s WHERE id='1'", tableName); + + // Validating from previous snapshot finds conflicts + List records = Lists.newArrayList(new SimpleRecord(1, "a")); + spark.createDataFrame(records, SimpleRecord.class).coalesce(1).writeTo(tableName).append(); + Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); + + assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwritePartitions()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found conflicting deleted files that can apply to records matching [id=1]"); + + // Validating from latest snapshot should succeed + table.refresh(); + long newSnapshotId = table.currentSnapshot().snapshotId(); + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(newSnapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwritePartitions(); + } + + @TestTemplate + public void testOverwritePartitionSnapshotIsolation3() throws Exception { + Table table = validationCatalog.loadTable(tableIdent); + final long snapshotId = table.currentSnapshot().snapshotId(); + + List records = Lists.newArrayList(new SimpleRecord(1, "a")); + spark.createDataFrame(records, SimpleRecord.class).writeTo(tableName).append(); + + // Validation should not find conflicting data file in snapshot isolation mode + Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwritePartitions(); + } + + @TestTemplate + public void testOverwritePartitionNoSnapshotIdValidation() throws Exception { + Table table = validationCatalog.loadTable(tableIdent); + + List records = Lists.newArrayList(new SimpleRecord(1, "a")); + spark.createDataFrame(records, SimpleRecord.class).writeTo(tableName).append(); + + // Validating from null snapshot is equivalent to validating from beginning + Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); + assertThatThrownBy( + () -> + conflictingDf + .writeTo(tableName) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwritePartitions()) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Found conflicting files that can contain records matching partitions [id=1]"); + + // Validating from latest snapshot should succeed + table.refresh(); + long snapshotId = table.currentSnapshot().snapshotId(); + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwritePartitions(); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java new file mode 100644 index 000000000000..f7ded0c4d7d2 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java @@ -0,0 +1,227 @@ +/* + * 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.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; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +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; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestCopyOnWriteDelete extends TestDelete { + + @Override + protected Map extraTableProperties() { + return ImmutableMap.of( + 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(); + + append(tableName, new Employee(1, "hr"), new Employee(3, "hr")); + createBranchIfNeeded(); + append(new Employee(1, "hardware"), new Employee(2, "hardware")); + + Map sqlConf = + ImmutableMap.of( + SQLConf.V2_BUCKETING_ENABLED().key(), + "true", + SparkSQLProperties.PRESERVE_DATA_GROUPING, + "true"); + + withSQLConf(sqlConf, () -> sql("DELETE FROM %s WHERE id = 2", commitTarget())); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + validateCopyOnWrite(currentSnapshot, "1", "1", "1"); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hardware"), row(1, "hr"), row(3, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); + } + + @TestTemplate + public void testEqualityDeletePreservation() throws NoSuchTableException, IOException { + createAndInitPartitionedTable(); + append(tableName, new Employee(1, "hr"), new Employee(2, "hr"), new Employee(3, "hr")); + + Table table = validationCatalog.loadTable(tableIdent); + OutputFile out = Files.localOutput(File.createTempFile("junit", null, temp.toFile())); + Schema deleteSchema = table.schema().select("id"); + GenericRecord deleteRecord = GenericRecord.create(deleteSchema); + DeleteFile eqDelete = + FileHelpers.writeDeleteFile( + table, + out, + TestHelpers.Row.of("hr"), + List.of(deleteRecord.copy("id", 2)), + deleteSchema); + + table.newRowDelta().addDeletes(eqDelete).commit(); + + sql("REFRESH TABLE %s", tableName); + + assertEquals( + "Equality delete should remove row with id 2", + ImmutableList.of(row(1, "hr"), row(3, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", tableName)); + + sql("DELETE FROM %s WHERE id = 3", tableName); + + assertEquals( + "COW Delete should remove row with id 3", + ImmutableList.of(row(1, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", tableName)); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java new file mode 100644 index 000000000000..fef8b28c689a --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java @@ -0,0 +1,195 @@ +/* + * 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.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; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestCopyOnWriteMerge extends TestMerge { + + @Override + protected Map extraTableProperties() { + return ImmutableMap.of( + 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"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 3, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + append( + commitTarget(), + "{ \"id\": 1, \"dep\": \"hardware\" }\n" + "{ \"id\": 2, \"dep\": \"hardware\" }"); + + createOrReplaceView("source", Collections.singletonList(2), Encoders.INT()); + + Map sqlConf = + ImmutableMap.of( + SQLConf.V2_BUCKETING_ENABLED().key(), + "true", + SparkSQLProperties.PRESERVE_DATA_GROUPING, + "true"); + + withSQLConf( + sqlConf, + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.value " + + "WHEN MATCHED THEN " + + " UPDATE SET id = -1", + commitTarget())); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + validateCopyOnWrite(currentSnapshot, "1", "1", "1"); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hardware"), row(1, "hardware"), row(1, "hr"), row(3, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java new file mode 100644 index 000000000000..21d1377b2b98 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.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; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestCopyOnWriteUpdate extends TestUpdate { + + @Override + protected Map extraTableProperties() { + return ImmutableMap.of( + 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"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 3, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + append( + commitTarget(), + "{ \"id\": 1, \"dep\": \"hardware\" }\n" + "{ \"id\": 2, \"dep\": \"hardware\" }"); + + Map sqlConf = + ImmutableMap.of( + SQLConf.V2_BUCKETING_ENABLED().key(), + "true", + SparkSQLProperties.PRESERVE_DATA_GROUPING, + "true"); + + withSQLConf(sqlConf, () -> sql("UPDATE %s SET id = -1 WHERE id = 2", commitTarget())); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + validateCopyOnWrite(currentSnapshot, "1", "1", "1"); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hardware"), row(1, "hardware"), row(1, "hr"), row(3, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteWithLineage.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteWithLineage.java new file mode 100644 index 000000000000..5b82f55ead33 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteWithLineage.java @@ -0,0 +1,35 @@ +/* + * 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.extensions; + +import java.util.Map; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestCopyOnWriteWithLineage extends TestRowLevelOperationsWithLineage { + + @Override + protected Map extraTableProperties() { + return ImmutableMap.of( + TableProperties.MERGE_MODE, RowLevelOperationMode.COPY_ON_WRITE.modeName(), + TableProperties.UPDATE_MODE, RowLevelOperationMode.COPY_ON_WRITE.modeName(), + TableProperties.DELETE_MODE, RowLevelOperationMode.COPY_ON_WRITE.modeName()); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java new file mode 100644 index 000000000000..3734d24e969b --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java @@ -0,0 +1,694 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.ChangelogOperation; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.spark.sql.types.StructField; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestCreateChangelogViewProcedure extends ExtensionsTestBase { + private static final String DELETE = ChangelogOperation.DELETE.name(); + private static final String INSERT = ChangelogOperation.INSERT.name(); + private static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name(); + private static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name(); + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + public void createTableWithTwoColumns() { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + sql("ALTER TABLE %s ADD PARTITION FIELD data", tableName); + } + + private void createTableWithThreeColumns() { + sql("CREATE TABLE %s (id INT, data STRING, age INT) USING iceberg", tableName); + sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName); + } + + private void createTableWithIdentifierField() { + sql("CREATE TABLE %s (id INT NOT NULL, data STRING) USING iceberg", tableName); + sql("ALTER TABLE %s SET IDENTIFIER FIELDS id", tableName); + } + + @TestTemplate + public void testCustomizedViewName() { + createTableWithTwoColumns(); + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + Snapshot snap1 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName); + + table.refresh(); + + Snapshot snap2 = table.currentSnapshot(); + + sql( + "CALL %s.system.create_changelog_view(" + + "table => '%s'," + + "options => map('%s','%s','%s','%s')," + + "changelog_view => '%s')", + catalogName, + tableName, + SparkReadOptions.START_SNAPSHOT_ID, + snap1.snapshotId(), + SparkReadOptions.END_SNAPSHOT_ID, + snap2.snapshotId(), + "cdc_view"); + + long rowCount = sql("select * from %s", "cdc_view").stream().count(); + assertThat(rowCount).isEqualTo(2); + } + + @TestTemplate + public void testNonStandardColumnNames() { + sql("CREATE TABLE %s (`the id` INT, `the.data` STRING) USING iceberg", tableName); + sql("ALTER TABLE %s ADD PARTITION FIELD `the.data`", tableName); + + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + Snapshot snap1 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName); + + table.refresh(); + + Snapshot snap2 = table.currentSnapshot(); + + sql( + "CALL %s.system.create_changelog_view(" + + "table => '%s'," + + "options => map('%s','%s','%s','%s')," + + "changelog_view => '%s')", + catalogName, + tableName, + SparkReadOptions.START_SNAPSHOT_ID, + snap1.snapshotId(), + SparkReadOptions.END_SNAPSHOT_ID, + snap2.snapshotId(), + "cdc_view"); + + var df = spark.sql("select * from cdc_view"); + var fieldNames = + Arrays.stream(df.schema().fields()).map(StructField::name).collect(Collectors.toList()); + assertThat(fieldNames) + .containsExactly( + "the id", "the.data", "_change_type", "_change_ordinal", "_commit_snapshot_id"); + + assertThat(df.collectAsList()).hasSize(2); + } + + @TestTemplate + public void testNoSnapshotIdInput() { + createTableWithTwoColumns(); + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap0 = table.currentSnapshot(); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + table.refresh(); + Snapshot snap1 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + List returns = + sql( + "CALL %s.system.create_changelog_view(" + "table => '%s')", + catalogName, tableName, "cdc_view"); + + String viewName = (String) returns.get(0)[0]; + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, "a", INSERT, 0, snap0.snapshotId()), + row(2, "b", INSERT, 1, snap1.snapshotId()), + row(-2, "b", INSERT, 2, snap2.snapshotId()), + row(2, "b", DELETE, 2, snap2.snapshotId())), + sql("select * from %s order by _change_ordinal, id", viewName)); + } + + @TestTemplate + public void testOnlyStartSnapshotIdInput() { + createTableWithTwoColumns(); + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap0 = table.currentSnapshot(); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + table.refresh(); + Snapshot snap1 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + List returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s'," + "options => map('%s', '%s'))", + catalogName, tableName, SparkReadOptions.START_SNAPSHOT_ID, snap0.snapshotId()); + + assertEquals( + "Rows should match", + ImmutableList.of( + row(2, "b", INSERT, 0, snap1.snapshotId()), + row(-2, "b", INSERT, 1, snap2.snapshotId()), + row(2, "b", DELETE, 1, snap2.snapshotId())), + sql("select * from %s order by _change_ordinal, id", returns.get(0)[0])); + } + + @TestTemplate + public void testOnlyEndTimestampIdInput() { + createTableWithTwoColumns(); + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap0 = table.currentSnapshot(); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + table.refresh(); + Snapshot snap1 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName); + + List returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s'," + "options => map('%s', '%s'))", + catalogName, tableName, SparkReadOptions.END_SNAPSHOT_ID, snap1.snapshotId()); + + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, "a", INSERT, 0, snap0.snapshotId()), row(2, "b", INSERT, 1, snap1.snapshotId())), + sql("select * from %s order by _change_ordinal, id", returns.get(0)[0])); + } + + @TestTemplate + public void testTimestampsBasedQuery() { + createTableWithTwoColumns(); + long beginning = System.currentTimeMillis(); + + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap0 = table.currentSnapshot(); + long afterFirstInsert = waitUntilAfter(snap0.timestampMillis()); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + table.refresh(); + Snapshot snap1 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + long afterInsertOverwrite = waitUntilAfter(snap2.timestampMillis()); + List returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s', " + + "options => map('%s', '%s','%s', '%s'))", + catalogName, + tableName, + SparkReadOptions.START_TIMESTAMP, + beginning, + SparkReadOptions.END_TIMESTAMP, + afterInsertOverwrite); + + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, "a", INSERT, 0, snap0.snapshotId()), + row(2, "b", INSERT, 1, snap1.snapshotId()), + row(-2, "b", INSERT, 2, snap2.snapshotId()), + row(2, "b", DELETE, 2, snap2.snapshotId())), + sql("select * from %s order by _change_ordinal, id", returns.get(0)[0])); + + // query the timestamps starting from the second insert + returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s', " + + "options => map('%s', '%s', '%s', '%s'))", + catalogName, + tableName, + SparkReadOptions.START_TIMESTAMP, + afterFirstInsert, + SparkReadOptions.END_TIMESTAMP, + afterInsertOverwrite); + + assertEquals( + "Rows should match", + ImmutableList.of( + row(2, "b", INSERT, 0, snap1.snapshotId()), + row(-2, "b", INSERT, 1, snap2.snapshotId()), + row(2, "b", DELETE, 1, snap2.snapshotId())), + sql("select * from %s order by _change_ordinal, id", returns.get(0)[0])); + } + + @TestTemplate + public void testOnlyStartTimestampInput() { + createTableWithTwoColumns(); + long beginning = System.currentTimeMillis(); + + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap0 = table.currentSnapshot(); + long afterFirstInsert = waitUntilAfter(snap0.timestampMillis()); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + table.refresh(); + Snapshot snap1 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + List returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s', " + "options => map('%s', '%s'))", + catalogName, tableName, SparkReadOptions.START_TIMESTAMP, beginning); + + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, "a", INSERT, 0, snap0.snapshotId()), + row(2, "b", INSERT, 1, snap1.snapshotId()), + row(-2, "b", INSERT, 2, snap2.snapshotId()), + row(2, "b", DELETE, 2, snap2.snapshotId())), + sql("select * from %s order by _change_ordinal, id", returns.get(0)[0])); + + returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s', " + "options => map('%s', '%s'))", + catalogName, tableName, SparkReadOptions.START_TIMESTAMP, afterFirstInsert); + + assertEquals( + "Rows should match", + ImmutableList.of( + row(2, "b", INSERT, 0, snap1.snapshotId()), + row(-2, "b", INSERT, 1, snap2.snapshotId()), + row(2, "b", DELETE, 1, snap2.snapshotId())), + sql("select * from %s order by _change_ordinal, id", returns.get(0)[0])); + } + + @TestTemplate + public void testOnlyEndTimestampInput() { + createTableWithTwoColumns(); + + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap0 = table.currentSnapshot(); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + table.refresh(); + Snapshot snap1 = table.currentSnapshot(); + long afterSecondInsert = waitUntilAfter(snap1.timestampMillis()); + + sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName); + + List returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s', " + "options => map('%s', '%s'))", + catalogName, tableName, SparkReadOptions.END_TIMESTAMP, afterSecondInsert); + + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, "a", INSERT, 0, snap0.snapshotId()), row(2, "b", INSERT, 1, snap1.snapshotId())), + sql("select * from %s order by _change_ordinal, id", returns.get(0)[0])); + } + + @TestTemplate + public void testStartTimeStampEndSnapshotId() { + createTableWithTwoColumns(); + + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap0 = table.currentSnapshot(); + long afterFirstInsert = waitUntilAfter(snap0.timestampMillis()); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + table.refresh(); + Snapshot snap1 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + List returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s', " + + "options => map('%s', '%s', '%s', '%s'))", + catalogName, + tableName, + SparkReadOptions.START_TIMESTAMP, + afterFirstInsert, + SparkReadOptions.END_SNAPSHOT_ID, + snap2.snapshotId()); + + assertEquals( + "Rows should match", + ImmutableList.of( + row(2, "b", INSERT, 0, snap1.snapshotId()), + row(-2, "b", INSERT, 1, snap2.snapshotId()), + row(2, "b", DELETE, 1, snap2.snapshotId())), + sql("select * from %s order by _change_ordinal, id", returns.get(0)[0])); + } + + @TestTemplate + public void testStartSnapshotIdEndTimestamp() { + createTableWithTwoColumns(); + + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap0 = table.currentSnapshot(); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + table.refresh(); + Snapshot snap1 = table.currentSnapshot(); + long afterSecondInsert = waitUntilAfter(snap1.timestampMillis()); + + sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName); + table.refresh(); + + List returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s', " + + "options => map('%s', '%s', '%s', '%s'))", + catalogName, + tableName, + SparkReadOptions.START_SNAPSHOT_ID, + snap0.snapshotId(), + SparkReadOptions.END_TIMESTAMP, + afterSecondInsert); + + assertEquals( + "Rows should match", + ImmutableList.of(row(2, "b", INSERT, 0, snap1.snapshotId())), + sql("select * from %s order by _change_ordinal, id", returns.get(0)[0])); + } + + @TestTemplate + public void testUpdate() { + createTableWithTwoColumns(); + sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName); + sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName); + + sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap1 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + List returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))", + catalogName, tableName); + + String viewName = (String) returns.get(0)[0]; + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, "a", INSERT, 0, snap1.snapshotId()), + row(2, "b", INSERT, 0, snap1.snapshotId()), + row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()), + row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()), + row(3, "c", INSERT, 1, snap2.snapshotId())), + sql("select * from %s order by _change_ordinal, id, data", viewName)); + } + + @TestTemplate + public void testUpdateWithIdentifierField() { + createTableWithIdentifierField(); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap1 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + List returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s', compute_updates => true)", + catalogName, tableName); + + String viewName = (String) returns.get(0)[0]; + assertEquals( + "Rows should match", + ImmutableList.of( + row(2, "b", INSERT, 0, snap1.snapshotId()), + row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()), + row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()), + row(3, "c", INSERT, 1, snap2.snapshotId())), + sql("select * from %s order by _change_ordinal, id, data", viewName)); + } + + @TestTemplate + public void testUpdateWithFilter() { + createTableWithTwoColumns(); + sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName); + sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName); + + sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap1 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + List returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))", + catalogName, tableName); + + String viewName = (String) returns.get(0)[0]; + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, "a", INSERT, 0, snap1.snapshotId()), + row(2, "b", INSERT, 0, snap1.snapshotId()), + row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()), + row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId())), + // the predicate on partition columns will filter out the insert of (3, 'c') at the planning + // phase + sql("select * from %s where id != 3 order by _change_ordinal, id, data", viewName)); + } + + @TestTemplate + public void testUpdateWithMultipleIdentifierColumns() { + createTableWithThreeColumns(); + + sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11)", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap1 = table.currentSnapshot(); + + sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + List returns = + sql( + "CALL %s.system.create_changelog_view(" + + "identifier_columns => array('id','age')," + + "table => '%s')", + catalogName, tableName); + + String viewName = (String) returns.get(0)[0]; + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, "a", 12, INSERT, 0, snap1.snapshotId()), + row(2, "b", 11, INSERT, 0, snap1.snapshotId()), + row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()), + row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()), + row(2, "e", 12, INSERT, 1, snap2.snapshotId()), + row(3, "c", 13, INSERT, 1, snap2.snapshotId())), + sql("select * from %s order by _change_ordinal, id, data", viewName)); + } + + @TestTemplate + public void testRemoveCarryOvers() { + createTableWithThreeColumns(); + + sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap1 = table.currentSnapshot(); + + // carry-over row (2, 'e', 12) + sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + List returns = + sql( + "CALL %s.system.create_changelog_view(" + + "identifier_columns => array('id','age'), " + + "table => '%s')", + catalogName, tableName); + + String viewName = (String) returns.get(0)[0]; + // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, "a", 12, INSERT, 0, snap1.snapshotId()), + row(2, "b", 11, INSERT, 0, snap1.snapshotId()), + row(2, "e", 12, INSERT, 0, snap1.snapshotId()), + row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()), + row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()), + row(3, "c", 13, INSERT, 1, snap2.snapshotId())), + sql("select * from %s order by _change_ordinal, id, data", viewName)); + } + + @TestTemplate + public void testRemoveCarryOversWithoutUpdatedRows() { + createTableWithThreeColumns(); + + sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap1 = table.currentSnapshot(); + + // carry-over row (2, 'e', 12) + sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + List returns = + sql("CALL %s.system.create_changelog_view(table => '%s')", catalogName, tableName); + + String viewName = (String) returns.get(0)[0]; + + // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed, even + // though update-row is not computed + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, "a", 12, INSERT, 0, snap1.snapshotId()), + row(2, "b", 11, INSERT, 0, snap1.snapshotId()), + row(2, "e", 12, INSERT, 0, snap1.snapshotId()), + row(2, "b", 11, DELETE, 1, snap2.snapshotId()), + row(2, "d", 11, INSERT, 1, snap2.snapshotId()), + row(3, "c", 13, INSERT, 1, snap2.snapshotId())), + sql("select * from %s order by _change_ordinal, id, data", viewName)); + } + + @TestTemplate + public void testNetChangesWithRemoveCarryOvers() { + // partitioned by id + createTableWithThreeColumns(); + + // insert rows: (1, 'a', 12) (2, 'b', 11) (2, 'e', 12) + sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap1 = table.currentSnapshot(); + + // delete rows: (2, 'b', 11) (2, 'e', 12) + // insert rows: (3, 'c', 13) (2, 'd', 11) (2, 'e', 12) + sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + + // delete rows: (2, 'd', 11) (2, 'e', 12) (3, 'c', 13) + // insert rows: (3, 'c', 15) (2, 'e', 12) + sql("INSERT OVERWRITE %s VALUES (3, 'c', 15), (2, 'e', 12)", tableName); + table.refresh(); + Snapshot snap3 = table.currentSnapshot(); + + // test with all snapshots + List returns = + sql( + "CALL %s.system.create_changelog_view(table => '%s', net_changes => true)", + catalogName, tableName); + + String viewName = (String) returns.get(0)[0]; + + assertEquals( + "Rows should match", + ImmutableList.of( + row(1, "a", 12, INSERT, 0, snap1.snapshotId()), + row(3, "c", 15, INSERT, 2, snap3.snapshotId()), + row(2, "e", 12, INSERT, 2, snap3.snapshotId())), + sql("select * from %s order by _change_ordinal, data", viewName)); + + // test with snap2 and snap3 + sql( + "CALL %s.system.create_changelog_view(table => '%s', " + + "options => map('start-snapshot-id','%s'), " + + "net_changes => true)", + catalogName, tableName, snap1.snapshotId()); + + assertEquals( + "Rows should match", + ImmutableList.of( + row(2, "b", 11, DELETE, 0, snap2.snapshotId()), + row(3, "c", 15, INSERT, 1, snap3.snapshotId())), + sql("select * from %s order by _change_ordinal, data", viewName)); + } + + @TestTemplate + public void testNetChangesWithComputeUpdates() { + createTableWithTwoColumns(); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'), net_changes => true)", + catalogName, tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Not support net changes with update images"); + } + + @TestTemplate + public void testUpdateWithInComparableType() { + sql( + "CREATE TABLE %s (id INT NOT NULL, data MAP, age INT) USING iceberg", + tableName); + + assertThatThrownBy( + () -> + sql("CALL %s.system.create_changelog_view(table => '%s')", catalogName, tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Identifier field is required as table contains unorderable columns: [data]"); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java new file mode 100644 index 000000000000..fbf6ce3559a7 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -0,0 +1,1486 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.DataOperations.DELETE; +import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; +import static org.apache.iceberg.SnapshotSummary.ADDED_DVS_PROP; +import static org.apache.iceberg.SnapshotSummary.ADD_POS_DELETE_FILES_PROP; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.TableProperties.DELETE_DISTRIBUTION_MODE; +import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; +import static org.apache.iceberg.TableProperties.DELETE_MODE; +import static org.apache.iceberg.TableProperties.DELETE_MODE_DEFAULT; +import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.iceberg.TableProperties.SPLIT_SIZE; +import static org.apache.spark.sql.functions.lit; +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.Arrays; +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 java.util.stream.Collectors; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.exceptions.ValidationException; +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.Lists; +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.spark.data.TestHelpers; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.catalyst.plans.logical.DeleteFromTableWithFilters; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.catalyst.plans.logical.RowLevelWrite; +import org.apache.spark.sql.execution.SparkPlan; +import org.apache.spark.sql.execution.datasources.v2.OptimizeMetadataOnlyDeleteFromTable; +import org.apache.spark.sql.internal.SQLConf; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TestDelete extends SparkRowLevelOperationsTestBase { + + @BeforeAll + public static void setupSparkConf() { + spark.conf().set("spark.sql.shuffle.partitions", "4"); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS deleted_id"); + sql("DROP TABLE IF EXISTS deleted_dep"); + sql("DROP TABLE IF EXISTS parquet_table"); + } + + @TestTemplate + public void testDeleteWithVectorizedReads() throws NoSuchTableException { + assumeThat(supportsVectorization()).isTrue(); + + createAndInitPartitionedTable(); + + append(tableName, new Employee(1, "hr"), new Employee(2, "hr")); + append(tableName, new Employee(3, "hardware"), new Employee(4, "hardware")); + + createBranchIfNeeded(); + + SparkPlan plan = executeAndKeepPlan("DELETE FROM %s WHERE id = 2", commitTarget()); + assertAllBatchScansVectorized(plan); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + if (mode(table) == COPY_ON_WRITE) { + validateCopyOnWrite(currentSnapshot, "1", "1", "1"); + } else { + validateMergeOnRead(currentSnapshot, "1", "1", null); + } + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(3, "hardware"), row(4, "hardware")), + sql("SELECT * FROM %s ORDER BY id ASC", selectTarget())); + } + + @TestTemplate + public void testCoalesceDelete() throws Exception { + createAndInitUnpartitionedTable(); + + Employee[] employees = new Employee[100]; + for (int index = 0; index < 100; index++) { + employees[index] = new Employee(index, "hr"); + } + append(tableName, employees); + append(tableName, employees); + append(tableName, employees); + append(tableName, employees); + + // set the open file cost large enough to produce a separate scan task per file + // use range distribution to trigger a shuffle + // set partitioned scoped deletes so that 1 delete file is written as part of the output task + Map tableProps = + ImmutableMap.of( + SPLIT_OPEN_FILE_COST, + String.valueOf(Integer.MAX_VALUE), + DELETE_DISTRIBUTION_MODE, + DistributionMode.RANGE.modeName(), + TableProperties.DELETE_GRANULARITY, + DeleteGranularity.PARTITION.toString()); + sql("ALTER TABLE %s SET TBLPROPERTIES (%s)", tableName, tablePropsAsString(tableProps)); + + createBranchIfNeeded(); + + // enable AQE and set the advisory partition size big enough to trigger combining + // set the number of shuffle partitions to 200 to distribute the work across reducers + // set the advisory partition size for shuffles small enough to ensure writes override it + withSQLConf( + ImmutableMap.of( + SQLConf.SHUFFLE_PARTITIONS().key(), + "200", + SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), + "true", + SQLConf.COALESCE_PARTITIONS_ENABLED().key(), + "true", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES().key(), + "100", + SparkSQLProperties.ADVISORY_PARTITION_SIZE, + String.valueOf(256 * 1024 * 1024)), + () -> { + SparkPlan plan = + executeAndKeepPlan("DELETE FROM %s WHERE mod(id, 2) = 0", commitTarget()); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + }); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snapshot = SnapshotUtil.latestSnapshot(table, branch); + + if (mode(table) == COPY_ON_WRITE) { + // CoW DELETE requests the remaining records to be range distributed by `_file`, `_pos` + // every task has data for each of 200 reducers + // AQE detects that all shuffle blocks are small and processes them in 1 task + // otherwise, there would be 200 tasks writing to the table + validateProperty(snapshot, SnapshotSummary.ADDED_FILES_PROP, "1"); + } else if (mode(table) == MERGE_ON_READ && formatVersion >= 3) { + validateProperty(snapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "4"); + validateProperty(snapshot, SnapshotSummary.ADDED_DVS_PROP, "4"); + } else { + // MoR DELETE requests the deleted records to be range distributed by partition and `_file` + // each task contains only 1 file and therefore writes only 1 shuffle block + // that means 4 shuffle blocks are distributed among 200 reducers + // AQE detects that all 4 shuffle blocks are small and processes them in 1 task + // otherwise, there would be 4 tasks processing 1 shuffle block each + validateProperty(snapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "1"); + } + + assertThat(scalarSql("SELECT COUNT(*) FROM %s", commitTarget())) + .as("Row count must match") + .isEqualTo(200L); + } + + @TestTemplate + public void testSkewDelete() throws Exception { + createAndInitPartitionedTable(); + + Employee[] employees = new Employee[100]; + for (int index = 0; index < 100; index++) { + employees[index] = new Employee(index, "hr"); + } + append(tableName, employees); + append(tableName, employees); + append(tableName, employees); + append(tableName, employees); + + // set the open file cost large enough to produce a separate scan task per file + // use hash distribution to trigger a shuffle + Map tableProps = + ImmutableMap.of( + SPLIT_OPEN_FILE_COST, + String.valueOf(Integer.MAX_VALUE), + DELETE_DISTRIBUTION_MODE, + DistributionMode.HASH.modeName()); + sql("ALTER TABLE %s SET TBLPROPERTIES (%s)", tableName, tablePropsAsString(tableProps)); + + createBranchIfNeeded(); + + // enable AQE and set the advisory partition size small enough to trigger a split + // set the number of shuffle partitions to 2 to only have 2 reducers + // set the advisory partition size for shuffles big enough to ensure writes override it + withSQLConf( + ImmutableMap.of( + SQLConf.SHUFFLE_PARTITIONS().key(), + "2", + SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), + "true", + SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED().key(), + "true", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES().key(), + "256MB", + SparkSQLProperties.ADVISORY_PARTITION_SIZE, + "100"), + () -> { + SparkPlan plan = + executeAndKeepPlan("DELETE FROM %s WHERE mod(id, 2) = 0", commitTarget()); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + }); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snapshot = SnapshotUtil.latestSnapshot(table, branch); + + if (mode(table) == COPY_ON_WRITE) { + // CoW DELETE requests the remaining records to be clustered by `_file` + // each task contains only 1 file and therefore writes only 1 shuffle block + // that means 4 shuffle blocks are distributed among 2 reducers + // AQE detects that all shuffle blocks are big and processes them in 4 independent tasks + // otherwise, there would be 2 tasks processing 2 shuffle blocks each + validateProperty(snapshot, SnapshotSummary.ADDED_FILES_PROP, "4"); + } else { + // MoR DELETE requests the deleted records to be clustered by `_spec_id` and `_partition` + // all tasks belong to the same partition and therefore write only 1 shuffle block per task + // that means there are 4 shuffle blocks, all assigned to the same reducer + // AQE detects that all 4 shuffle blocks are big and processes them in 4 separate tasks + // otherwise, there would be 1 task processing 4 shuffle blocks + validateProperty(snapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "4"); + } + + assertThat(scalarSql("SELECT COUNT(*) FROM %s", commitTarget())) + .as("Row count must match") + .isEqualTo(200L); + } + + @TestTemplate + public void testDeleteWithoutScanningTable() throws Exception { + createAndInitPartitionedTable(); + + append(tableName, new Employee(1, "hr"), new Employee(3, "hr")); + createBranchIfNeeded(); + append(new Employee(1, "hardware"), new Employee(2, "hardware")); + + Table table = validationCatalog.loadTable(tableIdent); + + List manifestLocations = + SnapshotUtil.latestSnapshot(table, branch).allManifests(table.io()).stream() + .map(ManifestFile::path) + .collect(Collectors.toList()); + + withUnavailableLocations( + manifestLocations, + () -> { + LogicalPlan parsed = parsePlan("DELETE FROM %s WHERE dep = 'hr'", commitTarget()); + + LogicalPlan analyzed = spark.sessionState().analyzer().execute(parsed); + assertThat(analyzed).isInstanceOf(RowLevelWrite.class); + + LogicalPlan optimized = OptimizeMetadataOnlyDeleteFromTable.apply(analyzed); + assertThat(optimized).isInstanceOf(DeleteFromTableWithFilters.class); + }); + + sql("DELETE FROM %s WHERE dep = 'hr'", commitTarget()); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hardware"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testDeleteFileThenMetadataDelete() throws Exception { + assumeThat(fileFormat) + .as("Avro does not support metadata delete") + .isNotEqualTo(FileFormat.AVRO); + createAndInitUnpartitionedTable(); + createBranchIfNeeded(); + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", commitTarget()); + + // MOR mode: writes a delete file as null cannot be deleted by metadata + sql("DELETE FROM %s AS t WHERE t.id IS NULL", commitTarget()); + + // Metadata Delete + Table table = Spark3Util.loadIcebergTable(spark, tableName); + List dataFilesBefore = TestHelpers.dataFiles(table, branch); + + sql("DELETE FROM %s AS t WHERE t.id = 1", commitTarget()); + + List dataFilesAfter = TestHelpers.dataFiles(table, branch); + assertThat(dataFilesAfter) + .as("Data file should have been removed") + .hasSizeLessThan(dataFilesBefore.size()); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testDeleteWithPartitionedTable() throws Exception { + createAndInitPartitionedTable(); + + append(tableName, new Employee(1, "hr"), new Employee(3, "hr")); + append(tableName, new Employee(1, "hardware"), new Employee(2, "hardware")); + + // row level delete + sql("DELETE FROM %s WHERE id = 1", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(3, "hr")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + List rowLevelDeletePartitions = + spark.sql("SELECT * FROM " + tableName + ".partitions ").collectAsList(); + assertThat(rowLevelDeletePartitions) + .as("row level delete does not reduce number of partition") + .hasSize(2); + + // partition aligned delete + sql("DELETE FROM %s WHERE dep = 'hr'", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + List actualPartitions = + spark.sql("SELECT * FROM " + tableName + ".partitions ").collectAsList(); + assertThat(actualPartitions).as("partition aligned delete results in 1 partition").hasSize(1); + } + + @TestTemplate + public void testDeleteWithFalseCondition() { + createAndInitUnpartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableName); + createBranchIfNeeded(); + + sql("DELETE FROM %s WHERE id = 1 AND id > 20", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testDeleteFromEmptyTable() { + assumeThat(branch).as("Custom branch does not exist for empty table").isNotEqualTo("test"); + createAndInitUnpartitionedTable(); + + sql("DELETE FROM %s WHERE id IN (1)", commitTarget()); + sql("DELETE FROM %s WHERE dep = 'hr'", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); + + assertEquals( + "Should have expected rows", + ImmutableList.of(), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testDeleteFromNonExistingCustomBranch() { + assumeThat(branch).as("Test only applicable to custom branch").isEqualTo("test"); + createAndInitUnpartitionedTable(); + + assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id IN (1)", commitTarget())) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot use branch (does not exist): test"); + } + + @TestTemplate + public void testExplain() { + createAndInitUnpartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); + createBranchIfNeeded(); + + sql("EXPLAIN DELETE FROM %s WHERE id <=> 1", commitTarget()); + + sql("EXPLAIN DELETE FROM %s WHERE true", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 1 snapshot").hasSize(1); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", commitTarget())); + } + + @TestTemplate + public void testDeleteWithAlias() { + createAndInitUnpartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); + createBranchIfNeeded(); + + sql("DELETE FROM %s AS t WHERE t.id IS NULL", commitTarget()); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testDeleteWithDynamicFileFiltering() throws NoSuchTableException { + createAndInitPartitionedTable(); + + append(tableName, new Employee(1, "hr"), new Employee(3, "hr")); + createBranchIfNeeded(); + append(new Employee(1, "hardware"), new Employee(2, "hardware")); + + sql("DELETE FROM %s WHERE id = 2", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + if (mode(table) == COPY_ON_WRITE) { + validateCopyOnWrite(currentSnapshot, "1", "1", "1"); + } else { + validateMergeOnRead(currentSnapshot, "1", "1", null); + } + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hardware"), row(1, "hr"), row(3, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); + } + + @TestTemplate + public void testDeleteNonExistingRecords() { + createAndInitPartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); + createBranchIfNeeded(); + + sql("DELETE FROM %s AS t WHERE t.id > 10", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + + if (fileFormat.equals(FileFormat.ORC) || fileFormat.equals(FileFormat.PARQUET)) { + validateDelete(currentSnapshot, "0", null); + } else { + if (mode(table) == COPY_ON_WRITE) { + validateCopyOnWrite(currentSnapshot, "0", null, null); + } else { + validateMergeOnRead(currentSnapshot, "0", null, null); + } + } + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void deleteSingleRecordProducesDeleteOperation() throws NoSuchTableException { + createAndInitPartitionedTable(); + append(tableName, new Employee(1, "eng"), new Employee(2, "eng"), new Employee(3, "eng")); + + sql("DELETE FROM %s WHERE id = 2", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).hasSize(2); + + Snapshot currentSnapshot = table.currentSnapshot(); + + if (mode(table) == COPY_ON_WRITE) { + // this is an OverwriteFiles and produces "overwrite" + validateCopyOnWrite(currentSnapshot, "1", "1", "1"); + } else { + // this is a RowDelta that produces a "delete" instead of "overwrite" + validateMergeOnRead(currentSnapshot, "1", "1", null); + String property = formatVersion >= 3 ? ADDED_DVS_PROP : ADD_POS_DELETE_FILES_PROP; + validateProperty(currentSnapshot, property, "1"); + } + + assertThat(sql("SELECT * FROM %s", tableName)) + .containsExactlyInAnyOrder(row(1, "eng"), row(3, "eng")); + } + + @TestTemplate + public void testDeleteWithoutCondition() { + createAndInitPartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); + createBranchIfNeeded(); + sql("INSERT INTO TABLE %s VALUES (2, 'hardware')", commitTarget()); + sql("INSERT INTO TABLE %s VALUES (null, 'hr')", commitTarget()); + + sql("DELETE FROM %s", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 4 snapshots").hasSize(4); + + // should be a delete instead of an overwrite as it is done through a metadata operation + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + validateDelete(currentSnapshot, "2", "3"); + + assertEquals( + "Should have expected rows", ImmutableList.of(), sql("SELECT * FROM %s", commitTarget())); + } + + @TestTemplate + public void testDeleteUsingMetadataWithComplexCondition() { + createAndInitPartitionedTable(); + + sql("INSERT INTO %s VALUES (1, 'dep1')", tableName); + createBranchIfNeeded(); + sql("INSERT INTO %s VALUES (2, 'dep2')", commitTarget()); + sql("INSERT INTO %s VALUES (null, 'dep3')", commitTarget()); + + sql( + "DELETE FROM %s WHERE dep > 'dep2' OR dep = CAST(4 AS STRING) OR dep = 'dep2'", + commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 4 snapshots").hasSize(4); + + // should be a delete instead of an overwrite as it is done through a metadata operation + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + validateDelete(currentSnapshot, "2", "2"); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "dep1")), + sql("SELECT * FROM %s", selectTarget())); + } + + @TestTemplate + public void testDeleteWithArbitraryPartitionPredicates() { + createAndInitPartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); + createBranchIfNeeded(); + sql("INSERT INTO TABLE %s VALUES (2, 'hardware')", commitTarget()); + sql("INSERT INTO TABLE %s VALUES (null, 'hr')", commitTarget()); + + // %% is an escaped version of % + sql("DELETE FROM %s WHERE id = 10 OR dep LIKE '%%ware'", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 4 snapshots").hasSize(4); + + // should be a "delete" instead of an "overwrite" as only data files have been removed (COW) / + // delete files have been added (MOR) + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + assertThat(currentSnapshot.operation()).isEqualTo(DELETE); + if (mode(table) == COPY_ON_WRITE) { + validateCopyOnWrite(currentSnapshot, "1", "1", null); + } else { + validateMergeOnRead(currentSnapshot, "1", "1", null); + } + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testDeleteWithNonDeterministicCondition() { + createAndInitPartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableName); + createBranchIfNeeded(); + + assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id = 1 AND rand() > 0.5", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The operator expects a deterministic expression"); + } + + @TestTemplate + public void testDeleteWithFoldableConditions() { + createAndInitPartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableName); + createBranchIfNeeded(); + + // should keep all rows and don't trigger execution + sql("DELETE FROM %s WHERE false", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + // should keep all rows and don't trigger execution + sql("DELETE FROM %s WHERE 50 <> 50", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + // should keep all rows and don't trigger execution + sql("DELETE FROM %s WHERE 1 > null", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + // should remove all rows + sql("DELETE FROM %s WHERE 21 = 21", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); + } + + @TestTemplate + public void testDeleteWithNullConditions() { + createAndInitPartitionedTable(); + + sql( + "INSERT INTO TABLE %s VALUES (0, null), (1, 'hr'), (2, 'hardware'), (null, 'hr')", + tableName); + createBranchIfNeeded(); + + // should keep all rows as null is never equal to null + sql("DELETE FROM %s WHERE dep = null", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(0, null), row(1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + // null = 'software' -> null + // should delete using metadata operation only + sql("DELETE FROM %s WHERE dep = 'software'", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(0, null), row(1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + // should delete using metadata operation only + sql("DELETE FROM %s WHERE dep <=> NULL", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + validateDelete(currentSnapshot, "1", "1"); + } + + @TestTemplate + public void testDeleteWithInAndNotInConditions() { + createAndInitUnpartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); + createBranchIfNeeded(); + + sql("DELETE FROM %s WHERE id IN (1, null)", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql("DELETE FROM %s WHERE id NOT IN (null, 1)", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql("DELETE FROM %s WHERE id NOT IN (1, 10)", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testDeleteWithMultipleRowGroupsParquet() throws NoSuchTableException { + assumeThat(fileFormat).isEqualTo(FileFormat.PARQUET); + + createAndInitPartitionedTable(); + + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%d')", + tableName, PARQUET_ROW_GROUP_SIZE_BYTES, 100); + sql("ALTER TABLE %s SET TBLPROPERTIES('%s' '%d')", tableName, SPLIT_SIZE, 100); + + List ids = Lists.newArrayListWithCapacity(200); + for (int id = 1; id <= 200; id++) { + ids.add(id); + } + Dataset df = + spark + .createDataset(ids, Encoders.INT()) + .withColumnRenamed("value", "id") + .withColumn("dep", lit("hr")); + df.coalesce(1).writeTo(tableName).append(); + createBranchIfNeeded(); + + assertThat(spark.table(commitTarget()).count()).isEqualTo(200); + + // delete a record from one of two row groups and copy over the second one + sql("DELETE FROM %s WHERE id IN (200, 201)", commitTarget()); + + assertThat(spark.table(commitTarget()).count()).isEqualTo(199); + } + + @TestTemplate + public void testDeleteWithConditionOnNestedColumn() { + createAndInitNestedColumnsTable(); + + sql("INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", 3, \"c2\", \"v1\"))", tableName); + createBranchIfNeeded(); + sql("INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", 2, \"c2\", \"v2\"))", commitTarget()); + + sql("DELETE FROM %s WHERE complex.c1 = id + 2", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2)), + sql("SELECT id FROM %s", selectTarget())); + + sql("DELETE FROM %s t WHERE t.complex.c1 = id", commitTarget()); + assertEquals( + "Should have expected rows", ImmutableList.of(), sql("SELECT id FROM %s", selectTarget())); + } + + @TestTemplate + public void testDeleteWithInSubquery() throws NoSuchTableException { + createAndInitUnpartitionedTable(); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); + createBranchIfNeeded(); + + createOrReplaceView("deleted_id", Arrays.asList(0, 1, null), Encoders.INT()); + createOrReplaceView("deleted_dep", Arrays.asList("software", "hr"), Encoders.STRING()); + + sql( + "DELETE FROM %s WHERE id IN (SELECT * FROM deleted_id) AND dep IN (SELECT * from deleted_dep)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + append(new Employee(1, "hr"), new Employee(-1, "hr")); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql( + "DELETE FROM %s WHERE id IS NULL OR id IN (SELECT value + 2 FROM deleted_id)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(1, "hr")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + append(new Employee(null, "hr"), new Employee(2, "hr")); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(1, "hr"), row(2, "hr"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql( + "DELETE FROM %s WHERE id IN (SELECT value + 2 FROM deleted_id) AND dep = 'hr'", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(1, "hr"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testDeleteWithMultiColumnInSubquery() throws NoSuchTableException { + createAndInitUnpartitionedTable(); + + append(tableName, new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); + createBranchIfNeeded(); + + List deletedEmployees = + Arrays.asList(new Employee(null, "hr"), new Employee(1, "hr")); + createOrReplaceView("deleted_employee", deletedEmployees, Encoders.bean(Employee.class)); + + sql("DELETE FROM %s WHERE (id, dep) IN (SELECT id, dep FROM deleted_employee)", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testDeleteWithNotInSubquery() throws NoSuchTableException { + createAndInitUnpartitionedTable(); + + append(tableName, new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); + createBranchIfNeeded(); + + createOrReplaceView("deleted_id", Arrays.asList(-1, -2, null), Encoders.INT()); + createOrReplaceView("deleted_dep", Arrays.asList("software", "hr"), Encoders.STRING()); + + // the file filter subquery (nested loop lef-anti join) returns 0 records + sql("DELETE FROM %s WHERE id NOT IN (SELECT * FROM deleted_id)", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql( + "DELETE FROM %s WHERE id NOT IN (SELECT * FROM deleted_id WHERE value IS NOT NULL)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware"), row(null, "hr"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql( + "DELETE FROM %s WHERE id NOT IN (SELECT * FROM deleted_id) OR dep IN ('software', 'hr')", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql( + "DELETE FROM %s t WHERE " + + "id NOT IN (SELECT * FROM deleted_id WHERE value IS NOT NULL) AND " + + "EXISTS (SELECT 1 FROM FROM deleted_dep WHERE t.dep = deleted_dep.value)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql( + "DELETE FROM %s t WHERE " + + "id NOT IN (SELECT * FROM deleted_id WHERE value IS NOT NULL) OR " + + "EXISTS (SELECT 1 FROM FROM deleted_dep WHERE t.dep = deleted_dep.value)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testDeleteOnNonIcebergTableNotSupported() { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + + sql("CREATE TABLE parquet_table (c1 INT, c2 INT) USING parquet"); + + assertThatThrownBy(() -> sql("DELETE FROM parquet_table WHERE c1 = -100")) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("does not support DELETE"); + } + + @TestTemplate + public void testDeleteWithExistSubquery() throws NoSuchTableException { + createAndInitUnpartitionedTable(); + + append(tableName, new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); + createBranchIfNeeded(); + + createOrReplaceView("deleted_id", Arrays.asList(-1, -2, null), Encoders.INT()); + createOrReplaceView("deleted_dep", Arrays.asList("software", "hr"), Encoders.STRING()); + + sql( + "DELETE FROM %s t WHERE EXISTS (SELECT 1 FROM deleted_id d WHERE t.id = d.value)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql( + "DELETE FROM %s t WHERE EXISTS (SELECT 1 FROM deleted_id d WHERE t.id = d.value + 2)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql( + "DELETE FROM %s t WHERE EXISTS (SELECT 1 FROM deleted_id d WHERE t.id = d.value) OR t.id IS NULL", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware")), + sql("SELECT * FROM %s", selectTarget())); + + sql( + "DELETE FROM %s t WHERE " + + "EXISTS (SELECT 1 FROM deleted_id di WHERE t.id = di.value) AND " + + "EXISTS (SELECT 1 FROM deleted_dep dd WHERE t.dep = dd.value)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware")), + sql("SELECT * FROM %s", selectTarget())); + } + + @TestTemplate + public void testDeleteWithNotExistsSubquery() throws NoSuchTableException { + createAndInitUnpartitionedTable(); + + append(tableName, new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); + createBranchIfNeeded(); + + createOrReplaceView("deleted_id", Arrays.asList(-1, -2, null), Encoders.INT()); + createOrReplaceView("deleted_dep", Arrays.asList("software", "hr"), Encoders.STRING()); + + sql( + "DELETE FROM %s t WHERE " + + "NOT EXISTS (SELECT 1 FROM deleted_id di WHERE t.id = di.value + 2) AND " + + "NOT EXISTS (SELECT 1 FROM deleted_dep dd WHERE t.dep = dd.value)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql( + "DELETE FROM %s t WHERE NOT EXISTS (SELECT 1 FROM deleted_id d WHERE t.id = d.value + 2)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + String subquery = "SELECT 1 FROM deleted_id d WHERE t.id = d.value + 2"; + sql("DELETE FROM %s t WHERE NOT EXISTS (%s) OR t.id = 1", commitTarget(), subquery); + assertEquals( + "Should have expected rows", + ImmutableList.of(), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testDeleteWithScalarSubquery() throws NoSuchTableException { + createAndInitUnpartitionedTable(); + + append(tableName, new Employee(1, "hr"), new Employee(2, "hardware"), new Employee(null, "hr")); + createBranchIfNeeded(); + + createOrReplaceView("deleted_id", Arrays.asList(1, 100, null), Encoders.INT()); + + // TODO: Spark does not support AQE and DPP with aggregates at the moment + withSQLConf( + ImmutableMap.of(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), "false"), + () -> { + sql("DELETE FROM %s t WHERE id <= (SELECT min(value) FROM deleted_id)", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + }); + } + + @TestTemplate + public void testDeleteThatRequiresGroupingBeforeWrite() throws NoSuchTableException { + createAndInitPartitionedTable(); + + append(tableName, new Employee(0, "hr"), new Employee(1, "hr"), new Employee(2, "hr")); + createBranchIfNeeded(); + append(new Employee(0, "ops"), new Employee(1, "ops"), new Employee(2, "ops")); + append(new Employee(0, "hr"), new Employee(1, "hr"), new Employee(2, "hr")); + append(new Employee(0, "ops"), new Employee(1, "ops"), new Employee(2, "ops")); + + createOrReplaceView("deleted_id", Arrays.asList(1, 100), Encoders.INT()); + + String originalNumOfShufflePartitions = spark.conf().get("spark.sql.shuffle.partitions"); + try { + // set the num of shuffle partitions to 1 to ensure we have only 1 writing task + spark.conf().set("spark.sql.shuffle.partitions", "1"); + + sql("DELETE FROM %s t WHERE id IN (SELECT * FROM deleted_id)", commitTarget()); + assertThat(spark.table(commitTarget()).count()) + .as("Should have expected num of rows") + .isEqualTo(8L); + } finally { + spark.conf().set("spark.sql.shuffle.partitions", originalNumOfShufflePartitions); + } + } + + @TestTemplate + public synchronized void testDeleteWithSerializableIsolation() throws InterruptedException { + // cannot run tests with concurrency for Hadoop tables without atomic renames + assumeThat(catalogName).isNotEqualToIgnoringCase("testhadoop"); + // if caching is off, the table is eagerly refreshed during runtime filtering + // this can cause a validation exception as concurrent changes would be visible + assumeThat(cachingCatalogEnabled()).isTrue(); + + createAndInitUnpartitionedTable(); + createOrReplaceView("deleted_id", Collections.singletonList(1), Encoders.INT()); + + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + tableName, DELETE_ISOLATION_LEVEL, "serializable"); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); + createBranchIfNeeded(); + + 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( + () -> { + // load the table via the validation catalog to use another table instance + Table table = validationCatalog.loadTable(tableIdent); + + 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(ValidationException.class) + .hasMessageContaining("Found conflicting files that can contain"); + } finally { + shouldAppend.set(false); + appendFuture.cancel(true); + } + + executorService.shutdown(); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); + } + + @TestTemplate + public synchronized void testDeleteWithSnapshotIsolation() + throws InterruptedException, ExecutionException { + // cannot run tests with concurrency for Hadoop tables without atomic renames + assumeThat(catalogName).isNotEqualToIgnoringCase("testhadoop"); + // if caching is off, the table is eagerly refreshed during runtime filtering + // this can cause a validation exception as concurrent changes would be visible + assumeThat(cachingCatalogEnabled()).isTrue(); + + createAndInitUnpartitionedTable(); + createOrReplaceView("deleted_id", Collections.singletonList(1), Encoders.INT()); + + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s'='%s', '%s'='%s', '%s'='%s', '%s'='%s')", + tableName, + DELETE_ISOLATION_LEVEL, + "snapshot", + COMMIT_MIN_RETRY_WAIT_MS, + "10", + COMMIT_MAX_RETRY_WAIT_MS, + "1000", + COMMIT_NUM_RETRIES, + "7"); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); + createBranchIfNeeded(); + + 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 < 20; 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( + () -> { + // load the table via the validation catalog to use another table instance for inserts + Table table = validationCatalog.loadTable(tableIdent); + + GenericRecord record = GenericRecord.create(SnapshotUtil.schemaFor(table, branch)); + record.set(0, 1); // id + record.set(1, "hr"); // dep + + for (int numOperations = 0; numOperations < 20; 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 { + deleteFuture.get(); + } finally { + shouldAppend.set(false); + appendFuture.cancel(true); + } + + executorService.shutdown(); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); + } + + @TestTemplate + public void testDeleteRefreshesRelationCache() throws NoSuchTableException { + createAndInitPartitionedTable(); + + append(tableName, new Employee(1, "hr"), new Employee(3, "hr")); + createBranchIfNeeded(); + append(new Employee(1, "hardware"), new Employee(2, "hardware")); + + Dataset query = spark.sql("SELECT * FROM " + commitTarget() + " WHERE id = 1"); + query.createOrReplaceTempView("tmp"); + + spark.sql("CACHE TABLE tmp"); + + assertEquals( + "View should have correct data", + ImmutableList.of(row(1, "hardware"), row(1, "hr")), + sql("SELECT * FROM tmp ORDER BY id, dep")); + + sql("DELETE FROM %s WHERE id = 1", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + if (mode(table) == COPY_ON_WRITE) { + validateCopyOnWrite(currentSnapshot, "2", "2", "2"); + } else { + validateMergeOnRead(currentSnapshot, "2", "2", null); + } + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(3, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", commitTarget())); + + assertEquals( + "Should refresh the relation cache", + ImmutableList.of(), + sql("SELECT * FROM tmp ORDER BY id, dep")); + + spark.sql("UNCACHE TABLE tmp"); + } + + @TestTemplate + public void testDeleteWithMultipleSpecs() { + createAndInitTable("id INT, dep STRING, category STRING"); + + // write an unpartitioned file + append(tableName, "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\"}"); + createBranchIfNeeded(); + + // write a file partitioned by dep + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + append( + commitTarget(), + "{ \"id\": 2, \"dep\": \"hr\", \"category\": \"c1\" }\n" + + "{ \"id\": 3, \"dep\": \"hr\", \"category\": \"c1\" }"); + + // write a file partitioned by dep and category + sql("ALTER TABLE %s ADD PARTITION FIELD category", tableName); + append(commitTarget(), "{ \"id\": 5, \"dep\": \"hr\", \"category\": \"c1\"}"); + + // write another file partitioned by dep + sql("ALTER TABLE %s DROP PARTITION FIELD category", tableName); + append(commitTarget(), "{ \"id\": 7, \"dep\": \"hr\", \"category\": \"c1\"}"); + + sql("DELETE FROM %s WHERE id IN (1, 3, 5, 7)", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 5 snapshots").hasSize(5); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + if (mode(table) == COPY_ON_WRITE) { + validateCopyOnWrite(currentSnapshot, "3", "4", "1"); + } else { + validateMergeOnRead(currentSnapshot, "3", "4", null); + } + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hr", "c1")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testDeleteToWapBranch() throws NoSuchTableException { + assumeThat(branch).as("WAP branch only works for table identifier without branch").isNull(); + + createAndInitPartitionedTable(); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' = 'true')", + tableName, TableProperties.WRITE_AUDIT_PUBLISH_ENABLED); + append(new Employee(0, "hr"), new Employee(1, "hr"), new Employee(2, "hr")); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), + () -> { + sql("DELETE FROM %s t WHERE id=0", tableName); + assertThat(spark.table(tableName).count()) + .as("Should have expected num of rows when reading table") + .isEqualTo(2L); + assertThat(spark.table(tableName + ".branch_wap").count()) + .as("Should have expected num of rows when reading WAP branch") + .isEqualTo(2L); + assertThat(spark.table(tableName + ".branch_main").count()) + .as("Should not modify main branch") + .isEqualTo(3L); + }); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), + () -> { + sql("DELETE FROM %s t WHERE id=1", tableName); + assertThat(spark.table(tableName).count()) + .as("Should have expected num of rows when reading table with multiple writes") + .isEqualTo(1L); + assertThat(spark.table(tableName + ".branch_wap").count()) + .as("Should have expected num of rows when reading WAP branch with multiple writes") + .isEqualTo(1L); + assertThat(spark.table(tableName + ".branch_main").count()) + .as("Should not modify main branch with multiple writes") + .isEqualTo(3L); + }); + } + + @TestTemplate + public void testDeleteToWapBranchWithTableBranchIdentifier() throws NoSuchTableException { + assumeThat(branch).as("Test must have branch name part in table identifier").isNotNull(); + + createAndInitPartitionedTable(); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' = 'true')", + tableName, TableProperties.WRITE_AUDIT_PUBLISH_ENABLED); + append(tableName, new Employee(0, "hr"), new Employee(1, "hr"), new Employee(2, "hr")); + createBranchIfNeeded(); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), + () -> + assertThatThrownBy(() -> sql("DELETE FROM %s t WHERE id=0", commitTarget())) + .isInstanceOf(ValidationException.class) + .hasMessage( + String.format( + "Cannot write to both branch and WAP branch, but got branch [%s] and WAP branch [wap]", + branch))); + } + + @TestTemplate + public void testDeleteToCustomWapBranchWithoutWhereClause() throws NoSuchTableException { + assumeThat(branch) + .as("Run only if custom WAP branch is not main") + .isNotNull() + .isNotEqualTo(SnapshotRef.MAIN_BRANCH); + + createAndInitPartitionedTable(); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' = 'true')", + tableName, TableProperties.WRITE_AUDIT_PUBLISH_ENABLED); + append(tableName, new Employee(0, "hr"), new Employee(1, "hr"), new Employee(2, "hr")); + createBranchIfNeeded(); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, branch), + () -> { + sql("DELETE FROM %s t WHERE id=1", tableName); + assertThat(spark.table(tableName).count()).isEqualTo(2L); + assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(2L); + assertThat(spark.table(tableName + ".branch_main").count()) + .as("Should not modify main branch") + .isEqualTo(3L); + }); + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, branch), + () -> { + sql("DELETE FROM %s t", tableName); + assertThat(spark.table(tableName).count()).isEqualTo(0L); + assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(0L); + assertThat(spark.table(tableName + ".branch_main").count()) + .as("Should not modify main branch") + .isEqualTo(3L); + }); + } + + @TestTemplate + public void testDeleteWithFilterOnNestedColumn() { + createAndInitNestedColumnsTable(); + + sql("INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", 3, \"c2\", \"v1\"))", tableName); + sql("INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", 2, \"c2\", \"v2\"))", tableName); + + sql("DELETE FROM %s WHERE complex.c1 > 3", tableName); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1), row(2)), + sql("SELECT id FROM %s order by id", tableName)); + + sql("DELETE FROM %s WHERE complex.c1 = 3", tableName); + assertEquals( + "Should have expected rows", ImmutableList.of(row(2)), sql("SELECT id FROM %s", tableName)); + + sql("DELETE FROM %s t WHERE t.complex.c1 = 2", tableName); + assertEquals( + "Should have expected rows", ImmutableList.of(), sql("SELECT id FROM %s", tableName)); + } + + // TODO: multiple stripes for ORC + + protected void createAndInitPartitionedTable() { + sql("CREATE TABLE %s (id INT, dep STRING) USING iceberg PARTITIONED BY (dep)", tableName); + initTable(); + } + + protected void createAndInitUnpartitionedTable() { + sql("CREATE TABLE %s (id INT, dep STRING) USING iceberg", tableName); + initTable(); + } + + protected void createAndInitNestedColumnsTable() { + sql("CREATE TABLE %s (id INT, complex STRUCT) USING iceberg", tableName); + initTable(); + } + + protected void append(Employee... employees) throws NoSuchTableException { + append(commitTarget(), employees); + } + + protected void append(String target, Employee... employees) throws NoSuchTableException { + List input = Arrays.asList(employees); + Dataset inputDF = spark.createDataFrame(input, Employee.class); + inputDF.coalesce(1).writeTo(target).append(); + } + + private RowLevelOperationMode mode(Table table) { + String modeName = table.properties().getOrDefault(DELETE_MODE, DELETE_MODE_DEFAULT); + return RowLevelOperationMode.fromName(modeName); + } + + private LogicalPlan parsePlan(String query, Object... args) { + try { + return spark.sessionState().sqlParser().parsePlan(String.format(query, args)); + } catch (ParseException e) { + throw new RuntimeException(e); + } + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java new file mode 100644 index 000000000000..c116cb4f857f --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestExpireSnapshotsProcedure.java @@ -0,0 +1,624 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.TableProperties.GC_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.sql.Timestamp; +import java.time.Instant; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionStatisticsFile; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Encoders; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestExpireSnapshotsProcedure extends ExtensionsTestBase { + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testExpireSnapshotsInEmptyTable() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + List output = sql("CALL %s.system.expire_snapshots('%s')", catalogName, tableIdent); + assertEquals( + "Should not delete any files", ImmutableList.of(row(0L, 0L, 0L, 0L, 0L, 0L)), output); + } + + @TestTemplate + public void testExpireSnapshotsUsingPositionalArgs() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + + waitUntilAfter(firstSnapshot.timestampMillis()); + + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + Timestamp secondSnapshotTimestamp = + Timestamp.from(Instant.ofEpochMilli(secondSnapshot.timestampMillis())); + + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); + + // expire without retainLast param + List output1 = + sql( + "CALL %s.system.expire_snapshots('%s', TIMESTAMP '%s')", + catalogName, tableIdent, secondSnapshotTimestamp); + assertEquals( + "Procedure output must match", ImmutableList.of(row(0L, 0L, 0L, 0L, 1L, 0L)), output1); + + table.refresh(); + + assertThat(table.snapshots()).as("Should expire one snapshot").hasSize(1); + + sql("INSERT OVERWRITE %s VALUES (3, 'c')", tableName); + sql("INSERT INTO TABLE %s VALUES (4, 'd')", tableName); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(3L, "c"), row(4L, "d")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + table.refresh(); + + waitUntilAfter(table.currentSnapshot().timestampMillis()); + + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + + assertThat(table.snapshots()).as("Should be 3 snapshots").hasSize(3); + + // expire with retainLast param + List output = + sql( + "CALL %s.system.expire_snapshots('%s', TIMESTAMP '%s', 2)", + catalogName, tableIdent, currentTimestamp); + assertEquals( + "Procedure output must match", ImmutableList.of(row(2L, 0L, 0L, 2L, 1L, 0L)), output); + } + + @TestTemplate + public void testExpireSnapshotUsingNamedArgs() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); + + waitUntilAfter(table.currentSnapshot().timestampMillis()); + + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + + List output = + sql( + "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", + catalogName, currentTimestamp, tableIdent); + assertEquals( + "Procedure output must match", ImmutableList.of(row(0L, 0L, 0L, 0L, 1L, 0L)), output); + } + + @TestTemplate + public void testExpireSnapshotsGCDisabled() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED); + + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('%s')", catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot expire snapshots: GC is disabled"); + } + + @TestTemplate + public void testInvalidExpireSnapshotsCases() { + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('n', table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.BOTH_POSITIONAL_AND_NAMED] Call to routine `expire_snapshots` is invalid because it includes multiple argument assignments to the same parameter name `table`. A positional argument and named argument both referred to the same parameter. Please remove the named argument referring to this parameter. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[FAILED_TO_LOAD_ROUTINE] Failed to load routine `%s`.`custom`.`expire_snapshots`. SQLSTATE: 38000", + catalogName); + + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots()", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `expire_snapshots` because the parameter named `table` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('n', 2.2)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "[DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE] Cannot resolve CALL due to data type mismatch: The second parameter requires the \"TIMESTAMP\" type, however \"2.2\" has the type \"DECIMAL(2,1)\". SQLSTATE: 42K09"); + + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for parameter 'table'"); + } + + @TestTemplate + public void testResolvingTableInAnotherCatalog() throws IOException { + String anotherCatalog = "another_" + catalogName; + spark.conf().set("spark.sql.catalog." + anotherCatalog, SparkCatalog.class.getName()); + spark.conf().set("spark.sql.catalog." + anotherCatalog + ".type", "hadoop"); + spark + .conf() + .set( + "spark.sql.catalog." + anotherCatalog + ".warehouse", + Files.createTempDirectory(temp, "junit").toFile().toURI().toString()); + + sql( + "CREATE TABLE %s.%s (id bigint NOT NULL, data string) USING iceberg", + anotherCatalog, tableIdent); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.expire_snapshots('%s')", + catalogName, anotherCatalog + "." + tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot run procedure in catalog"); + } + + @TestTemplate + public void testConcurrentExpireSnapshots() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + sql("INSERT INTO TABLE %s VALUES (3, 'c')", tableName); + sql("INSERT INTO TABLE %s VALUES (4, 'd')", tableName); + + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + List output = + sql( + "CALL %s.system.expire_snapshots(" + + "older_than => TIMESTAMP '%s'," + + "table => '%s'," + + "max_concurrent_deletes => %s)", + catalogName, currentTimestamp, tableIdent, 4); + assertEquals( + "Expiring snapshots concurrently should succeed", + ImmutableList.of(row(0L, 0L, 0L, 0L, 3L, 0L)), + output); + } + + @TestTemplate + public void testConcurrentExpireSnapshotsWithInvalidInput() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", + catalogName, tableIdent, 0)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("max_concurrent_deletes should have value > 0, value: 0"); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", + catalogName, tableIdent, -1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("max_concurrent_deletes should have value > 0, value: -1"); + } + + @TestTemplate + public void testExpireDeleteFiles() throws Exception { + sql( + "CREATE TABLE %s (id bigint, data string) USING iceberg TBLPROPERTIES" + + "('format-version'='2', 'write.delete.mode'='merge-on-read')", + tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + sql("DELETE FROM %s WHERE id=1", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(TestHelpers.deleteManifests(table)).as("Should have 1 delete manifest").hasSize(1); + assertThat(TestHelpers.deleteFiles(table)).as("Should have 1 delete file").hasSize(1); + Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); + DeleteFile deleteFile = TestHelpers.deleteFiles(table).iterator().next(); + Path deleteFilePath = new Path(deleteFile.location()); + + sql( + "CALL %s.system.rewrite_data_files(" + + "table => '%s'," + + "options => map(" + + "'delete-file-threshold','1'," + + "'use-starting-sequence-number', 'false'))", + catalogName, tableIdent); + table.refresh(); + + table + .newRowDelta() + .removeDeletes(deleteFile) + .commit(); // this txn moves the file to the DELETED state + sql("INSERT INTO TABLE %s VALUES (6, 'f')", tableName); // this txn removes the file reference + table.refresh(); + + assertThat(TestHelpers.deleteManifests(table)).as("Should have no delete manifests").isEmpty(); + assertThat(TestHelpers.deleteFiles(table)).as("Should have no delete files").isEmpty(); + + FileSystem localFs = FileSystem.getLocal(new Configuration()); + assertThat(localFs.exists(deleteManifestPath)) + .as("Delete manifest should still exist") + .isTrue(); + assertThat(localFs.exists(deleteFilePath)).as("Delete file should still exist").isTrue(); + + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + List output = + sql( + "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", + catalogName, currentTimestamp, tableIdent); + + assertEquals( + "Should deleted 1 data and pos delete file and 4 manifests and lists (one for each txn)", + ImmutableList.of(row(1L, 1L, 0L, 4L, 4L, 0L)), + output); + assertThat(localFs.exists(deleteManifestPath)) + .as("Delete manifest should be removed") + .isFalse(); + assertThat(localFs.exists(deleteFilePath)).as("Delete file should be removed").isFalse(); + } + + @TestTemplate + public void testExpireSnapshotWithStreamResultsEnabled() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); + + waitUntilAfter(table.currentSnapshot().timestampMillis()); + + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + + List output = + sql( + "CALL %s.system.expire_snapshots(" + + "older_than => TIMESTAMP '%s'," + + "table => '%s'," + + "stream_results => true)", + catalogName, currentTimestamp, tableIdent); + assertEquals( + "Procedure output must match", ImmutableList.of(row(0L, 0L, 0L, 0L, 1L, 0L)), output); + } + + @TestTemplate + public void testExpireSnapshotsWithSnapshotId() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); + + // Expiring the snapshot specified by snapshot_id should keep only a single snapshot. + long firstSnapshotId = table.currentSnapshot().parentId(); + sql( + "CALL %s.system.expire_snapshots(" + "table => '%s'," + "snapshot_ids => ARRAY(%d))", + catalogName, tableIdent, firstSnapshotId); + + // There should only be one single snapshot left. + table.refresh(); + assertThat(table.snapshots()) + .hasSize(1) + .as("Snapshot ID should not be present") + .filteredOn(snapshot -> snapshot.snapshotId() == firstSnapshotId) + .isEmpty(); + } + + @TestTemplate + public void testExpireSnapshotShouldFailForCurrentSnapshot() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.expire_snapshots(" + + "table => '%s'," + + "snapshot_ids => ARRAY(%d, %d))", + catalogName, + tableIdent, + table.currentSnapshot().snapshotId(), + table.currentSnapshot().parentId())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot expire"); + } + + @TestTemplate + public void testExpireSnapshotsProcedureWorksWithSqlComments() { + // Ensure that systems such as dbt, that inject comments into the generated SQL files, will + // work with Iceberg-specific DDL + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); + + waitUntilAfter(table.currentSnapshot().timestampMillis()); + + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + + String callStatement = + "/* CALL statement is used to expire snapshots */\n" + + "-- And we have single line comments as well \n" + + "/* And comments that span *multiple* \n" + + " lines */ CALL /* this is the actual CALL */ %s.system.expire_snapshots(" + + " older_than => TIMESTAMP '%s'," + + " table => '%s')"; + List output = sql(callStatement, catalogName, currentTimestamp, tableIdent); + assertEquals( + "Procedure output must match", ImmutableList.of(row(0L, 0L, 0L, 0L, 1L, 0L)), output); + + table.refresh(); + + assertThat(table.snapshots()).as("Should be 1 snapshot remaining").hasSize(1); + } + + @TestTemplate + public void testExpireSnapshotsWithStatisticFiles() throws Exception { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (10, 'abc')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + String statsFileLocation1 = ProcedureUtil.statsFileLocation(table.location()); + StatisticsFile statisticsFile1 = + writeStatsFile( + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + statsFileLocation1, + table.io()); + table.updateStatistics().setStatistics(statisticsFile1).commit(); + + sql("INSERT INTO %s SELECT 20, 'def'", tableName); + table.refresh(); + String statsFileLocation2 = ProcedureUtil.statsFileLocation(table.location()); + StatisticsFile statisticsFile2 = + writeStatsFile( + table.currentSnapshot().snapshotId(), + table.currentSnapshot().sequenceNumber(), + statsFileLocation2, + table.io()); + table.updateStatistics().setStatistics(statisticsFile2).commit(); + + waitUntilAfter(table.currentSnapshot().timestampMillis()); + + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + List output = + sql( + "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", + catalogName, currentTimestamp, tableIdent); + assertThat(output.get(0)[5]).as("should be 1 deleted statistics file").isEqualTo(1L); + + table.refresh(); + assertThat(table.statisticsFiles()) + .as( + "Statistics file entry in TableMetadata should be present only for the snapshot %s", + statisticsFile2.snapshotId()) + .extracting(StatisticsFile::snapshotId) + .containsExactly(statisticsFile2.snapshotId()); + + assertThat(new File(statsFileLocation1)) + .as("Statistics file should not exist for snapshot %s", statisticsFile1.snapshotId()) + .doesNotExist(); + + assertThat(new File(statsFileLocation2)) + .as("Statistics file should exist for snapshot %s", statisticsFile2.snapshotId()) + .exists(); + } + + @TestTemplate + public void testExpireSnapshotsWithPartitionStatisticFiles() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (10, 'abc')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + String partitionStatsFileLocation1 = ProcedureUtil.statsFileLocation(table.location()); + PartitionStatisticsFile partitionStatisticsFile1 = + ProcedureUtil.writePartitionStatsFile( + table.currentSnapshot().snapshotId(), partitionStatsFileLocation1, table.io()); + table.updatePartitionStatistics().setPartitionStatistics(partitionStatisticsFile1).commit(); + + sql("INSERT INTO %s SELECT 20, 'def'", tableName); + table.refresh(); + String partitionStatsFileLocation2 = ProcedureUtil.statsFileLocation(table.location()); + PartitionStatisticsFile partitionStatisticsFile2 = + ProcedureUtil.writePartitionStatsFile( + table.currentSnapshot().snapshotId(), partitionStatsFileLocation2, table.io()); + table.updatePartitionStatistics().setPartitionStatistics(partitionStatisticsFile2).commit(); + + waitUntilAfter(table.currentSnapshot().timestampMillis()); + + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + List output = + sql( + "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", + catalogName, currentTimestamp, tableIdent); + assertThat(output.get(0)[5]).as("should be 1 deleted partition statistics file").isEqualTo(1L); + + table.refresh(); + assertThat(table.partitionStatisticsFiles()) + .as( + "partition statistics file entry in TableMetadata should be present only for the snapshot %s", + partitionStatisticsFile2.snapshotId()) + .extracting(PartitionStatisticsFile::snapshotId) + .containsExactly(partitionStatisticsFile2.snapshotId()); + + assertThat(new File(partitionStatsFileLocation1)) + .as( + "partition statistics file should not exist for snapshot %s", + partitionStatisticsFile1.snapshotId()) + .doesNotExist(); + + assertThat(new File(partitionStatsFileLocation2)) + .as( + "partition statistics file should exist for snapshot %s", + partitionStatisticsFile2.snapshotId()) + .exists(); + } + + @TestTemplate + public void testNoExpiredMetadataCleanupByDefault() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("ALTER TABLE %s ADD COLUMN extra_col int", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b', 21)", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); + assertThat(table.schemas()).as("Should have 2 schemas").hasSize(2); + + waitUntilAfter(table.currentSnapshot().timestampMillis()); + + List output = + sql( + "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s', table => '%s')", + catalogName, + Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())), + tableIdent); + + table.refresh(); + assertThat(table.schemas()).as("Should have 2 schemas").hasSize(2); + assertEquals( + "Procedure output must match", ImmutableList.of(row(0L, 0L, 0L, 0L, 1L, 0L)), output); + } + + @TestTemplate + public void testCleanExpiredMetadata() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("ALTER TABLE %s ADD COLUMN extra_col int", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b', 21)", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); + assertThat(table.schemas()).as("Should have 2 schemas").hasSize(2); + + waitUntilAfter(table.currentSnapshot().timestampMillis()); + + List output = + sql( + "CALL %s.system.expire_snapshots(" + + "older_than => TIMESTAMP '%s', " + + "clean_expired_metadata => true, " + + "table => '%s')", + catalogName, + Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())), + tableIdent); + + table.refresh(); + + assertThat(table.schemas().keySet()) + .as("Should have only the latest schema") + .containsExactly(table.schema().schemaId()); + assertEquals( + "Procedure output must match", ImmutableList.of(row(0L, 0L, 0L, 0L, 1L, 0L)), output); + } + + private static StatisticsFile writeStatsFile( + long snapshotId, long snapshotSequenceNumber, String statsLocation, FileIO fileIO) + throws IOException { + try (PuffinWriter puffinWriter = Puffin.write(fileIO.newOutputFile(statsLocation)).build()) { + puffinWriter.add( + new Blob( + "some-blob-type", + ImmutableList.of(1), + snapshotId, + snapshotSequenceNumber, + ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8)))); + puffinWriter.finish(); + + return new GenericStatisticsFile( + snapshotId, + statsLocation, + puffinWriter.fileSize(), + puffinWriter.footerSize(), + puffinWriter.writtenBlobsMetadata().stream() + .map(GenericBlobMetadata::from) + .collect(ImmutableList.toImmutableList())); + } + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java new file mode 100644 index 000000000000..69920e1d5402 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java @@ -0,0 +1,264 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.spark.sql.AnalysisException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestFastForwardBranchProcedure extends ExtensionsTestBase { + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testFastForwardBranchUsingPositionalArgs() { + sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + table.refresh(); + + Snapshot currSnapshot = table.currentSnapshot(); + long sourceRef = currSnapshot.snapshotId(); + + String newBranch = "testBranch"; + String tableNameWithBranch = String.format("%s.branch_%s", tableName, newBranch); + + sql("ALTER TABLE %s CREATE BRANCH %s", tableName, newBranch); + sql("INSERT INTO TABLE %s VALUES(3,'c')", tableNameWithBranch); + + table.refresh(); + long updatedRef = table.snapshot(newBranch).snapshotId(); + + assertEquals( + "Main branch should not have the newly inserted record.", + ImmutableList.of(row(1, "a"), row(2, "b")), + sql("SELECT * FROM %s order by id", tableName)); + + assertEquals( + "Test branch should have the newly inserted record.", + ImmutableList.of(row(1, "a"), row(2, "b"), row(3, "c")), + sql("SELECT * FROM %s order by id", tableNameWithBranch)); + + List output = + sql( + "CALL %s.system.fast_forward('%s', '%s', '%s')", + catalogName, tableIdent, SnapshotRef.MAIN_BRANCH, newBranch); + + assertThat(Arrays.stream(output.get(0)).collect(Collectors.toList()).get(0)) + .isEqualTo(SnapshotRef.MAIN_BRANCH); + + assertThat(Arrays.stream(output.get(0)).collect(Collectors.toList()).get(1)) + .isEqualTo(sourceRef); + + assertThat(Arrays.stream(output.get(0)).collect(Collectors.toList()).get(2)) + .isEqualTo(updatedRef); + + assertEquals( + "Main branch should have the newly inserted record.", + ImmutableList.of(row(1, "a"), row(2, "b"), row(3, "c")), + sql("SELECT * FROM %s order by id", tableName)); + } + + @TestTemplate + public void testFastForwardBranchUsingNamedArgs() { + sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + String newBranch = "testBranch"; + String tableNameWithBranch = String.format("%s.branch_%s", tableName, newBranch); + + sql("ALTER TABLE %s CREATE BRANCH %s", tableName, newBranch); + sql("INSERT INTO TABLE %s VALUES(3,'c')", tableNameWithBranch); + + assertEquals( + "Main branch should not have the newly inserted record.", + ImmutableList.of(row(1, "a"), row(2, "b")), + sql("SELECT * FROM %s order by id", tableName)); + + assertEquals( + "Test branch should have the newly inserted record.", + ImmutableList.of(row(1, "a"), row(2, "b"), row(3, "c")), + sql("SELECT * FROM %s order by id", tableNameWithBranch)); + + List output = + sql( + "CALL %s.system.fast_forward(table => '%s', branch => '%s', to => '%s')", + catalogName, tableIdent, SnapshotRef.MAIN_BRANCH, newBranch); + + assertEquals( + "Main branch should now have the newly inserted record.", + ImmutableList.of(row(1, "a"), row(2, "b"), row(3, "c")), + sql("SELECT * FROM %s order by id", tableName)); + } + + @TestTemplate + public void testFastForwardWhenTargetIsNotAncestorFails() { + sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + String newBranch = "testBranch"; + String tableNameWithBranch = String.format("%s.branch_%s", tableName, newBranch); + + sql("ALTER TABLE %s CREATE BRANCH %s", tableName, newBranch); + sql("INSERT INTO TABLE %s VALUES(3,'c')", tableNameWithBranch); + + assertEquals( + "Main branch should not have the newly inserted record.", + ImmutableList.of(row(1, "a"), row(2, "b")), + sql("SELECT * FROM %s order by id", tableName)); + + assertEquals( + "Test branch should have the newly inserted record.", + ImmutableList.of(row(1, "a"), row(2, "b"), row(3, "c")), + sql("SELECT * FROM %s order by id", tableNameWithBranch)); + + // Commit a snapshot on main to deviate the branches + sql("INSERT INTO TABLE %s VALUES (4, 'd')", tableName); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.fast_forward(table => '%s', branch => '%s', to => '%s')", + catalogName, tableIdent, SnapshotRef.MAIN_BRANCH, newBranch)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot fast-forward: main is not an ancestor of testBranch"); + } + + @TestTemplate + public void testInvalidFastForwardBranchCases() { + assertThatThrownBy( + () -> + sql( + "CALL %s.system.fast_forward(table => 'test_table', 'main', to => 'newBranch')", + catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "[UNEXPECTED_POSITIONAL_ARGUMENT] Cannot invoke routine `fast_forward` " + + "because it contains positional argument(s) following the named argument assigned to `table`; " + + "please rearrange them so the positional arguments come first and then retry the query again. " + + "SQLSTATE: 4274K"); + + assertThatThrownBy( + () -> + sql("CALL %s.custom.fast_forward('test_table', 'main', 'newBranch')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[FAILED_TO_LOAD_ROUTINE] Failed to load routine `%s`.`custom`.`fast_forward`. SQLSTATE: 38000", + catalogName); + + assertThatThrownBy(() -> sql("CALL %s.system.fast_forward('test_table', 'main')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `fast_forward` because the parameter named `to` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy( + () -> sql("CALL %s.system.fast_forward('', 'main', 'newBranch')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for parameter 'table'"); + } + + @TestTemplate + public void testFastForwardNonExistingToRefFails() { + sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.fast_forward(table => '%s', branch => '%s', to => '%s')", + catalogName, tableIdent, SnapshotRef.MAIN_BRANCH, "non_existing_branch")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref does not exist: non_existing_branch"); + } + + @TestTemplate + public void testFastForwardNonMain() { + sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + table.refresh(); + + String branch1 = "branch1"; + sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branch1); + String tableNameWithBranch1 = String.format("%s.branch_%s", tableName, branch1); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableNameWithBranch1); + table.refresh(); + Snapshot branch1Snapshot = table.snapshot(branch1); + + // Create branch2 from branch1 + String branch2 = "branch2"; + sql( + "ALTER TABLE %s CREATE BRANCH %s AS OF VERSION %d", + tableName, branch2, branch1Snapshot.snapshotId()); + String tableNameWithBranch2 = String.format("%s.branch_%s", tableName, branch2); + sql("INSERT INTO TABLE %s VALUES (3, 'c')", tableNameWithBranch2); + table.refresh(); + Snapshot branch2Snapshot = table.snapshot(branch2); + assertThat( + sql( + "CALL %s.system.fast_forward('%s', '%s', '%s')", + catalogName, tableIdent, branch1, branch2)) + .containsExactly(row(branch1, branch1Snapshot.snapshotId(), branch2Snapshot.snapshotId())); + } + + @TestTemplate + public void testFastForwardNonExistingFromMainCreatesBranch() { + sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); + String branch1 = "branch1"; + sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branch1); + String branchIdentifier = String.format("%s.branch_%s", tableName, branch1); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", branchIdentifier); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", branchIdentifier); + Table table = validationCatalog.loadTable(tableIdent); + table.refresh(); + Snapshot branch1Snapshot = table.snapshot(branch1); + + assertThat( + sql( + "CALL %s.system.fast_forward('%s', '%s', '%s')", + catalogName, tableIdent, SnapshotRef.MAIN_BRANCH, branch1)) + .containsExactly(row(SnapshotRef.MAIN_BRANCH, null, branch1Snapshot.snapshotId())); + + // Ensure the same behavior for non-main branches + String branch2 = "branch2"; + assertThat( + sql( + "CALL %s.system.fast_forward('%s', '%s', '%s')", + catalogName, tableIdent, branch2, branch1)) + .containsExactly(row(branch2, null, branch1Snapshot.snapshotId())); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java new file mode 100644 index 000000000000..9b7ed8f9be95 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java @@ -0,0 +1,3046 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; +import static org.apache.iceberg.TableProperties.COMMIT_MAX_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_MIN_RETRY_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.TableProperties.MERGE_DISTRIBUTION_MODE; +import static org.apache.iceberg.TableProperties.MERGE_ISOLATION_LEVEL; +import static org.apache.iceberg.TableProperties.MERGE_MODE; +import static org.apache.iceberg.TableProperties.MERGE_MODE_DEFAULT; +import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.iceberg.TableProperties.SPLIT_SIZE; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.lit; +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 static scala.collection.JavaConverters.mapAsScalaMapConverter; + +import java.util.Arrays; +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.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.exceptions.ValidationException; +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.Lists; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; +import org.apache.iceberg.spark.SparkSQLProperties; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.SparkRuntimeException; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.execution.SparkPlan; +import org.apache.spark.sql.internal.SQLConf; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TestMerge extends SparkRowLevelOperationsTestBase { + + @BeforeAll + public static void setupSparkConf() { + spark.conf().set("spark.sql.shuffle.partitions", "4"); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS source"); + } + + private void setupMergeWithAllClauses() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + + "{ \"id\": 2, \"dep\": \"emp-id-two\" }\n" + + "{ \"id\": 3, \"dep\": \"emp-id-3\" }\n" + + "{ \"id\": 4, \"dep\": \"emp-id-4\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 5, \"dep\": \"emp-id-5\" }"); + } + + private void verifyMergeWithAllClauses() { + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(1, "emp-id-1"), // updated (matched) + // row(2, "emp-id-two) // deleted (matched) + row(3, "invalid"), // updated (not matched by source) + // row(4, "emp-id-4) // deleted (not matched by source) + row(5, "emp-id-5")), // new + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithAllClauses() { + setupMergeWithAllClauses(); + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET * " + + "WHEN MATCHED AND t.id = 2 THEN " + + " DELETE " + + "WHEN NOT MATCHED THEN " + + " INSERT * " + + "WHEN NOT MATCHED BY SOURCE AND t.id = 3 THEN " + + " UPDATE SET dep = 'invalid' " + + "WHEN NOT MATCHED BY SOURCE AND t.id = 4 THEN " + + " DELETE ", + commitTarget()); + verifyMergeWithAllClauses(); + } + + @TestTemplate + public void testMergeWithAllClausesUsingDataFrameAPI() { + setupMergeWithAllClauses(); + spark + .table("source") + .mergeInto(commitTarget(), col(commitTarget() + ".id").equalTo(col("source.id"))) + .whenMatched(col(commitTarget() + ".id").equalTo(lit(1))) + .updateAll() + .whenMatched(col(commitTarget() + ".id").equalTo(lit(2))) + .delete() + .whenNotMatched() + .insertAll() + .whenNotMatchedBySource(col(commitTarget() + ".id").equalTo(lit(3))) + .update( + scala.collection.immutable.Map.from( + mapAsScalaMapConverter(ImmutableMap.of("dep", lit("invalid"))).asScala())) + .whenNotMatchedBySource(col(commitTarget() + ".id").equalTo(lit(4))) + .delete() + .merge(); + verifyMergeWithAllClauses(); + } + + @TestTemplate + public void testMergeWithOneNotMatchedBySourceClause() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 3, \"dep\": \"emp-id-3\" }\n" + + "{ \"id\": 4, \"dep\": \"emp-id-4\" }"); + + createOrReplaceView("source", ImmutableList.of(1, 4), Encoders.INT()); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value " + + "WHEN NOT MATCHED BY SOURCE THEN " + + " DELETE ", + commitTarget()); + + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(1, "emp-id-1"), // existing + // row(2, "emp-id-2) // deleted (not matched by source) + // row(3, "emp-id-3") // deleted (not matched by source) + row(4, "emp-id-4")), // existing + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeNotMatchedBySourceClausesPartitionedTable() { + createAndInitTable( + "id INT, dep STRING", + "PARTITIONED BY (dep)", + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hr\" }\n" + + "{ \"id\": 3, \"dep\": \"support\" }"); + + createOrReplaceView("source", ImmutableList.of(1, 2), Encoders.INT()); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value AND t.dep = 'hr' " + + "WHEN MATCHED THEN " + + " UPDATE SET dep = 'support' " + + "WHEN NOT MATCHED BY SOURCE THEN " + + " UPDATE SET dep = 'invalid' ", + commitTarget()); + + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(1, "support"), // updated (matched) + row(2, "support"), // updated (matched) + row(3, "invalid")), // updated (not matched by source) + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithVectorizedReads() { + assumeThat(supportsVectorization()).isTrue(); + + createAndInitTable( + "id INT, value INT, dep STRING", + "PARTITIONED BY (dep)", + "{ \"id\": 1, \"value\": 100, \"dep\": \"hr\" }\n" + + "{ \"id\": 6, \"value\": 600, \"dep\": \"software\" }"); + + createOrReplaceView( + "source", + "id INT, value INT", + "{ \"id\": 2, \"value\": 201 }\n" + + "{ \"id\": 1, \"value\": 101 }\n" + + "{ \"id\": 6, \"value\": 601 }"); + + SparkPlan plan = + executeAndKeepPlan( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET t.value = s.value " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT (id, value, dep) VALUES (s.id, s.value, 'invalid')", + commitTarget()); + + assertAllBatchScansVectorized(plan); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, 101, "hr"), // updated + row(2, 201, "invalid")); // new + + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testCoalesceMerge() { + createAndInitTable("id INT, salary INT, dep STRING"); + + String[] records = new String[100]; + for (int index = 0; index < 100; index++) { + records[index] = String.format("{ \"id\": %d, \"salary\": 100, \"dep\": \"hr\" }", index); + } + append(tableName, records); + append(tableName, records); + append(tableName, records); + append(tableName, records); + + // set the open file cost large enough to produce a separate scan task per file + // disable any write distribution + Map tableProps = + ImmutableMap.of( + SPLIT_OPEN_FILE_COST, + String.valueOf(Integer.MAX_VALUE), + MERGE_DISTRIBUTION_MODE, + DistributionMode.NONE.modeName(), + TableProperties.DELETE_GRANULARITY, + DeleteGranularity.PARTITION.toString()); + sql("ALTER TABLE %s SET TBLPROPERTIES (%s)", tableName, tablePropsAsString(tableProps)); + + createBranchIfNeeded(); + + spark.range(0, 100).createOrReplaceTempView("source"); + + // enable AQE and set the advisory partition big enough to trigger combining + // set the number of shuffle partitions to 200 to distribute the work across reducers + // disable broadcast joins to make sure the join triggers a shuffle + // set the advisory partition size for shuffles small enough to ensure writes override it + withSQLConf( + ImmutableMap.of( + SQLConf.SHUFFLE_PARTITIONS().key(), + "200", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(), + "-1", + SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), + "true", + SQLConf.COALESCE_PARTITIONS_ENABLED().key(), + "true", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES().key(), + "100", + SparkSQLProperties.ADVISORY_PARTITION_SIZE, + String.valueOf(256 * 1024 * 1024)), + () -> { + sql( + "MERGE INTO %s t USING source " + + "ON t.id = source.id " + + "WHEN MATCHED THEN " + + " UPDATE SET salary = -1 ", + commitTarget()); + }); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + + if (mode(table) == COPY_ON_WRITE) { + // CoW MERGE would perform a join on `id` + // every task has data for each of 200 reducers + // AQE detects that all shuffle blocks are small and processes them in 1 task + // otherwise, there would be 200 tasks writing to the table + validateProperty(currentSnapshot, SnapshotSummary.ADDED_FILES_PROP, "1"); + } else if (mode(table) == MERGE_ON_READ && formatVersion >= 3) { + validateProperty(currentSnapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "4"); + validateProperty(currentSnapshot, SnapshotSummary.ADDED_DVS_PROP, "4"); + } else { + // MoR MERGE would perform a join on `id` + // every task has data for each of 200 reducers + // AQE detects that all shuffle blocks are small and processes them in 1 task + // otherwise, there would be 200 tasks writing to the table + validateProperty(currentSnapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "1"); + } + + assertThat(scalarSql("SELECT COUNT(*) FROM %s WHERE salary = -1", commitTarget())) + .as("Row count must match") + .isEqualTo(400L); + } + + @TestTemplate + public void testSkewMerge() { + createAndInitTable("id INT, salary INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + String[] records = new String[100]; + for (int index = 0; index < 100; index++) { + records[index] = String.format("{ \"id\": %d, \"salary\": 100, \"dep\": \"hr\" }", index); + } + append(tableName, records); + append(tableName, records); + append(tableName, records); + append(tableName, records); + + // set the open file cost large enough to produce a separate scan task per file + // use hash distribution to trigger a shuffle + Map tableProps = + ImmutableMap.of( + SPLIT_OPEN_FILE_COST, + String.valueOf(Integer.MAX_VALUE), + MERGE_DISTRIBUTION_MODE, + DistributionMode.HASH.modeName()); + sql("ALTER TABLE %s SET TBLPROPERTIES (%s)", tableName, tablePropsAsString(tableProps)); + + createBranchIfNeeded(); + + spark.range(0, 100).createOrReplaceTempView("source"); + + // enable AQE and set the advisory partition size small enough to trigger a split + // set the number of shuffle partitions to 2 to only have 2 reducers + // set the min coalesce partition size small enough to avoid coalescing + // set the advisory partition size for shuffles big enough to ensure writes override it + withSQLConf( + ImmutableMap.of( + SQLConf.SHUFFLE_PARTITIONS().key(), + "4", + SQLConf.COALESCE_PARTITIONS_MIN_PARTITION_SIZE().key(), + "100", + SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), + "true", + SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED().key(), + "true", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES().key(), + "256MB", + SparkSQLProperties.ADVISORY_PARTITION_SIZE, + "100"), + () -> { + SparkPlan plan = + executeAndKeepPlan( + "MERGE INTO %s t USING source " + + "ON t.id = source.id " + + "WHEN MATCHED THEN " + + " UPDATE SET salary = -1 ", + commitTarget()); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + }); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + + if (mode(table) == COPY_ON_WRITE) { + // CoW MERGE would perform a join on `id` and then cluster records by `dep` + // the first shuffle distributes records into 4 shuffle partitions so that rows can be merged + // after existing and new rows are merged, the data is clustered by `dep` + // each task with merged data contains records for the same table partition + // that means there are 4 shuffle blocks, all assigned to the same reducer + // AQE detects that all shuffle blocks are big and processes them in 4 independent tasks + // otherwise, there would be 1 task processing all 4 shuffle blocks + validateProperty(currentSnapshot, SnapshotSummary.ADDED_FILES_PROP, "4"); + } else { + // MoR MERGE would perform a join on `id` and then cluster data based on the partition + // all tasks belong to the same partition and therefore write only 1 shuffle block per task + // that means there are 4 shuffle blocks, all assigned to the same reducer + // AQE detects that all 4 shuffle blocks are big and processes them in 4 separate tasks + // otherwise, there would be 1 task processing 4 shuffle blocks + validateProperty(currentSnapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "4"); + } + + assertThat(scalarSql("SELECT COUNT(*) FROM %s WHERE salary = -1", commitTarget())) + .as("Row count must match") + .isEqualTo(400L); + } + + @TestTemplate + public void testMergeConditionSplitIntoTargetPredicateAndJoinCondition() { + createAndInitTable( + "id INT, salary INT, dep STRING, sub_dep STRING", + "PARTITIONED BY (dep, sub_dep)", + "{ \"id\": 1, \"salary\": 100, \"dep\": \"d1\", \"sub_dep\": \"sd1\" }\n" + + "{ \"id\": 6, \"salary\": 600, \"dep\": \"d6\", \"sub_dep\": \"sd6\" }"); + + createOrReplaceView( + "source", + "id INT, salary INT, dep STRING, sub_dep STRING", + "{ \"id\": 1, \"salary\": 101, \"dep\": \"d1\", \"sub_dep\": \"sd1\" }\n" + + "{ \"id\": 2, \"salary\": 200, \"dep\": \"d2\", \"sub_dep\": \"sd2\" }\n" + + "{ \"id\": 3, \"salary\": 300, \"dep\": \"d3\", \"sub_dep\": \"sd3\" }"); + + String query = + String.format( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id AND ((t.dep = 'd1' AND t.sub_dep IN ('sd1', 'sd3')) OR (t.dep = 'd6' AND t.sub_dep IN ('sd2', 'sd6'))) " + + "WHEN MATCHED THEN " + + " UPDATE SET salary = s.salary " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + + if (mode(table) == COPY_ON_WRITE) { + checkJoinAndFilterConditions( + query, + "Join [id], [id], FullOuter", + "((dep = 'd1' AND sub_dep IN ('sd1', 'sd3')) OR (dep = 'd6' AND sub_dep IN ('sd2', 'sd6')))"); + } else { + checkJoinAndFilterConditions( + query, + "Join [id], [id], RightOuter", + "((dep = 'd1' AND sub_dep IN ('sd1', 'sd3')) OR (dep = 'd6' AND sub_dep IN ('sd2', 'sd6')))"); + } + + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(1, 101, "d1", "sd1"), // updated + row(2, 200, "d2", "sd2"), // new + row(3, 300, "d3", "sd3"), // new + row(6, 600, "d6", "sd6")), // existing + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithStaticPredicatePushDown() { + createAndInitTable("id BIGINT, dep STRING"); + + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + // add a data file to the 'software' partition + append(tableName, "{ \"id\": 1, \"dep\": \"software\" }"); + createBranchIfNeeded(); + + // add a data file to the 'hr' partition + append(commitTarget(), "{ \"id\": 1, \"dep\": \"hr\" }"); + + Table table = validationCatalog.loadTable(tableIdent); + + Snapshot snapshot = SnapshotUtil.latestSnapshot(table, branch); + String dataFilesCount = snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP); + assertThat(dataFilesCount).as("Must have 2 files before MERGE").isEqualTo("2"); + + createOrReplaceView( + "source", "{ \"id\": 1, \"dep\": \"finance\" }\n" + "{ \"id\": 2, \"dep\": \"hardware\" }"); + + // remove the data file from the 'hr' partition to ensure it is not scanned + withUnavailableFiles( + snapshot.addedDataFiles(table.io()), + () -> { + // disable dynamic pruning and rely only on static predicate pushdown + withSQLConf( + ImmutableMap.of( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED().key(), "false", + SQLConf.RUNTIME_ROW_LEVEL_OPERATION_GROUP_FILTER_ENABLED().key(), "false"), + () -> { + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.id AND t.dep IN ('software') AND source.id < 10 " + + "WHEN MATCHED AND source.id = 1 THEN " + + " UPDATE SET dep = source.dep " + + "WHEN NOT MATCHED THEN " + + " INSERT (dep, id) VALUES (source.dep, source.id)", + commitTarget()); + }); + }); + + ImmutableList expectedRows = + ImmutableList.of( + row(1L, "finance"), // updated + row(1L, "hr"), // kept + row(2L, "hardware") // new + ); + assertEquals( + "Output should match", + expectedRows, + sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); + } + + @TestTemplate + public void testMergeIntoEmptyTargetInsertAllNonMatchingRows() { + assumeThat(branch).as("Custom branch does not exist for empty table").isNotEqualTo("test"); + createAndInitTable("id INT, dep STRING"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 3, \"dep\": \"emp-id-3\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + tableName); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "emp-id-1"), // new + row(2, "emp-id-2"), // new + row(3, "emp-id-3") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeIntoEmptyTargetInsertOnlyMatchingRows() { + assumeThat(branch).as("Custom branch does not exist for empty table").isNotEqualTo("test"); + createAndInitTable("id INT, dep STRING"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 3, \"dep\": \"emp-id-3\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN NOT MATCHED AND (s.id >=2) THEN " + + " INSERT *", + tableName); + + ImmutableList expectedRows = + ImmutableList.of( + row(2, "emp-id-2"), // new + row(3, "emp-id-3") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithOnlyUpdateClause() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-six\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "emp-id-1"), // updated + row(6, "emp-id-six") // kept + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithOnlyUpdateNullUnmatchedValues() { + createAndInitTable( + "id INT, value INT", "{ \"id\": 1, \"value\": 2 }\n" + "{ \"id\": 6, \"value\": null }"); + + createOrReplaceView("source", "id INT NOT NULL, value INT", "{ \"id\": 1, \"value\": 100 }\n"); + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET id=123, value=456", + commitTarget()); + + sql("SELECT * FROM %s", commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(6, null), // kept + row(123, 456)); // updated + + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithOnlyUpdateSingleFieldNullUnmatchedValues() { + createAndInitTable( + "id INT, value INT", "{ \"id\": 1, \"value\": 2 }\n" + "{ \"id\": 6, \"value\": null }"); + + createOrReplaceView("source", "id INT NOT NULL, value INT", "{ \"id\": 1, \"value\": 100 }\n"); + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET id=123", + commitTarget()); + + sql("SELECT * FROM %s", commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(6, null), // kept + row(123, 2)); // updated + + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithOnlyDeleteNullUnmatchedValues() { + createAndInitTable( + "id INT, value INT", "{ \"id\": 1, \"value\": 2 }\n" + "{ \"id\": 6, \"value\": null }"); + + createOrReplaceView("source", "id INT NOT NULL, value INT", "{ \"id\": 1, \"value\": 100 }\n"); + sql( + "MERGE INTO %s t USING source s " + "ON t.id == s.id " + "WHEN MATCHED THEN " + "DELETE", + commitTarget()); + + sql("SELECT * FROM %s", commitTarget()); + + ImmutableList expectedRows = ImmutableList.of(row(6, null)); // kept + + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithOnlyUpdateClauseAndNullValues() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": null, \"dep\": \"emp-id-one\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-six\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id AND t.id < 3 " + + "WHEN MATCHED THEN " + + " UPDATE SET *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(null, "emp-id-one"), // kept + row(1, "emp-id-1"), // updated + row(6, "emp-id-six")); // kept + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithOnlyDeleteClause() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "emp-id-one") // kept + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithMatchedAndNotMatchedClauses() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET * " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "emp-id-1"), // updated + row(2, "emp-id-2") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithAllCausesWithExplicitColumnSpecification() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET t.id = s.id, t.dep = s.dep " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT (t.id, t.dep) VALUES (s.id, s.dep)", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "emp-id-1"), // updated + row(2, "emp-id-2") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithSourceCTE() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-two\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-3\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 5, \"dep\": \"emp-id-6\" }"); + + sql( + "WITH cte1 AS (SELECT id + 1 AS id, dep FROM source) " + + "MERGE INTO %s AS t USING cte1 AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 2 THEN " + + " UPDATE SET * " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 3 THEN " + + " INSERT *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(2, "emp-id-2"), // updated + row(3, "emp-id-3") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithSourceFromSetOps() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + String derivedSource = + "SELECT * FROM source WHERE id = 2 " + + "UNION ALL " + + "SELECT * FROM source WHERE id = 1 OR id = 6"; + + sql( + "MERGE INTO %s AS t USING (%s) AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET * " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget(), derivedSource); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "emp-id-1"), // updated + row(2, "emp-id-2") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithOneMatchingBranchButMultipleSourceRowsForTargetRow() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 1, \"state\": \"on\" }\n" + + "{ \"id\": 1, \"state\": \"off\" }\n" + + "{ \"id\": 10, \"state\": \"on\" }"); + + String errorMsg = + "MERGE statement matched a single row from the target table with multiple rows of the source table."; + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, dep) VALUES (s.id, 'unknown')", + commitTarget())) + .isInstanceOf(SparkRuntimeException.class) + .hasMessageContaining(errorMsg); + + assertEquals( + "Target should be unchanged", + ImmutableList.of(row(1, "emp-id-one"), row(6, "emp-id-6")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + List sourceIds = Lists.newArrayList(); + for (int i = 0; i < 10_000; i++) { + sourceIds.add(i); + } + Dataset ds = spark.createDataset(sourceIds, Encoders.INT()); + ds.union(ds).createOrReplaceTempView("source"); + + String errorMsg = + "MERGE statement matched a single row from the target table with multiple rows of the source table."; + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET id = 10 " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.value = 2 THEN " + + " INSERT (id, dep) VALUES (s.value, null)", + commitTarget())) + .isInstanceOf(SparkRuntimeException.class) + .hasMessageContaining(errorMsg); + + assertEquals( + "Target should be unchanged", + ImmutableList.of(row(1, "emp-id-one"), row(6, "emp-id-6")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceEnabledHashShuffleJoin() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + List sourceIds = Lists.newArrayList(); + for (int i = 0; i < 10_000; i++) { + sourceIds.add(i); + } + Dataset ds = spark.createDataset(sourceIds, Encoders.INT()); + ds.union(ds).createOrReplaceTempView("source"); + + withSQLConf( + ImmutableMap.of(SQLConf.PREFER_SORTMERGEJOIN().key(), "false"), + () -> { + String errorMsg = + "MERGE statement matched a single row from the target table with multiple rows of the source table."; + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET id = 10 " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.value = 2 THEN " + + " INSERT (id, dep) VALUES (s.value, null)", + commitTarget())) + .isInstanceOf(SparkRuntimeException.class) + .hasMessageContaining(errorMsg); + }); + + assertEquals( + "Target should be unchanged", + ImmutableList.of(row(1, "emp-id-one"), row(6, "emp-id-6")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoEqualityCondition() { + createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"emp-id-one\" }"); + + List sourceIds = Lists.newArrayList(); + for (int i = 0; i < 10_000; i++) { + sourceIds.add(i); + } + Dataset ds = spark.createDataset(sourceIds, Encoders.INT()); + ds.union(ds).createOrReplaceTempView("source"); + + withSQLConf( + ImmutableMap.of(SQLConf.PREFER_SORTMERGEJOIN().key(), "false"), + () -> { + String errorMsg = + "MERGE statement matched a single row from the target table with multiple rows of the source table."; + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id > s.value " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET id = 10 " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.value = 2 THEN " + + " INSERT (id, dep) VALUES (s.value, null)", + commitTarget())) + .isInstanceOf(SparkRuntimeException.class) + .hasMessageContaining(errorMsg); + }); + + assertEquals( + "Target should be unchanged", + ImmutableList.of(row(1, "emp-id-one")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActions() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + List sourceIds = Lists.newArrayList(); + for (int i = 0; i < 10_000; i++) { + sourceIds.add(i); + } + Dataset ds = spark.createDataset(sourceIds, Encoders.INT()); + ds.union(ds).createOrReplaceTempView("source"); + + String errorMsg = + "MERGE statement matched a single row from the target table with multiple rows of the source table."; + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET id = 10 " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE", + commitTarget())) + .isInstanceOf(SparkRuntimeException.class) + .hasMessageContaining(errorMsg); + assertEquals( + "Target should be unchanged", + ImmutableList.of(row(1, "emp-id-one"), row(6, "emp-id-6")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void + testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActionsNoEqualityCondition() { + createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"emp-id-one\" }"); + + List sourceIds = Lists.newArrayList(); + for (int i = 0; i < 10_000; i++) { + sourceIds.add(i); + } + Dataset ds = spark.createDataset(sourceIds, Encoders.INT()); + ds.union(ds).createOrReplaceTempView("source"); + + String errorMsg = + "MERGE statement matched a single row from the target table with multiple rows of the source table."; + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id > s.value " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET id = 10 " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE", + commitTarget())) + .isInstanceOf(SparkRuntimeException.class) + .hasMessageContaining(errorMsg); + + assertEquals( + "Target should be unchanged", + ImmutableList.of(row(1, "emp-id-one")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testMergeWithMultipleUpdatesForTargetRow() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + String errorMsg = + "MERGE statement matched a single row from the target table with multiple rows of the source table."; + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET * " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget())) + .isInstanceOf(SparkRuntimeException.class) + .hasMessageContaining(errorMsg); + + assertEquals( + "Target should be unchanged", + ImmutableList.of(row(1, "emp-id-one"), row(6, "emp-id-6")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testMergeWithUnconditionalDelete() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(2, "emp-id-2") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithSingleConditionalDelete() { + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + String errorMsg = + "MERGE statement matched a single row from the target table with multiple rows of the source table."; + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget())) + .isInstanceOf(SparkRuntimeException.class) + .hasMessageContaining(errorMsg); + + assertEquals( + "Target should be unchanged", + ImmutableList.of(row(1, "emp-id-one"), row(6, "emp-id-6")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testMergeWithIdentityTransform() { + for (DistributionMode mode : DistributionMode.values()) { + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD identity(dep)", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + tableName, WRITE_DISTRIBUTION_MODE, mode.modeName()); + + append( + tableName, + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + createBranchIfNeeded(); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET * " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "emp-id-1"), // updated + row(2, "emp-id-2") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + removeTables(); + } + } + + @TestTemplate + public void testMergeWithDaysTransform() { + for (DistributionMode mode : DistributionMode.values()) { + createAndInitTable("id INT, ts TIMESTAMP"); + sql("ALTER TABLE %s ADD PARTITION FIELD days(ts)", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + tableName, WRITE_DISTRIBUTION_MODE, mode.modeName()); + + append( + tableName, + "id INT, ts TIMESTAMP", + "{ \"id\": 1, \"ts\": \"2000-01-01 00:00:00\" }\n" + + "{ \"id\": 6, \"ts\": \"2000-01-06 00:00:00\" }"); + createBranchIfNeeded(); + + createOrReplaceView( + "source", + "id INT, ts TIMESTAMP", + "{ \"id\": 2, \"ts\": \"2001-01-02 00:00:00\" }\n" + + "{ \"id\": 1, \"ts\": \"2001-01-01 00:00:00\" }\n" + + "{ \"id\": 6, \"ts\": \"2001-01-06 00:00:00\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET * " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "2001-01-01 00:00:00"), // updated + row(2, "2001-01-02 00:00:00") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT id, CAST(ts AS STRING) FROM %s ORDER BY id", selectTarget())); + + removeTables(); + } + } + + @TestTemplate + public void testMergeWithBucketTransform() { + for (DistributionMode mode : DistributionMode.values()) { + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD bucket(2, dep)", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + tableName, WRITE_DISTRIBUTION_MODE, mode.modeName()); + + append( + tableName, + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + createBranchIfNeeded(); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET * " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "emp-id-1"), // updated + row(2, "emp-id-2") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + removeTables(); + } + } + + @TestTemplate + public void testMergeWithTruncateTransform() { + for (DistributionMode mode : DistributionMode.values()) { + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD truncate(dep, 2)", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + tableName, WRITE_DISTRIBUTION_MODE, mode.modeName()); + + append( + tableName, + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + createBranchIfNeeded(); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET * " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "emp-id-1"), // updated + row(2, "emp-id-2") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + removeTables(); + } + } + + @TestTemplate + public void testMergeIntoPartitionedAndOrderedTable() { + for (DistributionMode mode : DistributionMode.values()) { + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + sql("ALTER TABLE %s WRITE ORDERED BY (id)", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + tableName, WRITE_DISTRIBUTION_MODE, mode.modeName()); + + append( + tableName, + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + createBranchIfNeeded(); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET * " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "emp-id-1"), // updated + row(2, "emp-id-2") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + removeTables(); + } + } + + @TestTemplate + public void testSelfMerge() { + createAndInitTable( + "id INT, v STRING", "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); + + sql( + "MERGE INTO %s t USING %s s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET v = 'x' " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + commitTarget(), commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "x"), // updated + row(2, "v2") // kept + ); + assertEquals( + "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testSelfMergeWithCaching() { + createAndInitTable( + "id INT, v STRING", "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); + + sql("CACHE TABLE %s", tableName); + + sql( + "MERGE INTO %s t USING %s s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET v = 'x' " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + commitTarget(), commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "x"), // updated + row(2, "v2") // kept + ); + assertEquals( + "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", commitTarget())); + } + + @TestTemplate + public void testMergeWithSourceAsSelfSubquery() { + createAndInitTable( + "id INT, v STRING", "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); + + createOrReplaceView("source", Arrays.asList(1, null), Encoders.INT()); + + sql( + "MERGE INTO %s t USING (SELECT id AS value FROM %s r JOIN source ON r.id = source.value) s " + + "ON t.id == s.value " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET v = 'x' " + + "WHEN NOT MATCHED THEN " + + " INSERT (v, id) VALUES ('invalid', -1) ", + commitTarget(), commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "x"), // updated + row(2, "v2") // kept + ); + assertEquals( + "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public synchronized void testMergeWithSerializableIsolation() throws InterruptedException { + // cannot run tests with concurrency for Hadoop tables without atomic renames + assumeThat(catalogName).isNotEqualToIgnoringCase("testhadoop"); + // if caching is off, the table is eagerly refreshed during runtime filtering + // this can cause a validation exception as concurrent changes would be visible + assumeThat(cachingCatalogEnabled()).isTrue(); + + createAndInitTable("id INT, dep STRING"); + createOrReplaceView("source", Collections.singletonList(1), Encoders.INT()); + + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + tableName, MERGE_ISOLATION_LEVEL, "serializable"); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); + createBranchIfNeeded(); + + 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'", + commitTarget()); + + barrier.incrementAndGet(); + } + }); + + // append thread + Future appendFuture = + executorService.submit( + () -> { + // load the table via the validation catalog to use another table instance + Table table = validationCatalog.loadTable(tableIdent); + + 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)); + AppendFiles appendFiles = table.newFastAppend().appendFile(dataFile); + if (branch != null) { + appendFiles.toBranch(branch); + } + appendFiles.commit(); + } + + barrier.incrementAndGet(); + } + }); + + try { + assertThatThrownBy(mergeFuture::get) + .isInstanceOf(ExecutionException.class) + .cause() + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Found conflicting files that can contain"); + } finally { + shouldAppend.set(false); + appendFuture.cancel(true); + } + + executorService.shutdown(); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); + } + + @TestTemplate + public synchronized void testMergeWithSnapshotIsolation() + throws InterruptedException, ExecutionException { + // cannot run tests with concurrency for Hadoop tables without atomic renames + assumeThat(catalogName).isNotEqualToIgnoringCase("testhadoop"); + // if caching is off, the table is eagerly refreshed during runtime filtering + // this can cause a validation exception as concurrent changes would be visible + assumeThat(cachingCatalogEnabled()).isTrue(); + + createAndInitTable("id INT, dep STRING"); + createOrReplaceView("source", Collections.singletonList(1), Encoders.INT()); + + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s'='%s', '%s'='%s', '%s'='%s', '%s'='%s')", + tableName, + MERGE_ISOLATION_LEVEL, + "snapshot", + COMMIT_MIN_RETRY_WAIT_MS, + "10", + COMMIT_MAX_RETRY_WAIT_MS, + "1000", + COMMIT_NUM_RETRIES, + "7"); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); + createBranchIfNeeded(); + + 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 < 20; 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'", + commitTarget()); + + barrier.incrementAndGet(); + } + }); + + // append thread + Future appendFuture = + executorService.submit( + () -> { + // load the table via the validation catalog to use another table instance for inserts + Table table = validationCatalog.loadTable(tableIdent); + + GenericRecord record = GenericRecord.create(table.schema()); + record.set(0, 1); // id + record.set(1, "hr"); // dep + + for (int numOperations = 0; numOperations < 20; 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 { + mergeFuture.get(); + } finally { + shouldAppend.set(false); + appendFuture.cancel(true); + } + + executorService.shutdown(); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); + } + + @TestTemplate + public void testMergeWithExtraColumnsInSource() { + createAndInitTable( + "id INT, v STRING", "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); + createOrReplaceView( + "source", + "{ \"id\": 1, \"extra_col\": -1, \"v\": \"v1_1\" }\n" + + "{ \"id\": 3, \"extra_col\": -1, \"v\": \"v3\" }\n" + + "{ \"id\": 4, \"extra_col\": -1, \"v\": \"v4\" }"); + + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.id " + + "WHEN MATCHED THEN " + + " UPDATE SET v = source.v " + + "WHEN NOT MATCHED THEN " + + " INSERT (v, id) VALUES (source.v, source.id)", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "v1_1"), // new + row(2, "v2"), // kept + row(3, "v3"), // new + row(4, "v4") // new + ); + assertEquals( + "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithNullsInTargetAndSource() { + createAndInitTable( + "id INT, v STRING", "{ \"id\": null, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); + + createOrReplaceView( + "source", "{ \"id\": null, \"v\": \"v1_1\" }\n" + "{ \"id\": 4, \"v\": \"v4\" }"); + + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.id " + + "WHEN MATCHED THEN " + + " UPDATE SET v = source.v " + + "WHEN NOT MATCHED THEN " + + " INSERT (v, id) VALUES (source.v, source.id)", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(null, "v1"), // kept + row(null, "v1_1"), // new + row(2, "v2"), // kept + row(4, "v4") // new + ); + assertEquals( + "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY v", selectTarget())); + } + + @TestTemplate + public void testMergeWithNullSafeEquals() { + createAndInitTable( + "id INT, v STRING", "{ \"id\": null, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); + + createOrReplaceView( + "source", "{ \"id\": null, \"v\": \"v1_1\" }\n" + "{ \"id\": 4, \"v\": \"v4\" }"); + + sql( + "MERGE INTO %s t USING source " + + "ON t.id <=> source.id " + + "WHEN MATCHED THEN " + + " UPDATE SET v = source.v " + + "WHEN NOT MATCHED THEN " + + " INSERT (v, id) VALUES (source.v, source.id)", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(null, "v1_1"), // updated + row(2, "v2"), // kept + row(4, "v4") // new + ); + assertEquals( + "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY v", selectTarget())); + } + + @TestTemplate + public void testMergeWithNullCondition() { + createAndInitTable( + "id INT, v STRING", "{ \"id\": null, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); + + createOrReplaceView( + "source", "{ \"id\": null, \"v\": \"v1_1\" }\n" + "{ \"id\": 2, \"v\": \"v2_2\" }"); + + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.id AND NULL " + + "WHEN MATCHED THEN " + + " UPDATE SET v = source.v " + + "WHEN NOT MATCHED THEN " + + " INSERT (v, id) VALUES (source.v, source.id)", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(null, "v1"), // kept + row(null, "v1_1"), // new + row(2, "v2"), // kept + row(2, "v2_2") // new + ); + assertEquals( + "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY v", selectTarget())); + } + + @TestTemplate + public void testMergeWithNullActionConditions() { + createAndInitTable( + "id INT, v STRING", "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); + + createOrReplaceView( + "source", + "{ \"id\": 1, \"v\": \"v1_1\" }\n" + + "{ \"id\": 2, \"v\": \"v2_2\" }\n" + + "{ \"id\": 3, \"v\": \"v3_3\" }"); + + // all conditions are NULL and will never match any rows + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.id " + + "WHEN MATCHED AND source.id = 1 AND NULL THEN " + + " UPDATE SET v = source.v " + + "WHEN MATCHED AND source.v = 'v1_1' AND NULL THEN " + + " DELETE " + + "WHEN NOT MATCHED AND source.id = 3 AND NULL THEN " + + " INSERT (v, id) VALUES (source.v, source.id)", + commitTarget()); + + ImmutableList expectedRows1 = + ImmutableList.of( + row(1, "v1"), // kept + row(2, "v2") // kept + ); + assertEquals( + "Output should match", expectedRows1, sql("SELECT * FROM %s ORDER BY v", selectTarget())); + + // only the update and insert conditions are NULL + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.id " + + "WHEN MATCHED AND source.id = 1 AND NULL THEN " + + " UPDATE SET v = source.v " + + "WHEN MATCHED AND source.v = 'v1_1' THEN " + + " DELETE " + + "WHEN NOT MATCHED AND source.id = 3 AND NULL THEN " + + " INSERT (v, id) VALUES (source.v, source.id)", + commitTarget()); + + ImmutableList expectedRows2 = + ImmutableList.of( + row(2, "v2") // kept + ); + assertEquals( + "Output should match", expectedRows2, sql("SELECT * FROM %s ORDER BY v", selectTarget())); + } + + @TestTemplate + public void testMergeWithMultipleMatchingActions() { + createAndInitTable( + "id INT, v STRING", "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); + + createOrReplaceView( + "source", "{ \"id\": 1, \"v\": \"v1_1\" }\n" + "{ \"id\": 2, \"v\": \"v2_2\" }"); + + // the order of match actions is important in this case + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.id " + + "WHEN MATCHED AND source.id = 1 THEN " + + " UPDATE SET v = source.v " + + "WHEN MATCHED AND source.v = 'v1_1' THEN " + + " DELETE " + + "WHEN NOT MATCHED THEN " + + " INSERT (v, id) VALUES (source.v, source.id)", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "v1_1"), // updated (also matches the delete cond but update is first) + row(2, "v2") // kept (matches neither the update nor the delete cond) + ); + assertEquals( + "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY v", selectTarget())); + } + + @TestTemplate + public void testMergeWithMultipleRowGroupsParquet() throws NoSuchTableException { + assumeThat(fileFormat).isEqualTo(FileFormat.PARQUET); + + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%d')", + tableName, PARQUET_ROW_GROUP_SIZE_BYTES, 100); + sql("ALTER TABLE %s SET TBLPROPERTIES('%s' '%d')", tableName, SPLIT_SIZE, 100); + + createOrReplaceView("source", Collections.singletonList(1), Encoders.INT()); + + List ids = Lists.newArrayListWithCapacity(200); + for (int id = 1; id <= 200; id++) { + ids.add(id); + } + Dataset df = + spark + .createDataset(ids, Encoders.INT()) + .withColumnRenamed("value", "id") + .withColumn("dep", lit("hr")); + df.coalesce(1).writeTo(tableName).append(); + createBranchIfNeeded(); + + assertThat(spark.table(commitTarget()).count()).isEqualTo(200); + + // update a record from one of two row groups and copy over the second one + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.value " + + "WHEN MATCHED THEN " + + " UPDATE SET dep = 'x'", + commitTarget()); + + assertThat(spark.table(commitTarget()).count()).isEqualTo(200); + } + + @TestTemplate + public void testMergeInsertOnly() { + createAndInitTable( + "id STRING, v STRING", + "{ \"id\": \"a\", \"v\": \"v1\" }\n" + "{ \"id\": \"b\", \"v\": \"v2\" }"); + createOrReplaceView( + "source", + "{ \"id\": \"a\", \"v\": \"v1_1\" }\n" + + "{ \"id\": \"a\", \"v\": \"v1_2\" }\n" + + "{ \"id\": \"c\", \"v\": \"v3\" }\n" + + "{ \"id\": \"d\", \"v\": \"v4_1\" }\n" + + "{ \"id\": \"d\", \"v\": \"v4_2\" }"); + + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.id " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row("a", "v1"), // kept + row("b", "v2"), // kept + row("c", "v3"), // new + row("d", "v4_1"), // new + row("d", "v4_2") // new + ); + assertEquals( + "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeInsertOnlyWithCondition() { + createAndInitTable("id INTEGER, v INTEGER", "{ \"id\": 1, \"v\": 1 }"); + createOrReplaceView( + "source", + "{ \"id\": 1, \"v\": 11, \"is_new\": true }\n" + + "{ \"id\": 2, \"v\": 21, \"is_new\": true }\n" + + "{ \"id\": 2, \"v\": 22, \"is_new\": false }"); + + // validate assignments are reordered to match the table attrs + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.id " + + "WHEN NOT MATCHED AND is_new = TRUE THEN " + + " INSERT (v, id) VALUES (s.v + 100, s.id)", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, 1), // kept + row(2, 121) // new + ); + assertEquals( + "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeAlignsUpdateAndInsertActions() { + createAndInitTable("id INT, a INT, b STRING", "{ \"id\": 1, \"a\": 2, \"b\": \"str\" }"); + createOrReplaceView( + "source", + "{ \"id\": 1, \"c1\": -2, \"c2\": \"new_str_1\" }\n" + + "{ \"id\": 2, \"c1\": -20, \"c2\": \"new_str_2\" }"); + + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.id " + + "WHEN MATCHED THEN " + + " UPDATE SET b = c2, a = c1, t.id = source.id " + + "WHEN NOT MATCHED THEN " + + " INSERT (b, a, id) VALUES (c2, c1, id)", + commitTarget()); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, -2, "new_str_1"), row(2, -20, "new_str_2")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeMixedCaseAlignsUpdateAndInsertActions() { + createAndInitTable("id INT, a INT, b STRING", "{ \"id\": 1, \"a\": 2, \"b\": \"str\" }"); + createOrReplaceView( + "source", + "{ \"id\": 1, \"c1\": -2, \"c2\": \"new_str_1\" }\n" + + "{ \"id\": 2, \"c1\": -20, \"c2\": \"new_str_2\" }"); + + sql( + "MERGE INTO %s t USING source " + + "ON t.iD == source.Id " + + "WHEN MATCHED THEN " + + " UPDATE SET B = c2, A = c1, t.Id = source.ID " + + "WHEN NOT MATCHED THEN " + + " INSERT (b, A, iD) VALUES (c2, c1, id)", + commitTarget()); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, -2, "new_str_1"), row(2, -20, "new_str_2")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, -2, "new_str_1")), + sql("SELECT * FROM %s WHERE id = 1 ORDER BY id", selectTarget())); + assertEquals( + "Output should match", + ImmutableList.of(row(2, -20, "new_str_2")), + sql("SELECT * FROM %s WHERE b = 'new_str_2'ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeUpdatesNestedStructFields() { + createAndInitTable( + "id INT, s STRUCT,m:MAP>>", + "{ \"id\": 1, \"s\": { \"c1\": 2, \"c2\": { \"a\": [1,2], \"m\": { \"a\": \"b\"} } } } }"); + createOrReplaceView("source", "{ \"id\": 1, \"c1\": -2 }"); + + // update primitive, array, map columns inside a struct + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.id " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.c1 = source.c1, t.s.c2.a = array(-1, -2), t.s.c2.m = map('k', 'v')", + commitTarget()); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, row(-2, row(ImmutableList.of(-1, -2), ImmutableMap.of("k", "v"))))), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + // set primitive, array, map columns to NULL (proper casts should be in place) + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.id " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.c1 = NULL, t.s.c2 = NULL", + commitTarget()); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, row(null, null))), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + // update all fields in a struct + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.id " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s = named_struct('c1', 100, 'c2', named_struct('a', array(1), 'm', map('x', 'y')))", + commitTarget()); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, row(100, row(ImmutableList.of(1), ImmutableMap.of("x", "y"))))), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithInferredCasts() { + createAndInitTable("id INT, s STRING", "{ \"id\": 1, \"s\": \"value\" }"); + createOrReplaceView("source", "{ \"id\": 1, \"c1\": -2}"); + + // -2 in source should be casted to "-2" in target + sql( + "MERGE INTO %s t USING source " + + "ON t.id == source.id " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s = source.c1", + commitTarget()); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, "-2")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeModifiesNullStruct() { + createAndInitTable("id INT, s STRUCT", "{ \"id\": 1, \"s\": null }"); + createOrReplaceView("source", "{ \"id\": 1, \"n1\": -10 }"); + + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.n1 = s.n1", + commitTarget()); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, row(-10, null))), + sql("SELECT * FROM %s", selectTarget())); + } + + @TestTemplate + public void testMergeRefreshesRelationCache() { + createAndInitTable("id INT, name STRING", "{ \"id\": 1, \"name\": \"n1\" }"); + createOrReplaceView("source", "{ \"id\": 1, \"name\": \"n2\" }"); + + Dataset query = spark.sql("SELECT name FROM " + commitTarget()); + query.createOrReplaceTempView("tmp"); + + spark.sql("CACHE TABLE tmp"); + + assertEquals( + "View should have correct data", ImmutableList.of(row("n1")), sql("SELECT * FROM tmp")); + + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET t.name = s.name", + commitTarget()); + + assertEquals( + "View should have correct data", ImmutableList.of(row("n2")), sql("SELECT * FROM tmp")); + + spark.sql("UNCACHE TABLE tmp"); + } + + @TestTemplate + public void testMergeWithMultipleNotMatchedActions() { + createAndInitTable("id INT, dep STRING", "{ \"id\": 0, \"dep\": \"emp-id-0\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 3, \"dep\": \"emp-id-3\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN NOT MATCHED AND s.id = 1 THEN " + + " INSERT (dep, id) VALUES (s.dep, -1)" + + "WHEN NOT MATCHED THEN " + + " INSERT *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(-1, "emp-id-1"), // new + row(0, "emp-id-0"), // kept + row(2, "emp-id-2"), // new + row(3, "emp-id-3") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithMultipleConditionalNotMatchedActions() { + createAndInitTable("id INT, dep STRING", "{ \"id\": 0, \"dep\": \"emp-id-0\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 3, \"dep\": \"emp-id-3\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN NOT MATCHED AND s.id = 1 THEN " + + " INSERT (dep, id) VALUES (s.dep, -1)" + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(-1, "emp-id-1"), // new + row(0, "emp-id-0"), // kept + row(2, "emp-id-2") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeResolvesColumnsByName() { + createAndInitTable( + "id INT, badge INT, dep STRING", + "{ \"id\": 1, \"badge\": 1000, \"dep\": \"emp-id-one\" }\n" + + "{ \"id\": 6, \"badge\": 6000, \"dep\": \"emp-id-6\" }"); + + createOrReplaceView( + "source", + "badge INT, id INT, dep STRING", + "{ \"badge\": 1001, \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"badge\": 6006, \"id\": 6, \"dep\": \"emp-id-6\" }\n" + + "{ \"badge\": 7007, \"id\": 7, \"dep\": \"emp-id-7\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET * " + + "WHEN NOT MATCHED THEN " + + " INSERT * ", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, 1001, "emp-id-1"), // updated + row(6, 6006, "emp-id-6"), // updated + row(7, 7007, "emp-id-7") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT id, badge, dep FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeShouldResolveWhenThereAreNoUnresolvedExpressionsOrColumns() { + // ensures that MERGE INTO will resolve into the correct action even if no columns + // or otherwise unresolved expressions exist in the query (testing SPARK-34962) + createAndInitTable("id INT, dep STRING"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 3, \"dep\": \"emp-id-3\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON 1 != 1 " + + "WHEN MATCHED THEN " + + " UPDATE SET * " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + tableName); + createBranchIfNeeded(); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "emp-id-1"), // new + row(2, "emp-id-2"), // new + row(3, "emp-id-3") // new + ); + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithTableWithNonNullableColumn() { + createAndInitTable( + "id INT NOT NULL, dep STRING", + "{ \"id\": 1, \"dep\": \"emp-id-one\" }\n" + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + createOrReplaceView( + "source", + "id INT NOT NULL, dep STRING", + "{ \"id\": 2, \"dep\": \"emp-id-2\" }\n" + + "{ \"id\": 1, \"dep\": \"emp-id-1\" }\n" + + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET * " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, "emp-id-1"), // updated + row(2, "emp-id-2")); // new + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithNonExistingColumns() { + createAndInitTable( + "id INT, c STRUCT>", + "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.invalid_col = s.c2", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "A column, variable, or function parameter with name `t`.`invalid_col` cannot be resolved"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n2.invalid_col = s.c2", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("No such struct field `invalid_col`"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n2.dn1 = s.c2 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, invalid_col) VALUES (s.c1, null)", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "A column, variable, or function parameter with name `invalid_col` cannot be resolved"); + } + + @TestTemplate + public void testMergeWithInvalidColumnsInInsert() { + createAndInitTable( + "id INT, c STRUCT> NOT NULL", + "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n2.dn1 = s.c2 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, c.n2) VALUES (s.c1, null)", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("INSERT assignment keys cannot be nested fields"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n2.dn1 = s.c2 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, id) VALUES (s.c1, null)", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Multiple assignments for 'id'"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id) VALUES (s.c1)", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("No assignment for 'c'"); + } + + @TestTemplate + public void testMergeWithMissingOptionalColumnsInInsert() { + createAndInitTable("id INT, value LONG", "{ \"id\": 1, \"value\": 100}"); + createOrReplaceView("source", "{ \"c1\": 2, \"c2\": 200 }"); + + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id) VALUES (s.c1)", + commitTarget()); + + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(1, 100L), // existing + row(2, null)), // new + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithInvalidUpdates() { + createAndInitTable( + "id INT, a ARRAY>, m MAP", + "{ \"id\": 1, \"a\": [ { \"c1\": 2, \"c2\": 3 } ], \"m\": { \"k\": \"v\"} }"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.a.c1 = s.c2", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updating nested fields is only supported for StructType"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.m.key = 'new_key'", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updating nested fields is only supported for StructType"); + } + + @TestTemplate + public void testMergeWithConflictingUpdates() { + createAndInitTable( + "id INT, c STRUCT>", + "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.id = 1, t.c.n1 = 2, t.id = 2", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Multiple assignments for 'id"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n1 = 1, t.id = 2, t.c.n1 = 2", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Multiple assignments for 'c.n1'"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET c.n1 = 1, c = named_struct('n1', 1, 'n2', named_struct('dn1', 1, 'dn2', 2))", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Conflicting assignments for 'c'"); + } + + @TestTemplate + public void testMergeWithInvalidAssignmentsAnsi() { + createAndInitTable( + "id INT NOT NULL, s STRUCT> NOT NULL", + "{ \"id\": 1, \"s\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); + createOrReplaceView( + "source", + "c1 INT, c2 STRUCT NOT NULL, c3 STRING NOT NULL, c4 STRUCT", + "{ \"c1\": 1, \"c2\": { \"n1\" : 1 }, \"c3\" : 'str', \"c4\": { \"dn3\": 1, \"dn1\": 2 } }"); + + withSQLConf( + ImmutableMap.of(SQLConf.STORE_ASSIGNMENT_POLICY().key(), "ansi"), + () -> { + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.id = cast(NULL as int)", + commitTarget())) + .isInstanceOf(SparkRuntimeException.class) + .hasMessageContaining( + "[NOT_NULL_ASSERT_VIOLATION] NULL value appeared in non-nullable field"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.n1 = NULL", + commitTarget())) + .isInstanceOf(SparkRuntimeException.class) + .hasMessageContaining( + "[NOT_NULL_ASSERT_VIOLATION] NULL value appeared in non-nullable field"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s = s.c2", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot find data for the output column `s`.`n2`"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.n1 = s.c3", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot safely cast `s`.`n1` \"STRING\" to \"INT\"."); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.n2 = s.c4", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot find data for the output column `s`.`n2`.`dn2`"); + }); + } + + @TestTemplate + public void testMergeWithInvalidAssignmentsStrict() { + createAndInitTable( + "id INT NOT NULL, s STRUCT> NOT NULL", + "{ \"id\": 1, \"s\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); + createOrReplaceView( + "source", + "c1 INT, c2 STRUCT NOT NULL, c3 STRING NOT NULL, c4 STRUCT", + "{ \"c1\": 1, \"c2\": { \"n1\" : 1 }, \"c3\" : 'str', \"c4\": { \"dn3\": 1, \"dn1\": 2 } }"); + + withSQLConf( + ImmutableMap.of(SQLConf.STORE_ASSIGNMENT_POLICY().key(), "strict"), + () -> { + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.id = NULL", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot safely cast `id` \"VOID\" to \"INT\""); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.n1 = NULL", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot safely cast `s`.`n1` \"VOID\" to \"INT\""); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s = s.c2", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot find data for the output column `s`.`n2`"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.n1 = s.c3", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot safely cast `s`.`n1` \"STRING\" to \"INT\"."); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.n2 = s.c4", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot find data for the output column `s`.`n2`.`dn2`"); + }); + } + + @TestTemplate + public void testMergeWithNonDeterministicConditions() { + createAndInitTable( + "id INT, c STRUCT>", + "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 AND rand() > t.id " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n1 = -1", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "MERGE operation contains unsupported SEARCH condition. Non-deterministic expressions are not allowed"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND rand() > t.id THEN " + + " UPDATE SET t.c.n1 = -1", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "MERGE operation contains unsupported UPDATE condition. Non-deterministic expressions are not allowed"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND rand() > t.id THEN " + + " DELETE", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "MERGE operation contains unsupported DELETE condition. Non-deterministic expressions are not allowed"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN NOT MATCHED AND rand() > c1 THEN " + + " INSERT (id, c) VALUES (1, null)", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "MERGE operation contains unsupported INSERT condition. Non-deterministic expressions are not allowed"); + } + + @TestTemplate + public void testMergeWithAggregateExpressions() { + createAndInitTable( + "id INT, c STRUCT>", + "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 AND max(t.id) == 1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n1 = -1", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "MERGE operation contains unsupported SEARCH condition. Aggregates are not allowed"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND sum(t.id) < 1 THEN " + + " UPDATE SET t.c.n1 = -1", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "MERGE operation contains unsupported UPDATE condition. Aggregates are not allowed"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND sum(t.id) THEN " + + " DELETE", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "MERGE operation contains unsupported DELETE condition. Aggregates are not allowed"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN NOT MATCHED AND sum(c1) < 1 THEN " + + " INSERT (id, c) VALUES (1, null)", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "MERGE operation contains unsupported INSERT condition. Aggregates are not allowed"); + } + + @TestTemplate + public void testMergeWithSubqueriesInConditions() { + createAndInitTable( + "id INT, c STRUCT>", + "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 AND t.id < (SELECT max(c2) FROM source) " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n1 = s.c2", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "MERGE operation contains unsupported SEARCH condition. Subqueries are not allowed"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND t.id < (SELECT max(c2) FROM source) THEN " + + " UPDATE SET t.c.n1 = s.c2", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "MERGE operation contains unsupported UPDATE condition. Subqueries are not allowed"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND t.id NOT IN (SELECT c2 FROM source) THEN " + + " DELETE", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "MERGE operation contains unsupported DELETE condition. Subqueries are not allowed"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN NOT MATCHED AND s.c1 IN (SELECT c2 FROM source) THEN " + + " INSERT (id, c) VALUES (1, null)", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "MERGE operation contains unsupported INSERT condition. Subqueries are not allowed"); + } + + @TestTemplate + public void testMergeWithTargetColumnsInInsertConditions() { + createAndInitTable("id INT, c2 INT", "{ \"id\": 1, \"c2\": 2 }"); + createOrReplaceView("source", "{ \"id\": 1, \"value\": 11 }"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.id " + + "WHEN NOT MATCHED AND c2 = 1 THEN " + + " INSERT (id, c2) VALUES (s.id, null)", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "A column, variable, or function parameter with name `c2` cannot be resolved"); + } + + @TestTemplate + public void testMergeWithNonIcebergTargetTableNotSupported() { + createOrReplaceView("target", "{ \"c1\": -100, \"c2\": -200 }"); + createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); + + assertThatThrownBy( + () -> + sql( + "MERGE INTO target t USING source s " + + "ON t.c1 == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET *")) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("MERGE INTO TABLE is not supported temporarily."); + } + + /** + * Tests a merge where both the source and target are evaluated to be partitioned by + * SingePartition at planning time but DynamicFileFilterExec will return an empty target. + */ + @TestTemplate + public void testMergeSinglePartitionPartitioning() { + // This table will only have a single file and a single partition + createAndInitTable("id INT", "{\"id\": -1}"); + + // Coalesce forces our source into a SinglePartition distribution + spark.range(0, 5).coalesce(1).createOrReplaceTempView("source"); + + sql( + "MERGE INTO %s t USING source s ON t.id = s.id " + + "WHEN MATCHED THEN UPDATE SET *" + + "WHEN NOT MATCHED THEN INSERT *", + commitTarget()); + + ImmutableList expectedRows = + ImmutableList.of(row(-1), row(0), row(1), row(2), row(3), row(4)); + + List result = sql("SELECT * FROM %s ORDER BY id", selectTarget()); + assertEquals("Should correctly add the non-matching rows", expectedRows, result); + } + + @TestTemplate + public void testMergeEmptyTable() { + assumeThat(branch).as("Custom branch does not exist for empty table").isNotEqualTo("test"); + // This table will only have a single file and a single partition + createAndInitTable("id INT", null); + + // Coalesce forces our source into a SinglePartition distribution + spark.range(0, 5).coalesce(1).createOrReplaceTempView("source"); + + sql( + "MERGE INTO %s t USING source s ON t.id = s.id " + + "WHEN MATCHED THEN UPDATE SET *" + + "WHEN NOT MATCHED THEN INSERT *", + commitTarget()); + + ImmutableList expectedRows = ImmutableList.of(row(0), row(1), row(2), row(3), row(4)); + + List result = sql("SELECT * FROM %s ORDER BY id", selectTarget()); + assertEquals("Should correctly add the non-matching rows", expectedRows, result); + } + + @TestTemplate + public void testMergeNonExistingBranch() { + assumeThat(branch).as("Test only applicable to custom branch").isEqualTo("test"); + createAndInitTable("id INT", null); + + // Coalesce forces our source into a SinglePartition distribution + spark.range(0, 5).coalesce(1).createOrReplaceTempView("source"); + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s ON t.id = s.id " + + "WHEN MATCHED THEN UPDATE SET *" + + "WHEN NOT MATCHED THEN INSERT *", + commitTarget())) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot use branch (does not exist): test"); + } + + @TestTemplate + public void testMergeToWapBranch() { + assumeThat(branch).as("WAP branch only works for table identifier without branch").isNull(); + + createAndInitTable("id INT", "{\"id\": -1}"); + ImmutableList originalRows = ImmutableList.of(row(-1)); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' = 'true')", + tableName, TableProperties.WRITE_AUDIT_PUBLISH_ENABLED); + spark.range(0, 5).coalesce(1).createOrReplaceTempView("source"); + ImmutableList expectedRows = + ImmutableList.of(row(-1), row(0), row(1), row(2), row(3), row(4)); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), + () -> { + sql( + "MERGE INTO %s t USING source s ON t.id = s.id " + + "WHEN MATCHED THEN UPDATE SET *" + + "WHEN NOT MATCHED THEN INSERT *", + tableName); + assertEquals( + "Should have expected rows when reading table", + expectedRows, + sql("SELECT * FROM %s ORDER BY id", tableName)); + assertEquals( + "Should have expected rows when reading WAP branch", + expectedRows, + sql("SELECT * FROM %s.branch_wap ORDER BY id", tableName)); + assertEquals( + "Should not modify main branch", + originalRows, + sql("SELECT * FROM %s.branch_main ORDER BY id", tableName)); + }); + + spark.range(3, 6).coalesce(1).createOrReplaceTempView("source2"); + ImmutableList expectedRows2 = + ImmutableList.of(row(-1), row(0), row(1), row(2), row(5)); + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), + () -> { + sql( + "MERGE INTO %s t USING source2 s ON t.id = s.id " + + "WHEN MATCHED THEN DELETE " + + "WHEN NOT MATCHED THEN INSERT *", + tableName); + assertEquals( + "Should have expected rows when reading table with multiple writes", + expectedRows2, + sql("SELECT * FROM %s ORDER BY id", tableName)); + assertEquals( + "Should have expected rows when reading WAP branch with multiple writes", + expectedRows2, + sql("SELECT * FROM %s.branch_wap ORDER BY id", tableName)); + assertEquals( + "Should not modify main branch with multiple writes", + originalRows, + sql("SELECT * FROM %s.branch_main ORDER BY id", tableName)); + }); + } + + @TestTemplate + public void testMergeToWapBranchWithTableBranchIdentifier() { + assumeThat(branch).as("Test must have branch name part in table identifier").isNotNull(); + + createAndInitTable("id INT", "{\"id\": -1}"); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' = 'true')", + tableName, TableProperties.WRITE_AUDIT_PUBLISH_ENABLED); + spark.range(0, 5).coalesce(1).createOrReplaceTempView("source"); + ImmutableList expectedRows = + ImmutableList.of(row(-1), row(0), row(1), row(2), row(3), row(4)); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), + () -> + assertThatThrownBy( + () -> + sql( + "MERGE INTO %s t USING source s ON t.id = s.id " + + "WHEN MATCHED THEN UPDATE SET *" + + "WHEN NOT MATCHED THEN INSERT *", + commitTarget())) + .isInstanceOf(ValidationException.class) + .hasMessage( + String.format( + "Cannot write to both branch and WAP branch, but got branch [%s] and WAP branch [wap]", + branch))); + } + + private void checkJoinAndFilterConditions(String query, String join, String icebergFilters) { + // disable runtime filtering for easier validation + withSQLConf( + ImmutableMap.of( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED().key(), "false", + SQLConf.RUNTIME_ROW_LEVEL_OPERATION_GROUP_FILTER_ENABLED().key(), "false"), + () -> { + SparkPlan sparkPlan = executeAndKeepPlan(() -> sql(query)); + String planAsString = sparkPlan.toString().replaceAll("#(\\d+L?)", ""); + + assertThat(planAsString).as("Join should match").contains(join + "\n"); + + assertThat(planAsString) + .as("Pushed filters must match") + .contains("[filters=" + icebergFilters + ","); + }); + } + + private RowLevelOperationMode mode(Table table) { + String modeName = table.properties().getOrDefault(MERGE_MODE, MERGE_MODE_DEFAULT); + return RowLevelOperationMode.fromName(modeName); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java new file mode 100644 index 000000000000..bf9c53f82dbd --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java @@ -0,0 +1,352 @@ +/* + * 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.extensions; + +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 static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.exceptions.CommitStateUnknownException; +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.Lists; +import org.apache.iceberg.spark.SparkSQLProperties; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.iceberg.spark.source.SparkTable; +import org.apache.iceberg.spark.source.TestSparkCatalog; +import org.apache.iceberg.util.ContentFileUtil; +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.connector.catalog.Identifier; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMergeOnReadDelete extends TestDelete { + + @Override + protected Map extraTableProperties() { + return ImmutableMap.of( + TableProperties.DELETE_MODE, RowLevelOperationMode.MERGE_ON_READ.modeName()); + } + + @BeforeEach + public void clearTestSparkCatalogCache() { + TestSparkCatalog.clearTables(); + } + + @TestTemplate + public void testDeleteWithExecutorCacheLocality() throws NoSuchTableException { + createAndInitPartitionedTable(); + + append(tableName, new Employee(1, "hr"), new Employee(2, "hr")); + append(tableName, new Employee(3, "hr"), new Employee(4, "hr")); + append(tableName, new Employee(1, "hardware"), new Employee(2, "hardware")); + append(tableName, new Employee(3, "hardware"), new Employee(4, "hardware")); + + createBranchIfNeeded(); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.EXECUTOR_CACHE_LOCALITY_ENABLED, "true"), + () -> { + sql("DELETE FROM %s WHERE id = 1", commitTarget()); + sql("DELETE FROM %s WHERE id = 3", commitTarget()); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(2, "hr"), row(4, "hardware"), row(4, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC, dep ASC", selectTarget())); + }); + } + + @TestTemplate + public void testDeleteFileGranularity() throws NoSuchTableException { + assumeThat(formatVersion).isEqualTo(2); + checkDeleteFileGranularity(DeleteGranularity.FILE); + } + + @TestTemplate + public void testDeletePartitionGranularity() throws NoSuchTableException { + assumeThat(formatVersion).isEqualTo(2); + checkDeleteFileGranularity(DeleteGranularity.PARTITION); + } + + @TestTemplate + public void testPositionDeletesAreMaintainedDuringDelete() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id int, data string) USING iceberg PARTITIONED BY (id) TBLPROPERTIES" + + "('%s'='%s', '%s'='%s', '%s'='%s')", + tableName, + TableProperties.FORMAT_VERSION, + 2, + TableProperties.DELETE_MODE, + "merge-on-read", + TableProperties.DELETE_GRANULARITY, + "file"); + createBranchIfNeeded(); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(1, "b"), + new SimpleRecord(1, "c"), + new SimpleRecord(2, "d"), + new SimpleRecord(2, "e")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(commitTarget()) + .append(); + + sql("DELETE FROM %s WHERE id = 1 and data='a'", commitTarget()); + sql("DELETE FROM %s WHERE id = 2 and data='d'", commitTarget()); + sql("DELETE FROM %s WHERE id = 1 and data='c'", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot latest = SnapshotUtil.latestSnapshot(table, branch); + assertThat(latest.removedDeleteFiles(table.io())).hasSize(1); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "b"), row(2, "e")), + sql("SELECT * FROM %s ORDER BY id ASC", selectTarget())); + } + + @TestTemplate + public void testUnpartitionedPositionDeletesAreMaintainedDuringDelete() + throws NoSuchTableException { + sql( + "CREATE TABLE %s (id int, data string) USING iceberg TBLPROPERTIES" + + "('%s'='%s', '%s'='%s', '%s'='%s')", + tableName, + TableProperties.FORMAT_VERSION, + 2, + TableProperties.DELETE_MODE, + "merge-on-read", + TableProperties.DELETE_GRANULARITY, + "file"); + createBranchIfNeeded(); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(1, "b"), + new SimpleRecord(1, "c"), + new SimpleRecord(2, "d"), + new SimpleRecord(2, "e")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(commitTarget()) + .append(); + + sql("DELETE FROM %s WHERE id = 1 and data='a'", commitTarget()); + sql("DELETE FROM %s WHERE id = 2 and data='d'", commitTarget()); + sql("DELETE FROM %s WHERE id = 1 and data='c'", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot latest = SnapshotUtil.latestSnapshot(table, branch); + assertThat(latest.removedDeleteFiles(table.io())).hasSize(1); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "b"), row(2, "e")), + sql("SELECT * FROM %s ORDER BY id ASC", selectTarget())); + } + + @TestTemplate + public void testDeleteWithDVAndHistoricalPositionDeletes() { + assumeThat(formatVersion).isEqualTo(2); + createTableWithDeleteGranularity( + "id INT, dep STRING", "PARTITIONED BY (dep)", DeleteGranularity.PARTITION); + createBranchIfNeeded(); + append( + commitTarget(), + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hr\" }\n" + + "{ \"id\": 3, \"dep\": \"hr\" }"); + append( + commitTarget(), + "{ \"id\": 4, \"dep\": \"hr\" }\n" + + "{ \"id\": 5, \"dep\": \"hr\" }\n" + + "{ \"id\": 6, \"dep\": \"hr\" }"); + + // Produce partition scoped deletes for the two modified files + sql("DELETE FROM %s WHERE id = 1 or id = 4", commitTarget()); + + // Produce 1 file-scoped deletes for the second update + Map fileGranularityProps = + ImmutableMap.of(TableProperties.DELETE_GRANULARITY, DeleteGranularity.FILE.toString()); + sql( + "ALTER TABLE %s SET TBLPROPERTIES (%s)", + tableName, tablePropsAsString(fileGranularityProps)); + sql("DELETE FROM %s WHERE id = 5", commitTarget()); + + // Produce a DV which will contain 3 positions from the second data file + // 2 existing deleted positions from the earlier file-scoped and partition-scoped deletes + // and 1 new deleted position + Map updateFormatProperties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, "3"); + sql( + "ALTER TABLE %s SET TBLPROPERTIES (%s)", + tableName, tablePropsAsString(updateFormatProperties)); + sql("DELETE FROM %s where id = 6", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + Set deleteFiles = + TestHelpers.deleteFiles(table, SnapshotUtil.latestSnapshot(table, branch)); + List dvs = + deleteFiles.stream().filter(ContentFileUtil::isDV).collect(Collectors.toList()); + assertThat(dvs).hasSize(1); + assertThat(dvs).allMatch(dv -> dv.recordCount() == 3); + assertThat(dvs).allMatch(dv -> FileFormat.fromFileName(dv.location()) == FileFormat.PUFFIN); + } + + private void checkDeleteFileGranularity(DeleteGranularity deleteGranularity) + throws NoSuchTableException { + createTableWithDeleteGranularity( + "id INT, dep STRING", "PARTITIONED BY (dep)", deleteGranularity); + + append(tableName, new Employee(1, "hr"), new Employee(2, "hr")); + append(tableName, new Employee(3, "hr"), new Employee(4, "hr")); + append(tableName, new Employee(1, "hardware"), new Employee(2, "hardware")); + append(tableName, new Employee(3, "hardware"), new Employee(4, "hardware")); + + createBranchIfNeeded(); + + sql("DELETE FROM %s WHERE id = 1 OR id = 3", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).hasSize(5); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + String expectedDeleteFilesCount = deleteGranularity == DeleteGranularity.FILE ? "4" : "2"; + validateMergeOnRead(currentSnapshot, "2", expectedDeleteFilesCount, null); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(2, "hardware"), row(2, "hr"), row(4, "hardware"), row(4, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC, dep ASC", selectTarget())); + } + + @TestTemplate + public void testCommitUnknownException() { + createAndInitTable("id INT, dep STRING, category STRING"); + + // write unpartitioned files + append(tableName, "{ \"id\": 1, \"dep\": \"hr\", \"category\": \"c1\"}"); + createBranchIfNeeded(); + append( + commitTarget(), + "{ \"id\": 2, \"dep\": \"hr\", \"category\": \"c1\" }\n" + + "{ \"id\": 3, \"dep\": \"hr\", \"category\": \"c1\" }"); + + Table table = validationCatalog.loadTable(tableIdent); + + RowDelta newRowDelta = table.newRowDelta(); + if (branch != null) { + newRowDelta.toBranch(branch); + } + + RowDelta spyNewRowDelta = spy(newRowDelta); + doAnswer( + invocation -> { + newRowDelta.commit(); + throw new CommitStateUnknownException(new RuntimeException("Datacenter on Fire")); + }) + .when(spyNewRowDelta) + .commit(); + + Table spyTable = spy(table); + when(spyTable.newRowDelta()).thenReturn(spyNewRowDelta); + SparkTable sparkTable = + branch == null ? new SparkTable(spyTable, false) : new SparkTable(spyTable, branch, false); + + ImmutableMap config = + ImmutableMap.of( + "type", "hive", + "default-namespace", "default"); + spark + .conf() + .set("spark.sql.catalog.dummy_catalog", "org.apache.iceberg.spark.source.TestSparkCatalog"); + config.forEach( + (key, value) -> spark.conf().set("spark.sql.catalog.dummy_catalog." + key, value)); + Identifier ident = Identifier.of(new String[] {"default"}, "table"); + TestSparkCatalog.setTable(ident, sparkTable); + + // Although an exception is thrown here, write and commit have succeeded + assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id = 2", "dummy_catalog.default.table")) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("Datacenter on Fire"); + + // Since write and commit succeeded, the rows should be readable + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr", "c1"), row(3, "hr", "c1")), + sql("SELECT * FROM %s ORDER BY id", "dummy_catalog.default.table")); + } + + @TestTemplate + public void testAggregatePushDownInMergeOnReadDelete() { + createAndInitTable("id LONG, data INT"); + sql( + "INSERT INTO TABLE %s VALUES (1, 1111), (1, 2222), (2, 3333), (2, 4444), (3, 5555), (3, 6666) ", + tableName); + createBranchIfNeeded(); + + sql("DELETE FROM %s WHERE data = 1111", commitTarget()); + String select = "SELECT max(data), min(data), count(data) FROM %s"; + + List explain = sql("EXPLAIN " + select, selectTarget()); + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates = false; + if (explainString.contains("max(data)") + || explainString.contains("min(data)") + || explainString.contains("count(data)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("min/max/count not pushed down for deleted") + .isFalse(); + + List actual = sql(select, selectTarget()); + List expected = Lists.newArrayList(); + expected.add(new Object[] {6666, 2222, 5L}); + assertEquals("min/max/count push down", expected, actual); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadMerge.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadMerge.java new file mode 100644 index 000000000000..737f19e86a95 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadMerge.java @@ -0,0 +1,173 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +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.deletes.DeleteGranularity; +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.data.TestHelpers; +import org.apache.iceberg.util.ContentFileUtil; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.sql.Encoders; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMergeOnReadMerge extends TestMerge { + + @Override + protected Map extraTableProperties() { + return ImmutableMap.of( + TableProperties.MERGE_MODE, RowLevelOperationMode.MERGE_ON_READ.modeName()); + } + + @TestTemplate + public void testMergeDeleteFileGranularity() { + assumeThat(formatVersion).isEqualTo(2); + checkMergeDeleteGranularity(DeleteGranularity.FILE); + } + + @TestTemplate + public void testMergeDeletePartitionGranularity() { + assumeThat(formatVersion).isEqualTo(2); + checkMergeDeleteGranularity(DeleteGranularity.PARTITION); + } + + @TestTemplate + public void testMergeWithDVAndHistoricalPositionDeletes() { + assumeThat(formatVersion).isEqualTo(2); + createTableWithDeleteGranularity( + "id INT, dep STRING", "PARTITIONED BY (dep)", DeleteGranularity.PARTITION); + createBranchIfNeeded(); + createOrReplaceView( + "source", IntStream.rangeClosed(1, 9).boxed().collect(Collectors.toList()), Encoders.INT()); + append( + commitTarget(), + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hr\" }\n" + + "{ \"id\": 3, \"dep\": \"hr\" }"); + append( + commitTarget(), + "{ \"id\": 4, \"dep\": \"hr\" }\n" + + "{ \"id\": 5, \"dep\": \"hr\" }\n" + + "{ \"id\": 6, \"dep\": \"hr\" }"); + + // Produce partition scoped deletes for the two modified files + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value and (id = 1 or id = 4) " + + "WHEN MATCHED THEN " + + " DELETE " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, dep) VALUES (-1, 'other')", + commitTarget()); + + // Produce 1 file-scoped deletes for the second update + Map fileGranularityProps = + ImmutableMap.of(TableProperties.DELETE_GRANULARITY, DeleteGranularity.FILE.toString()); + sql( + "ALTER TABLE %s SET TBLPROPERTIES (%s)", + tableName, tablePropsAsString(fileGranularityProps)); + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value and id = 5 " + + "WHEN MATCHED THEN " + + " UPDATE SET id = id + 2 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, dep) VALUES (-1, 'other')", + commitTarget()); + + Map updateFormatProperties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, "3"); + sql( + "ALTER TABLE %s SET TBLPROPERTIES (%s)", + tableName, tablePropsAsString(updateFormatProperties)); + + // Produce a DV which will contain 3 positions from the second data file + // 2 existing deleted positions from the earlier file-scoped and partition-scoped deletes + // and 1 new deleted position + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value and id = 6 " + + "WHEN MATCHED THEN " + + " UPDATE SET id = id + 1 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, dep) VALUES (-1, 'other')", + commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + Set deleteFiles = + TestHelpers.deleteFiles(table, SnapshotUtil.latestSnapshot(table, branch)); + List dvs = + deleteFiles.stream().filter(ContentFileUtil::isDV).collect(Collectors.toList()); + assertThat(dvs).hasSize(1); + assertThat(dvs).allMatch(dv -> dv.recordCount() == 3); + assertThat(dvs).allMatch(dv -> FileFormat.fromFileName(dv.location()) == FileFormat.PUFFIN); + } + + private void checkMergeDeleteGranularity(DeleteGranularity deleteGranularity) { + createTableWithDeleteGranularity( + "id INT, dep STRING", "PARTITIONED BY (dep)", deleteGranularity); + + append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 3, \"dep\": \"hr\" }\n" + "{ \"id\": 4, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 1, \"dep\": \"it\" }\n" + "{ \"id\": 2, \"dep\": \"it\" }"); + append(tableName, "{ \"id\": 3, \"dep\": \"it\" }\n" + "{ \"id\": 4, \"dep\": \"it\" }"); + + createBranchIfNeeded(); + + createOrReplaceView("source", ImmutableList.of(1, 3, 5), Encoders.INT()); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value " + + "WHEN MATCHED THEN " + + " DELETE " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, dep) VALUES (-1, 'other')", + commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).hasSize(5); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + String expectedDeleteFilesCount = deleteGranularity == DeleteGranularity.FILE ? "4" : "2"; + validateMergeOnRead(currentSnapshot, "3", expectedDeleteFilesCount, "1"); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "other"), row(2, "hr"), row(2, "it"), row(4, "hr"), row(4, "it")), + sql("SELECT * FROM %s ORDER BY id ASC, dep ASC", selectTarget())); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java new file mode 100644 index 000000000000..1bec21b9b68d --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadUpdate.java @@ -0,0 +1,226 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +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.deletes.DeleteGranularity; +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.data.TestHelpers; +import org.apache.iceberg.util.ContentFileUtil; +import org.apache.iceberg.util.SnapshotUtil; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMergeOnReadUpdate extends TestUpdate { + + @Override + protected Map extraTableProperties() { + return ImmutableMap.of( + TableProperties.UPDATE_MODE, RowLevelOperationMode.MERGE_ON_READ.modeName()); + } + + @TestTemplate + public void testUpdateFileGranularity() { + assumeThat(formatVersion).isEqualTo(2); + checkUpdateFileGranularity(DeleteGranularity.FILE); + } + + @TestTemplate + public void testUpdatePartitionGranularity() { + assumeThat(formatVersion).isEqualTo(2); + checkUpdateFileGranularity(DeleteGranularity.PARTITION); + } + + @TestTemplate + public void testPositionDeletesAreMaintainedDuringUpdate() { + assumeThat(formatVersion).isEqualTo(2); + // Range distribution will produce partition scoped deletes which will not be cleaned up + assumeThat(distributionMode).isNotEqualToIgnoringCase("range"); + + checkUpdateFileGranularity(DeleteGranularity.FILE); + sql("UPDATE %s SET id = id + 1 WHERE id = 4", commitTarget()); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + String expectedDeleteFilesCount = "2"; + validateMergeOnRead(currentSnapshot, "2", expectedDeleteFilesCount, "2"); + + assertThat(currentSnapshot.removedDeleteFiles(table.io())).hasSize(2); + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(0, "hr"), + row(2, "hr"), + row(2, "hr"), + row(5, "hr"), + row(0, "it"), + row(2, "it"), + row(2, "it"), + row(5, "it")), + sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); + } + + @TestTemplate + public void testUnpartitionedPositionDeletesAreMaintainedDuringUpdate() { + assumeThat(formatVersion).isEqualTo(2); + // Range distribution will produce partition scoped deletes which will not be cleaned up + assumeThat(distributionMode).isNotEqualToIgnoringCase("range"); + initTable("", DeleteGranularity.FILE); + + sql("UPDATE %s SET id = id - 1 WHERE id = 1 OR id = 3", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).hasSize(5); + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + String expectedDeleteFilesCount = "4"; + validateMergeOnRead(currentSnapshot, "1", expectedDeleteFilesCount, "1"); + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(0, "hr"), + row(2, "hr"), + row(2, "hr"), + row(4, "hr"), + row(0, "it"), + row(2, "it"), + row(2, "it"), + row(4, "it")), + sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); + + sql("UPDATE %s SET id = id + 1 WHERE id = 4", commitTarget()); + table.refresh(); + currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + expectedDeleteFilesCount = "2"; + + validateMergeOnRead(currentSnapshot, "1", expectedDeleteFilesCount, "1"); + assertThat(currentSnapshot.removedDeleteFiles(table.io())).hasSize(2); + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(0, "hr"), + row(2, "hr"), + row(2, "hr"), + row(5, "hr"), + row(0, "it"), + row(2, "it"), + row(2, "it"), + row(5, "it")), + sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); + } + + private void checkUpdateFileGranularity(DeleteGranularity deleteGranularity) { + initTable("PARTITIONED BY (dep)", deleteGranularity); + + sql("UPDATE %s SET id = id - 1 WHERE id = 1 OR id = 3", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).hasSize(5); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + String expectedDeleteFilesCount = deleteGranularity == DeleteGranularity.FILE ? "4" : "2"; + validateMergeOnRead(currentSnapshot, "2", expectedDeleteFilesCount, "2"); + + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(0, "hr"), + row(2, "hr"), + row(2, "hr"), + row(4, "hr"), + row(0, "it"), + row(2, "it"), + row(2, "it"), + row(4, "it")), + sql("SELECT * FROM %s ORDER BY dep ASC, id ASC", selectTarget())); + } + + @TestTemplate + public void testUpdateWithDVAndHistoricalPositionDeletes() { + assumeThat(formatVersion).isEqualTo(2); + createTableWithDeleteGranularity( + "id INT, dep STRING", "PARTITIONED BY (dep)", DeleteGranularity.PARTITION); + createBranchIfNeeded(); + append( + commitTarget(), + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hr\" }\n" + + "{ \"id\": 3, \"dep\": \"hr\" }"); + append( + commitTarget(), + "{ \"id\": 4, \"dep\": \"hr\" }\n" + + "{ \"id\": 5, \"dep\": \"hr\" }\n" + + "{ \"id\": 6, \"dep\": \"hr\" }"); + + // Produce partition scoped deletes for the two modified files + sql("UPDATE %s SET id = id - 1 WHERE id = 1 or id = 4", commitTarget()); + + // Produce 1 file-scoped deletes for the second update + Map fileGranularityProps = + ImmutableMap.of(TableProperties.DELETE_GRANULARITY, DeleteGranularity.FILE.toString()); + sql( + "ALTER TABLE %s SET TBLPROPERTIES (%s)", + tableName, tablePropsAsString(fileGranularityProps)); + sql("UPDATE %s SET id = id + 2 WHERE id = 5", commitTarget()); + + Map updateFormatProperties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, "3"); + sql( + "ALTER TABLE %s SET TBLPROPERTIES (%s)", + tableName, tablePropsAsString(updateFormatProperties)); + + // Produce a DV which will contain 3 positions from the second data file + // 2 existing deleted positions from the earlier file-scoped and partition-scoped deletes + // and 1 new deleted position + sql("UPDATE %s SET id = id + 1 where id = 6", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + Set deleteFiles = + TestHelpers.deleteFiles(table, SnapshotUtil.latestSnapshot(table, branch)); + List dvs = + deleteFiles.stream().filter(ContentFileUtil::isDV).collect(Collectors.toList()); + assertThat(dvs).hasSize(1); + assertThat(dvs.get(0).recordCount()).isEqualTo(3); + assertThat(dvs).allMatch(dv -> FileFormat.fromFileName(dv.location()) == FileFormat.PUFFIN); + } + + private void initTable(String partitionedBy, DeleteGranularity deleteGranularity) { + createTableWithDeleteGranularity("id INT, dep STRING", partitionedBy, deleteGranularity); + + append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 3, \"dep\": \"hr\" }\n" + "{ \"id\": 4, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 1, \"dep\": \"it\" }\n" + "{ \"id\": 2, \"dep\": \"it\" }"); + append(tableName, "{ \"id\": 3, \"dep\": \"it\" }\n" + "{ \"id\": 4, \"dep\": \"it\" }"); + + createBranchIfNeeded(); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadWithLineage.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadWithLineage.java new file mode 100644 index 000000000000..3f038276af63 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadWithLineage.java @@ -0,0 +1,35 @@ +/* + * 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.extensions; + +import java.util.Map; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestMergeOnReadWithLineage extends TestRowLevelOperationsWithLineage { + + @Override + protected Map extraTableProperties() { + return ImmutableMap.of( + TableProperties.MERGE_MODE, RowLevelOperationMode.MERGE_ON_READ.modeName(), + TableProperties.UPDATE_MODE, RowLevelOperationMode.MERGE_ON_READ.modeName(), + TableProperties.DELETE_MODE, RowLevelOperationMode.MERGE_ON_READ.modeName()); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java new file mode 100644 index 000000000000..b783a006ef73 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java @@ -0,0 +1,125 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.UUID; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.ScanTaskSetManager; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMetaColumnProjectionWithStageScan extends ExtensionsTestBase { + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties() + } + }; + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + private void stageTask( + Table tab, String fileSetID, CloseableIterable tasks) { + ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + taskSetManager.stageTasks(tab, fileSetID, Lists.newArrayList(tasks)); + } + + @TestTemplate + public void testReadStageTableMeta() throws Exception { + sql( + "CREATE TABLE %s (id bigint, data string) USING iceberg TBLPROPERTIES" + + "('format-version'='2', 'write.delete.mode'='merge-on-read')", + tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + table.refresh(); + String tableLocation = table.location(); + + try (CloseableIterable tasks = table.newBatchScan().planFiles()) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(table, fileSetID, tasks); + Dataset scanDF2 = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.FILE_OPEN_COST, "0") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .load(tableLocation); + + assertThat(scanDF2.columns()).hasSize(2); + } + + try (CloseableIterable tasks = table.newBatchScan().planFiles()) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(table, fileSetID, tasks); + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.FILE_OPEN_COST, "0") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .load(tableLocation) + .select("*", "_pos"); + + List rows = scanDF.collectAsList(); + ImmutableList expectedRows = + ImmutableList.of(row(1L, "a", 0L), row(2L, "b", 1L), row(3L, "c", 2L), row(4L, "d", 3L)); + assertEquals("result should match", expectedRows, rowsToJava(rows)); + } + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java new file mode 100644 index 000000000000..ac528d1c470e --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java @@ -0,0 +1,993 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_REST; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.util.Comparator; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.avro.generic.GenericData.Record; +import org.apache.commons.collections.ListUtils; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.HistoryEntry; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +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.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMetadataTables extends ExtensionsTestBase { + @Parameter(index = 3) + private int formatVersion; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, formatVersion = {3}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties(), + 2 + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + 2 + }, + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties(), + 2 + }, + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties(), + 3 + }, + { + SparkCatalogConfig.REST.catalogName(), + SparkCatalogConfig.REST.implementation(), + ImmutableMap.builder() + .putAll(SparkCatalogConfig.REST.properties()) + .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) + .build(), + 2 + }, + { + SparkCatalogConfig.REST.catalogName(), + SparkCatalogConfig.REST.implementation(), + ImmutableMap.builder() + .putAll(SparkCatalogConfig.REST.properties()) + .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) + .build(), + 3 + } + }; + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testUnpartitionedTable() throws Exception { + sql( + "CREATE TABLE %s (id bigint, data string) USING iceberg TBLPROPERTIES" + + "('format-version'='%s', 'write.delete.mode'='merge-on-read')", + tableName, formatVersion); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + sql("DELETE FROM %s WHERE id=1", tableName); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + List expectedDataManifests = TestHelpers.dataManifests(table); + List expectedDeleteManifests = TestHelpers.deleteManifests(table); + assertThat(expectedDataManifests).as("Should have 1 data manifest").hasSize(1); + assertThat(expectedDeleteManifests).as("Should have 1 delete manifest").hasSize(1); + + Schema entriesTableSchema = + TypeUtil.selectNot( + Spark3Util.loadIcebergTable(spark, tableName + ".entries").schema(), + Set.of(DataFile.FIRST_ROW_ID.fieldId())); + Schema filesTableSchema = Spark3Util.loadIcebergTable(spark, tableName + ".files").schema(); + + // check delete files table + Dataset actualDeleteFilesDs = spark.sql("SELECT * FROM " + tableName + ".delete_files"); + List actualDeleteFiles = TestHelpers.selectNonDerived(actualDeleteFilesDs).collectAsList(); + assertThat(actualDeleteFiles).as("Metadata table should return one delete file").hasSize(1); + + List expectedDeleteFiles = + expectedEntries( + table, FileContent.POSITION_DELETES, entriesTableSchema, expectedDeleteManifests, null); + assertThat(expectedDeleteFiles).as("Should be one delete file manifest entry").hasSize(1); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualDeleteFilesDs), + expectedDeleteFiles.get(0), + actualDeleteFiles.get(0)); + + // check data files table + Dataset actualDataFilesDs = spark.sql("SELECT * FROM " + tableName + ".data_files"); + List actualDataFiles = TestHelpers.selectNonDerived(actualDataFilesDs).collectAsList(); + assertThat(actualDataFiles).as("Metadata table should return one data file").hasSize(1); + + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); + assertThat(expectedDataFiles).as("Should be one data file manifest entry").hasSize(1); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualDataFilesDs), + expectedDataFiles.get(0), + actualDataFiles.get(0)); + + // check all files table + Dataset actualFilesDs = + spark.sql("SELECT * FROM " + tableName + ".files ORDER BY content"); + List actualFiles = TestHelpers.selectNonDerived(actualFilesDs).collectAsList(); + + assertThat(actualFiles).as("Metadata table should return two files").hasSize(2); + + List expectedFiles = + Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) + .collect(Collectors.toList()); + assertThat(expectedFiles).as("Should have two files manifest entries").hasSize(2); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(0), actualFiles.get(0)); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(1), actualFiles.get(1)); + } + + @TestTemplate + public void testPositionDeletesTable() throws Exception { + sql( + "CREATE TABLE %s (id bigint, data string) USING iceberg TBLPROPERTIES" + + "('format-version'='%s', 'write.delete.mode'='merge-on-read')", + tableName, formatVersion); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + sql("DELETE FROM %s WHERE id=1 OR id=3", tableName); + + // check delete files table + assertThat(sql("SELECT * FROM %s.delete_files", tableName)).hasSize(1); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + DataFile dataFile = Iterables.getOnlyElement(TestHelpers.dataFiles(table)); + DeleteFile deleteFile = Iterables.getOnlyElement(TestHelpers.deleteFiles(table)); + + List expectedRows; + if (formatVersion >= 3) { + expectedRows = + ImmutableList.of( + row( + dataFile.location(), + 0L, + null, + dataFile.specId(), + deleteFile.location(), + deleteFile.contentOffset(), + deleteFile.contentSizeInBytes()), + row( + dataFile.location(), + 2L, + null, + dataFile.specId(), + deleteFile.location(), + deleteFile.contentOffset(), + deleteFile.contentSizeInBytes())); + } else { + expectedRows = + ImmutableList.of( + row(dataFile.location(), 0L, null, dataFile.specId(), deleteFile.location()), + row(dataFile.location(), 2L, null, dataFile.specId(), deleteFile.location())); + } + + // check position_deletes table + assertThat(sql("SELECT * FROM %s.position_deletes", tableName)) + .hasSize(2) + .containsExactlyElementsOf(expectedRows); + } + + @TestTemplate + public void testPartitionedTable() throws Exception { + sql( + "CREATE TABLE %s (id bigint, data string) " + + "USING iceberg " + + "PARTITIONED BY (data) " + + "TBLPROPERTIES" + + "('format-version'='%s', 'write.delete.mode'='merge-on-read')", + tableName, formatVersion); + + List recordsA = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "a")); + spark + .createDataset(recordsA, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + List recordsB = + Lists.newArrayList(new SimpleRecord(1, "b"), new SimpleRecord(2, "b")); + spark + .createDataset(recordsB, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + sql("DELETE FROM %s WHERE id=1 AND data='a'", tableName); + sql("DELETE FROM %s WHERE id=1 AND data='b'", tableName); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + Schema entriesTableSchema = + TypeUtil.selectNot( + Spark3Util.loadIcebergTable(spark, tableName + ".entries").schema(), + Set.of(DataFile.FIRST_ROW_ID.fieldId())); + + List expectedDataManifests = TestHelpers.dataManifests(table); + List expectedDeleteManifests = TestHelpers.deleteManifests(table); + assertThat(expectedDataManifests).as("Should have 2 data manifest").hasSize(2); + assertThat(expectedDeleteManifests).as("Should have 2 delete manifest").hasSize(2); + + Schema filesTableSchema = + Spark3Util.loadIcebergTable(spark, tableName + ".delete_files").schema(); + + // Check delete files table + List expectedDeleteFiles = + expectedEntries( + table, FileContent.POSITION_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); + assertThat(expectedDeleteFiles).as("Should have one delete file manifest entry").hasSize(1); + + Dataset actualDeleteFilesDs = + spark.sql("SELECT * FROM " + tableName + ".delete_files " + "WHERE partition.data='a'"); + List actualDeleteFiles = TestHelpers.selectNonDerived(actualDeleteFilesDs).collectAsList(); + + assertThat(actualDeleteFiles).as("Metadata table should return one delete file").hasSize(1); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualDeleteFilesDs), + expectedDeleteFiles.get(0), + actualDeleteFiles.get(0)); + + // Check data files table + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); + assertThat(expectedDataFiles).as("Should have one data file manifest entry").hasSize(1); + + Dataset actualDataFilesDs = + spark.sql("SELECT * FROM " + tableName + ".data_files " + "WHERE partition.data='a'"); + + List actualDataFiles = TestHelpers.selectNonDerived(actualDataFilesDs).collectAsList(); + assertThat(actualDataFiles).as("Metadata table should return one data file").hasSize(1); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualDataFilesDs), + expectedDataFiles.get(0), + actualDataFiles.get(0)); + + List actualPartitionsWithProjection = + spark.sql("SELECT file_count FROM " + tableName + ".partitions ").collectAsList(); + assertThat(actualPartitionsWithProjection) + .as("Metadata table should return two partitions record") + .hasSize(2) + .containsExactly(RowFactory.create(1), RowFactory.create(1)); + + // Check files table + List expectedFiles = + Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) + .collect(Collectors.toList()); + assertThat(expectedFiles).as("Should have two file manifest entries").hasSize(2); + + Dataset actualFilesDs = + spark.sql( + "SELECT * FROM " + tableName + ".files " + "WHERE partition.data='a' ORDER BY content"); + List actualFiles = TestHelpers.selectNonDerived(actualFilesDs).collectAsList(); + assertThat(actualFiles).as("Metadata table should return two files").hasSize(2); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(0), actualFiles.get(0)); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(1), actualFiles.get(1)); + } + + @TestTemplate + public void testAllFilesUnpartitioned() throws Exception { + sql( + "CREATE TABLE %s (id bigint, data string) USING iceberg TBLPROPERTIES" + + "('format-version'='%s', 'write.delete.mode'='merge-on-read')", + tableName, formatVersion); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + // Create delete file + sql("DELETE FROM %s WHERE id=1", tableName); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + List expectedDataManifests = TestHelpers.dataManifests(table); + assertThat(expectedDataManifests).as("Should have 1 data manifest").hasSize(1); + List expectedDeleteManifests = TestHelpers.deleteManifests(table); + assertThat(expectedDeleteManifests).as("Should have 1 delete manifest").hasSize(1); + + // Clear table to test whether 'all_files' can read past files + List results = sql("DELETE FROM %s", tableName); + assertThat(results).as("Table should be cleared").isEmpty(); + + Schema entriesTableSchema = + TypeUtil.selectNot( + Spark3Util.loadIcebergTable(spark, tableName + ".entries").schema(), + Set.of(DataFile.FIRST_ROW_ID.fieldId())); + Schema filesTableSchema = + Spark3Util.loadIcebergTable(spark, tableName + ".all_data_files").schema(); + + // Check all data files table + Dataset actualDataFilesDs = spark.sql("SELECT * FROM " + tableName + ".all_data_files"); + List actualDataFiles = TestHelpers.selectNonDerived(actualDataFilesDs).collectAsList(); + + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); + assertThat(expectedDataFiles).as("Should be one data file manifest entry").hasSize(1); + assertThat(actualDataFiles).as("Metadata table should return one data file").hasSize(1); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualDataFilesDs), + expectedDataFiles.get(0), + actualDataFiles.get(0)); + + // Check all delete files table + Dataset actualDeleteFilesDs = + spark.sql("SELECT * FROM " + tableName + ".all_delete_files"); + List actualDeleteFiles = TestHelpers.selectNonDerived(actualDeleteFilesDs).collectAsList(); + List expectedDeleteFiles = + expectedEntries( + table, FileContent.POSITION_DELETES, entriesTableSchema, expectedDeleteManifests, null); + assertThat(expectedDeleteFiles).as("Should be one delete file manifest entry").hasSize(1); + assertThat(actualDeleteFiles).as("Metadata table should return one delete file").hasSize(1); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualDeleteFilesDs), + expectedDeleteFiles.get(0), + actualDeleteFiles.get(0)); + + // Check all files table + Dataset actualFilesDs = + spark.sql("SELECT * FROM " + tableName + ".all_files ORDER BY content"); + List actualFiles = TestHelpers.selectNonDerived(actualFilesDs).collectAsList(); + List expectedFiles = ListUtils.union(expectedDataFiles, expectedDeleteFiles); + expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); + assertThat(actualFiles).as("Metadata table should return two files").hasSize(2); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles, actualFiles); + } + + @TestTemplate + public void testAllFilesPartitioned() throws Exception { + // Create table and insert data + sql( + "CREATE TABLE %s (id bigint, data string) " + + "USING iceberg " + + "PARTITIONED BY (data) " + + "TBLPROPERTIES" + + "('format-version'='%s', 'write.delete.mode'='merge-on-read')", + tableName, formatVersion); + + List recordsA = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "a")); + spark + .createDataset(recordsA, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + List recordsB = + Lists.newArrayList(new SimpleRecord(1, "b"), new SimpleRecord(2, "b")); + spark + .createDataset(recordsB, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + // Create delete file + sql("DELETE FROM %s WHERE id=1", tableName); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + List expectedDataManifests = TestHelpers.dataManifests(table); + assertThat(expectedDataManifests).as("Should have 2 data manifests").hasSize(2); + List expectedDeleteManifests = TestHelpers.deleteManifests(table); + assertThat(expectedDeleteManifests).as("Should have 1 delete manifest").hasSize(1); + + // Clear table to test whether 'all_files' can read past files + List results = sql("DELETE FROM %s", tableName); + assertThat(results).as("Table should be cleared").isEmpty(); + + Schema entriesTableSchema = + TypeUtil.selectNot( + Spark3Util.loadIcebergTable(spark, tableName + ".entries").schema(), + Set.of(DataFile.FIRST_ROW_ID.fieldId())); + Schema filesTableSchema = + Spark3Util.loadIcebergTable(spark, tableName + ".all_data_files").schema(); + + // Check all data files table + Dataset actualDataFilesDs = + spark.sql("SELECT * FROM " + tableName + ".all_data_files " + "WHERE partition.data='a'"); + List actualDataFiles = TestHelpers.selectNonDerived(actualDataFilesDs).collectAsList(); + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); + assertThat(expectedDataFiles).as("Should be one data file manifest entry").hasSize(1); + assertThat(actualDataFiles).as("Metadata table should return one data file").hasSize(1); + TestHelpers.assertEqualsSafe( + SparkSchemaUtil.convert(TestHelpers.selectNonDerived(actualDataFilesDs).schema()) + .asStruct(), + expectedDataFiles.get(0), + actualDataFiles.get(0)); + + // Check all delete files table + Dataset actualDeleteFilesDs = + spark.sql("SELECT * FROM " + tableName + ".all_delete_files " + "WHERE partition.data='a'"); + List actualDeleteFiles = TestHelpers.selectNonDerived(actualDeleteFilesDs).collectAsList(); + + List expectedDeleteFiles = + expectedEntries( + table, FileContent.POSITION_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); + assertThat(expectedDeleteFiles).as("Should be one data file manifest entry").hasSize(1); + assertThat(actualDeleteFiles).as("Metadata table should return one data file").hasSize(1); + + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualDeleteFilesDs), + expectedDeleteFiles.get(0), + actualDeleteFiles.get(0)); + + // Check all files table + Dataset actualFilesDs = + spark.sql( + "SELECT * FROM " + + tableName + + ".all_files WHERE partition.data='a' " + + "ORDER BY content"); + List actualFiles = TestHelpers.selectNonDerived(actualFilesDs).collectAsList(); + + List expectedFiles = ListUtils.union(expectedDataFiles, expectedDeleteFiles); + expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); + assertThat(actualFiles).as("Metadata table should return two files").hasSize(2); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualDataFilesDs), expectedFiles, actualFiles); + } + + @TestTemplate + public void testMetadataLogEntries() throws Exception { + // Create table and insert data + sql( + "CREATE TABLE %s (id bigint, data string) " + + "USING iceberg " + + "PARTITIONED BY (data) " + + "TBLPROPERTIES " + + "('format-version'='%s')", + tableName, formatVersion); + + List recordsA = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "a")); + spark.createDataset(recordsA, Encoders.bean(SimpleRecord.class)).writeTo(tableName).append(); + + List recordsB = + Lists.newArrayList(new SimpleRecord(1, "b"), new SimpleRecord(2, "b")); + spark.createDataset(recordsB, Encoders.bean(SimpleRecord.class)).writeTo(tableName).append(); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + Long currentSnapshotId = table.currentSnapshot().snapshotId(); + TableMetadata tableMetadata = ((HasTableOperations) table).operations().current(); + Snapshot currentSnapshot = tableMetadata.currentSnapshot(); + Snapshot parentSnapshot = table.snapshot(currentSnapshot.parentId()); + List metadataLogEntries = + Lists.newArrayList(tableMetadata.previousFiles()); + + // Check metadataLog table + List metadataLogs = sql("SELECT * FROM %s.metadata_log_entries", tableName); + assertEquals( + "MetadataLogEntriesTable result should match the metadataLog entries", + ImmutableList.of( + row( + DateTimeUtils.toJavaTimestamp(metadataLogEntries.get(0).timestampMillis() * 1000), + metadataLogEntries.get(0).file(), + null, + null, + null), + row( + DateTimeUtils.toJavaTimestamp(metadataLogEntries.get(1).timestampMillis() * 1000), + metadataLogEntries.get(1).file(), + parentSnapshot.snapshotId(), + parentSnapshot.schemaId(), + parentSnapshot.sequenceNumber()), + row( + DateTimeUtils.toJavaTimestamp(currentSnapshot.timestampMillis() * 1000), + tableMetadata.metadataFileLocation(), + currentSnapshot.snapshotId(), + currentSnapshot.schemaId(), + currentSnapshot.sequenceNumber())), + metadataLogs); + + // test filtering + List metadataLogWithFilters = + sql( + "SELECT * FROM %s.metadata_log_entries WHERE latest_snapshot_id = %s", + tableName, currentSnapshotId); + assertThat(metadataLogWithFilters) + .as("metadataLogEntries table should return 1 row") + .hasSize(1); + assertEquals( + "Result should match the latest snapshot entry", + ImmutableList.of( + row( + DateTimeUtils.toJavaTimestamp( + tableMetadata.currentSnapshot().timestampMillis() * 1000), + tableMetadata.metadataFileLocation(), + tableMetadata.currentSnapshot().snapshotId(), + tableMetadata.currentSnapshot().schemaId(), + tableMetadata.currentSnapshot().sequenceNumber())), + metadataLogWithFilters); + + // test projection + List metadataFiles = + metadataLogEntries.stream() + .map(TableMetadata.MetadataLogEntry::file) + .collect(Collectors.toList()); + metadataFiles.add(tableMetadata.metadataFileLocation()); + List metadataLogWithProjection = + sql("SELECT file FROM %s.metadata_log_entries", tableName); + assertThat(metadataLogWithProjection) + .as("metadataLogEntries table should return 3 rows") + .hasSize(3); + assertEquals( + "metadataLog entry should be of same file", + metadataFiles.stream().map(this::row).collect(Collectors.toList()), + metadataLogWithProjection); + } + + @TestTemplate + public void testFilesTableTimeTravelWithSchemaEvolution() throws Exception { + // Create table and insert data + sql( + "CREATE TABLE %s (id bigint, data string) " + + "USING iceberg " + + "PARTITIONED BY (data) " + + "TBLPROPERTIES" + + "('format-version'='%s', 'write.delete.mode'='merge-on-read')", + tableName, formatVersion); + + List recordsA = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "a")); + spark + .createDataset(recordsA, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + table.updateSchema().addColumn("category", Types.StringType.get()).commit(); + + List newRecords = + Lists.newArrayList(RowFactory.create(3, "b", "c"), RowFactory.create(4, "b", "c")); + + StructType newSparkSchema = + SparkSchemaUtil.convert( + new Schema( + optional(1, "id", Types.IntegerType.get()), + optional(2, "data", Types.StringType.get()), + optional(3, "category", Types.StringType.get()))); + + spark.createDataFrame(newRecords, newSparkSchema).coalesce(1).writeTo(tableName).append(); + table.refresh(); + Long currentSnapshotId = table.currentSnapshot().snapshotId(); + + Dataset actualFilesDs = + spark.sql( + "SELECT * FROM " + + tableName + + ".files VERSION AS OF " + + currentSnapshotId + + " ORDER BY content"); + List actualFiles = TestHelpers.selectNonDerived(actualFilesDs).collectAsList(); + Schema entriesTableSchema = + TypeUtil.selectNot( + Spark3Util.loadIcebergTable(spark, tableName + ".entries").schema(), + Set.of(DataFile.FIRST_ROW_ID.fieldId())); + List expectedDataManifests = TestHelpers.dataManifests(table); + List expectedFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); + + assertThat(actualFiles).as("actualFiles size should be 2").hasSize(2); + + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(0), actualFiles.get(0)); + + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(1), actualFiles.get(1)); + + assertThat(actualFiles) + .as("expectedFiles and actualFiles size should be the same") + .hasSameSizeAs(expectedFiles); + } + + @TestTemplate + public void testSnapshotReferencesMetatable() throws Exception { + // Create table and insert data + sql( + "CREATE TABLE %s (id bigint, data string) " + + "USING iceberg " + + "PARTITIONED BY (data) " + + "TBLPROPERTIES" + + "('format-version'='%s', 'write.delete.mode'='merge-on-read')", + tableName, formatVersion); + + List recordsA = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "a")); + spark + .createDataset(recordsA, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + List recordsB = + Lists.newArrayList(new SimpleRecord(1, "b"), new SimpleRecord(2, "b")); + spark + .createDataset(recordsB, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + Long currentSnapshotId = table.currentSnapshot().snapshotId(); + + // Create branch + table + .manageSnapshots() + .createBranch("testBranch", currentSnapshotId) + .setMaxRefAgeMs("testBranch", 10) + .setMinSnapshotsToKeep("testBranch", 20) + .setMaxSnapshotAgeMs("testBranch", 30) + .commit(); + // Create Tag + table + .manageSnapshots() + .createTag("testTag", currentSnapshotId) + .setMaxRefAgeMs("testTag", 50) + .commit(); + // Check refs table + List references = spark.sql("SELECT * FROM " + tableName + ".refs").collectAsList(); + assertThat(references).as("Refs table should return 3 rows").hasSize(3); + List branches = + spark.sql("SELECT * FROM " + tableName + ".refs WHERE type='BRANCH'").collectAsList(); + assertThat(branches).as("Refs table should return 2 branches").hasSize(2); + List tags = + spark.sql("SELECT * FROM " + tableName + ".refs WHERE type='TAG'").collectAsList(); + assertThat(tags).as("Refs table should return 1 tag").hasSize(1); + + // Check branch entries in refs table + List mainBranch = + spark + .sql("SELECT * FROM " + tableName + ".refs WHERE name = 'main' AND type='BRANCH'") + .collectAsList(); + assertThat(mainBranch) + .hasSize(1) + .containsExactly(RowFactory.create("main", "BRANCH", currentSnapshotId, null, null, null)); + assertThat(mainBranch.get(0).schema().fieldNames()) + .containsExactly( + "name", + "type", + "snapshot_id", + "max_reference_age_in_ms", + "min_snapshots_to_keep", + "max_snapshot_age_in_ms"); + + List testBranch = + spark + .sql("SELECT * FROM " + tableName + ".refs WHERE name = 'testBranch' AND type='BRANCH'") + .collectAsList(); + assertThat(testBranch) + .hasSize(1) + .containsExactly( + RowFactory.create("testBranch", "BRANCH", currentSnapshotId, 10L, 20L, 30L)); + assertThat(testBranch.get(0).schema().fieldNames()) + .containsExactly( + "name", + "type", + "snapshot_id", + "max_reference_age_in_ms", + "min_snapshots_to_keep", + "max_snapshot_age_in_ms"); + + // Check tag entries in refs table + List testTag = + spark + .sql("SELECT * FROM " + tableName + ".refs WHERE name = 'testTag' AND type='TAG'") + .collectAsList(); + assertThat(testTag) + .hasSize(1) + .containsExactly(RowFactory.create("testTag", "TAG", currentSnapshotId, 50L, null, null)); + assertThat(testTag.get(0).schema().fieldNames()) + .containsExactly( + "name", + "type", + "snapshot_id", + "max_reference_age_in_ms", + "min_snapshots_to_keep", + "max_snapshot_age_in_ms"); + + // Check projection in refs table + List testTagProjection = + spark + .sql( + "SELECT name,type,snapshot_id,max_reference_age_in_ms,min_snapshots_to_keep FROM " + + tableName + + ".refs where type='TAG'") + .collectAsList(); + assertThat(testTagProjection) + .hasSize(1) + .containsExactly(RowFactory.create("testTag", "TAG", currentSnapshotId, 50L, null)); + assertThat(testTagProjection.get(0).schema().fieldNames()) + .containsExactly( + "name", "type", "snapshot_id", "max_reference_age_in_ms", "min_snapshots_to_keep"); + + List mainBranchProjection = + spark + .sql( + "SELECT name, type FROM " + + tableName + + ".refs WHERE name = 'main' AND type = 'BRANCH'") + .collectAsList(); + assertThat(mainBranchProjection) + .hasSize(1) + .containsExactly(RowFactory.create("main", "BRANCH")); + assertThat(mainBranchProjection.get(0).schema().fieldNames()).containsExactly("name", "type"); + + List testBranchProjection = + spark + .sql( + "SELECT name, type, snapshot_id, max_reference_age_in_ms FROM " + + tableName + + ".refs WHERE name = 'testBranch' AND type = 'BRANCH'") + .collectAsList(); + assertThat(testBranchProjection) + .hasSize(1) + .containsExactly(RowFactory.create("testBranch", "BRANCH", currentSnapshotId, 10L)); + assertThat(testBranchProjection.get(0).schema().fieldNames()) + .containsExactly("name", "type", "snapshot_id", "max_reference_age_in_ms"); + } + + /** + * Find matching manifest entries of an Iceberg table + * + * @param table iceberg table + * @param expectedContent file content to populate on entries + * @param entriesTableSchema schema of Manifest entries + * @param manifestsToExplore manifests to explore of the table + * @param partValue partition value that manifest entries must match, or null to skip filtering + */ + private List expectedEntries( + Table table, + FileContent expectedContent, + Schema entriesTableSchema, + List manifestsToExplore, + String partValue) + throws IOException { + List expected = Lists.newArrayList(); + for (ManifestFile manifest : manifestsToExplore) { + InputFile in = table.io().newInputFile(manifest.path()); + try (CloseableIterable rows = Avro.read(in).project(entriesTableSchema).build()) { + for (Record record : rows) { + if ((Integer) record.get("status") < 2 /* added or existing */) { + Record file = (Record) record.get("data_file"); + if (partitionMatch(file, partValue)) { + TestHelpers.asMetadataRecord(file, expectedContent); + expected.add(file); + } + } + } + } + } + return expected; + } + + private boolean partitionMatch(Record file, String partValue) { + if (partValue == null) { + return true; + } + Record partition = (Record) file.get(4); + return partValue.equals(partition.get(0).toString()); + } + + @TestTemplate + public void metadataLogEntriesAfterReplacingTable() throws Exception { + assumeThat(catalogConfig.get(ICEBERG_CATALOG_TYPE)) + .as( + "need to fix https://github.com/apache/iceberg/issues/11109 before enabling this for the REST catalog") + .isNotEqualTo(ICEBERG_CATALOG_TYPE_REST); + + sql( + "CREATE TABLE %s (id bigint, data string) " + + "USING iceberg " + + "PARTITIONED BY (data) " + + "TBLPROPERTIES " + + "('format-version'='%s')", + tableName, formatVersion); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + TableMetadata tableMetadata = ((HasTableOperations) table).operations().current(); + assertThat(tableMetadata.snapshots()).isEmpty(); + assertThat(tableMetadata.snapshotLog()).isEmpty(); + assertThat(tableMetadata.currentSnapshot()).isNull(); + + Object[] firstEntry = + row( + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), + tableMetadata.metadataFileLocation(), + null, + null, + null); + + assertThat(sql("SELECT * FROM %s.metadata_log_entries", tableName)).containsExactly(firstEntry); + + sql("INSERT INTO %s (id, data) VALUES (1, 'a')", tableName); + + tableMetadata = ((HasTableOperations) table).operations().refresh(); + assertThat(tableMetadata.snapshots()).hasSize(1); + assertThat(tableMetadata.snapshotLog()).hasSize(1); + Snapshot currentSnapshot = tableMetadata.currentSnapshot(); + + Object[] secondEntry = + row( + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), + tableMetadata.metadataFileLocation(), + currentSnapshot.snapshotId(), + currentSnapshot.schemaId(), + currentSnapshot.sequenceNumber()); + + assertThat(sql("SELECT * FROM %s.metadata_log_entries", tableName)) + .containsExactly(firstEntry, secondEntry); + + sql("INSERT INTO %s (id, data) VALUES (1, 'a')", tableName); + + tableMetadata = ((HasTableOperations) table).operations().refresh(); + assertThat(tableMetadata.snapshots()).hasSize(2); + assertThat(tableMetadata.snapshotLog()).hasSize(2); + currentSnapshot = tableMetadata.currentSnapshot(); + + Object[] thirdEntry = + row( + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), + tableMetadata.metadataFileLocation(), + currentSnapshot.snapshotId(), + currentSnapshot.schemaId(), + currentSnapshot.sequenceNumber()); + + assertThat(sql("SELECT * FROM %s.metadata_log_entries", tableName)) + .containsExactly(firstEntry, secondEntry, thirdEntry); + + sql( + "CREATE OR REPLACE TABLE %s (id bigint, data string) " + + "USING iceberg " + + "PARTITIONED BY (data) " + + "TBLPROPERTIES " + + "('format-version'='%s')", + tableName, formatVersion); + + tableMetadata = ((HasTableOperations) table).operations().refresh(); + assertThat(tableMetadata.snapshots()).hasSize(2); + assertThat(tableMetadata.snapshotLog()).hasSize(2); + + // currentSnapshot is null but the metadata_log_entries will refer to the last snapshot from the + // snapshotLog + assertThat(tableMetadata.currentSnapshot()).isNull(); + HistoryEntry historyEntry = tableMetadata.snapshotLog().get(1); + Snapshot lastSnapshot = tableMetadata.snapshot(historyEntry.snapshotId()); + + Object[] fourthEntry = + row( + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), + tableMetadata.metadataFileLocation(), + lastSnapshot.snapshotId(), + lastSnapshot.schemaId(), + lastSnapshot.sequenceNumber()); + + assertThat(sql("SELECT * FROM %s.metadata_log_entries", tableName)) + .containsExactly(firstEntry, secondEntry, thirdEntry, fourthEntry); + + sql("INSERT INTO %s (id, data) VALUES (1, 'a')", tableName); + + tableMetadata = ((HasTableOperations) table).operations().refresh(); + assertThat(tableMetadata.snapshots()).hasSize(3); + assertThat(tableMetadata.snapshotLog()).hasSize(3); + currentSnapshot = tableMetadata.currentSnapshot(); + + Object[] fifthEntry = + row( + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), + tableMetadata.metadataFileLocation(), + currentSnapshot.snapshotId(), + currentSnapshot.schemaId(), + currentSnapshot.sequenceNumber()); + + assertThat(sql("SELECT * FROM %s.metadata_log_entries", tableName)) + .containsExactly(firstEntry, secondEntry, thirdEntry, fourthEntry, fifthEntry); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java new file mode 100644 index 000000000000..9246671c0023 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java @@ -0,0 +1,291 @@ +/* + * 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.extensions; + +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.IOException; +import java.nio.file.Files; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.spark.sql.AnalysisException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMigrateTableProcedure extends ExtensionsTestBase { + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s_BACKUP_", tableName); + } + + @TestTemplate + public void testMigrate() throws IOException { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + + assertThat(result).as("Should have added one file").isEqualTo(1L); + + Table createdTable = validationCatalog.loadTable(tableIdent); + + String tableLocation = createdTable.location().replace("file:", ""); + assertThat(tableLocation).as("Table should have original location").isEqualTo(location); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + sql("DROP TABLE IF EXISTS %s", tableName + "_BACKUP_"); + } + + @TestTemplate + public void testMigrateWithOptions() throws IOException { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Object result = + scalarSql("CALL %s.system.migrate('%s', map('foo', 'bar'))", catalogName, tableName); + + assertThat(result).as("Should have added one file").isEqualTo(1L); + + Table createdTable = validationCatalog.loadTable(tableIdent); + + Map props = createdTable.properties(); + assertThat(props).containsEntry("foo", "bar"); + + String tableLocation = createdTable.location().replace("file:", ""); + assertThat(tableLocation).as("Table should have original location").isEqualTo(location); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + sql("DROP TABLE IF EXISTS %s", tableName + "_BACKUP_"); + } + + @TestTemplate + public void testMigrateWithDropBackup() throws IOException { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Object result = + scalarSql( + "CALL %s.system.migrate(table => '%s', drop_backup => true)", catalogName, tableName); + assertThat(result).as("Should have added one file").isEqualTo(1L); + assertThat(spark.catalog().tableExists(tableName + "_BACKUP_")).isFalse(); + } + + @TestTemplate + public void testMigrateWithBackupTableName() throws IOException { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + String backupTableName = "backup_table"; + Object result = + scalarSql( + "CALL %s.system.migrate(table => '%s', backup_table_name => '%s')", + catalogName, tableName, backupTableName); + + assertThat(result).isEqualTo(1L); + String dbName = tableName.split("\\.")[0]; + assertThat(spark.catalog().tableExists(dbName + "." + backupTableName)).isTrue(); + } + + @TestTemplate + public void testMigrateWithInvalidMetricsConfig() throws IOException { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + + assertThatThrownBy( + () -> { + String props = "map('write.metadata.metrics.column.x', 'X')"; + sql("CALL %s.system.migrate('%s', %s)", catalogName, tableName, props); + }) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Invalid metrics config"); + } + + @TestTemplate + public void testMigrateWithConflictingProps() throws IOException { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Object result = + scalarSql("CALL %s.system.migrate('%s', map('migrated', 'false'))", catalogName, tableName); + assertThat(result).as("Should have added one file").isEqualTo(1L); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s", tableName)); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.properties()).containsEntry("migrated", "true"); + } + + @TestTemplate + public void testInvalidMigrateCases() { + assertThatThrownBy(() -> sql("CALL %s.system.migrate()", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `migrate` because the parameter named `table` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.migrate(map('foo','bar'))", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "[DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE] Cannot resolve CALL due to data type mismatch: The first parameter requires the \"STRING\" type, however \"map(foo, bar)\" has the type \"MAP\". SQLSTATE: 42K09"); + + assertThatThrownBy(() -> sql("CALL %s.system.migrate('')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument table"); + } + + @TestTemplate + public void testMigratePartitionWithSpecialCharacter() throws IOException { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string, dt date) USING parquet " + + "PARTITIONED BY (data, dt) LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, '2023/05/30', date '2023-05-30')", tableName); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "2023/05/30", java.sql.Date.valueOf("2023-05-30"))), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testMigrateEmptyPartitionedTable() throws Exception { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'", + tableName, location); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + assertThat(result).isEqualTo(0L); + } + + @TestTemplate + public void testMigrateEmptyTable() throws Exception { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); + assertThat(result).isEqualTo(0L); + } + + @TestTemplate + public void testMigrateWithParallelism() throws IOException { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + List result = + sql("CALL %s.system.migrate(table => '%s', parallelism => %d)", catalogName, tableName, 2); + assertEquals("Procedure output must match", ImmutableList.of(row(2L)), result); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testMigrateWithInvalidParallelism() throws IOException { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.migrate(table => '%s', parallelism => %d)", + catalogName, tableName, -1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Parallelism should be larger than 0"); + } + + @TestTemplate + public void testMigratePartitionedWithParallelism() throws IOException { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s (id, data) VALUES (1, 'a'), (2, 'b')", tableName); + List result = + sql("CALL %s.system.migrate(table => '%s', parallelism => %d)", catalogName, tableName, 2); + assertEquals("Procedure output must match", ImmutableList.of(row(2L)), result); + assertEquals( + "Should have expected rows", + ImmutableList.of(row("a", 1L), row("b", 2L)), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java new file mode 100644 index 000000000000..4958fde15d55 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestPublishChangesProcedure.java @@ -0,0 +1,187 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestPublishChangesProcedure extends ExtensionsTestBase { + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testApplyWapChangesUsingPositionalArgs() { + String wapId = "wap_id_1"; + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); + + spark.conf().set("spark.wap.id", wapId); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should not see rows from staged snapshot", + ImmutableList.of(), + sql("SELECT * FROM %s", tableName)); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot wapSnapshot = Iterables.getOnlyElement(table.snapshots()); + + List output = + sql("CALL %s.system.publish_changes('%s', '%s')", catalogName, tableIdent, wapId); + + table.refresh(); + + Snapshot currentSnapshot = table.currentSnapshot(); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(wapSnapshot.snapshotId(), currentSnapshot.snapshotId())), + output); + + assertEquals( + "Apply of WAP changes must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s", tableName)); + } + + @TestTemplate + public void testApplyWapChangesUsingNamedArgs() { + String wapId = "wap_id_1"; + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); + + spark.conf().set("spark.wap.id", wapId); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should not see rows from staged snapshot", + ImmutableList.of(), + sql("SELECT * FROM %s", tableName)); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot wapSnapshot = Iterables.getOnlyElement(table.snapshots()); + + List output = + sql( + "CALL %s.system.publish_changes(wap_id => '%s', table => '%s')", + catalogName, wapId, tableIdent); + + table.refresh(); + + Snapshot currentSnapshot = table.currentSnapshot(); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(wapSnapshot.snapshotId(), currentSnapshot.snapshotId())), + output); + + assertEquals( + "Apply of WAP changes must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s", tableName)); + } + + @TestTemplate + public void testApplyWapChangesRefreshesRelationCache() { + String wapId = "wap_id_1"; + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); + + Dataset query = spark.sql("SELECT * FROM " + tableName + " WHERE id = 1"); + query.createOrReplaceTempView("tmp"); + + spark.sql("CACHE TABLE tmp"); + + assertEquals("View should not produce rows", ImmutableList.of(), sql("SELECT * FROM tmp")); + + spark.conf().set("spark.wap.id", wapId); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should not see rows from staged snapshot", + ImmutableList.of(), + sql("SELECT * FROM %s", tableName)); + + sql("CALL %s.system.publish_changes('%s', '%s')", catalogName, tableIdent, wapId); + + assertEquals( + "Apply of WAP changes should be visible", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM tmp")); + + sql("UNCACHE TABLE tmp"); + } + + @TestTemplate + public void testApplyInvalidWapId() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + assertThatThrownBy( + () -> sql("CALL %s.system.publish_changes('%s', 'not_valid')", catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot apply unknown WAP ID 'not_valid'"); + } + + @TestTemplate + public void testInvalidApplyWapChangesCases() { + assertThatThrownBy( + () -> + sql("CALL %s.system.publish_changes('n', table => 't', 'not_valid')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[UNEXPECTED_POSITIONAL_ARGUMENT] Cannot invoke routine `publish_changes` because it contains positional argument(s) following the named argument assigned to `table`; please rearrange them so the positional arguments come first and then retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy( + () -> sql("CALL %s.custom.publish_changes('n', 't', 'not_valid')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[FAILED_TO_LOAD_ROUTINE] Failed to load routine `%s`.`custom`.`publish_changes`. SQLSTATE: 38000", + catalogName); + + assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('t')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `publish_changes` because the parameter named `wap_id` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('', 'not_valid')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for parameter 'table'"); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java new file mode 100644 index 000000000000..a06a67b7d612 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java @@ -0,0 +1,85 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.atIndex; + +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableUtil; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.types.DataTypes; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRegisterTableProcedure extends ExtensionsTestBase { + + private String targetName; + + @BeforeEach + public void setTargetName() { + targetName = tableName("register_table"); + } + + @AfterEach + public void dropTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s", targetName); + } + + @TestTemplate + public void testRegisterTable() throws NoSuchTableException, ParseException { + long numRows = 1000; + + sql("CREATE TABLE %s (id int, data string) using ICEBERG", tableName); + spark + .range(0, numRows) + .withColumn("data", functions.col("id").cast(DataTypes.StringType)) + .writeTo(tableName) + .append(); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + long originalFileCount = (long) scalarSql("SELECT COUNT(*) from %s.files", tableName); + long currentSnapshotId = table.currentSnapshot().snapshotId(); + String metadataJson = TableUtil.metadataFileLocation(table); + + List result = + sql("CALL %s.system.register_table('%s', '%s')", catalogName, targetName, metadataJson); + assertThat(result.get(0)) + .as("Current Snapshot is not correct") + .contains(currentSnapshotId, atIndex(0)); + + List original = sql("SELECT * FROM %s", tableName); + List registered = sql("SELECT * FROM %s", targetName); + assertEquals("Registered table rows should match original table rows", original, registered); + assertThat(result.get(0)) + .as("Should have the right row count in the procedure result") + .contains(numRows, atIndex(1)) + .as("Should have the right datafile count in the procedure result") + .contains(originalFileCount, atIndex(2)); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java new file mode 100644 index 000000000000..14e6c358898c --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java @@ -0,0 +1,58 @@ +/* + * 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.extensions; + +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.RESTCatalogProperties; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.sql.TestSelect; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRemoteScanPlanning extends TestSelect { + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, binaryTableName = {3}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.REST.catalogName(), + SparkCatalogConfig.REST.implementation(), + ImmutableMap.builder() + .putAll(SparkCatalogConfig.REST.properties()) + .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) + // this flag is typically only set by the server, but we set it from the client for + // testing + .put(RESTCatalogProperties.REST_SCAN_PLANNING_ENABLED, "true") + .build(), + SparkCatalogConfig.REST.catalogName() + ".default.binary_table" + } + }; + } + + @TestTemplate + @Disabled( + "binary filter that is used by Spark is not working because ExpressionParser.fromJSON doesn't have the Schema to properly parse the filter expression") + public void testBinaryInFilter() { + super.testBinaryInFilter(); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java new file mode 100644 index 000000000000..a5ac8a7e01ac --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java @@ -0,0 +1,752 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.TableProperties.GC_ENABLED; +import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.sql.Timestamp; +import java.time.Instant; +import java.util.List; +import java.util.UUID; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.Files; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionStatisticsFile; +import org.apache.iceberg.ReachableFileUtil; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.spark.source.FilePathLastModifiedRecord; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRemoveOrphanFilesProcedure extends ExtensionsTestBase { + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s PURGE", tableName); + sql("DROP TABLE IF EXISTS p PURGE"); + } + + @TestTemplate + public void testRemoveOrphanFilesInEmptyTable() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + List output = + sql("CALL %s.system.remove_orphan_files('%s')", catalogName, tableIdent); + assertEquals("Should be no orphan files", ImmutableList.of(), output); + + assertEquals("Should have no rows", ImmutableList.of(), sql("SELECT * FROM %s", tableName)); + } + + @TestTemplate + public void testRemoveOrphanFilesInDataFolder() throws IOException { + if (catalogName.equals("testhadoop")) { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } else { + // give a fresh location to Hive tables as Spark will not clean up the table location + // correctly while dropping tables through spark_catalog + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); + } + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + String metadataLocation = table.location() + "/metadata"; + String dataLocation = table.location() + "/data"; + + // produce orphan files in the data location using parquet + sql("CREATE TABLE p (id bigint) USING parquet LOCATION '%s'", dataLocation); + sql("INSERT INTO TABLE p VALUES (1)"); + + // wait to ensure files are old enough + waitUntilAfter(System.currentTimeMillis()); + + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + + // check for orphans in the metadata folder + List output1 = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s'," + + "location => '%s')", + catalogName, tableIdent, currentTimestamp, metadataLocation); + assertEquals("Should be no orphan files in the metadata folder", ImmutableList.of(), output1); + + // check for orphans in the table location + List output2 = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s')", + catalogName, tableIdent, currentTimestamp); + assertThat(output2).as("Should be orphan files in the data folder").hasSize(1); + + // the previous call should have deleted all orphan files + List output3 = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s')", + catalogName, tableIdent, currentTimestamp); + assertThat(output3).as("Should be no more orphan files in the data folder").isEmpty(); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testRemoveOrphanFilesDryRun() throws IOException { + if (catalogName.equals("testhadoop")) { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } else { + // give a fresh location to Hive tables as Spark will not clean up the table location + // correctly while dropping tables through spark_catalog + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); + } + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + // produce orphan files in the table location using parquet + sql("CREATE TABLE p (id bigint) USING parquet LOCATION '%s'", table.location()); + sql("INSERT INTO TABLE p VALUES (1)"); + + // wait to ensure files are old enough + waitUntilAfter(System.currentTimeMillis()); + + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + + // check for orphans without deleting + List output1 = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s'," + + "dry_run => true)", + catalogName, tableIdent, currentTimestamp); + assertThat(output1).as("Should be one orphan files").hasSize(1); + + // actually delete orphans + List output2 = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s')", + catalogName, tableIdent, currentTimestamp); + assertThat(output2).as("Should be one orphan files").hasSize(1); + + // the previous call should have deleted all orphan files + List output3 = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s')", + catalogName, tableIdent, currentTimestamp); + assertThat(output3).as("Should be no more orphan files").isEmpty(); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testRemoveOrphanFilesGCDisabled() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED); + + assertThatThrownBy( + () -> sql("CALL %s.system.remove_orphan_files('%s')", catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Cannot delete orphan files: GC is disabled (deleting files may corrupt other tables)"); + + // reset the property to enable the table purging in removeTable. + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, GC_ENABLED); + } + + @TestTemplate + public void testRemoveOrphanFilesWap() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); + + spark.conf().set("spark.wap.id", "1"); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should not see rows from staged snapshot", + ImmutableList.of(), + sql("SELECT * FROM %s", tableName)); + + List output = + sql("CALL %s.system.remove_orphan_files('%s')", catalogName, tableIdent); + assertEquals("Should be no orphan files", ImmutableList.of(), output); + } + + @TestTemplate + public void testInvalidRemoveOrphanFilesCases() { + assertThatThrownBy( + () -> sql("CALL %s.system.remove_orphan_files('n', table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.BOTH_POSITIONAL_AND_NAMED] Call to routine `remove_orphan_files` is invalid because it includes multiple argument assignments to the same parameter name `table`. A positional argument and named argument both referred to the same parameter. Please remove the named argument referring to this parameter. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.custom.remove_orphan_files('n', 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[FAILED_TO_LOAD_ROUTINE] Failed to load routine `%s`.`custom`.`remove_orphan_files`. SQLSTATE: 38000", + catalogName); + + assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files()", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `remove_orphan_files` because the parameter named `table` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files('n', 2.2)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "[DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE] Cannot resolve CALL due to data type mismatch: The second parameter requires the \"TIMESTAMP\" type, however \"2.2\" has the type \"DECIMAL(2,1)\". SQLSTATE: 42K09"); + + assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files('')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for parameter 'table'"); + } + + @TestTemplate + public void testConcurrentRemoveOrphanFiles() throws IOException { + if (catalogName.equals("testhadoop")) { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } else { + // give a fresh location to Hive tables as Spark will not clean up the table location + // correctly while dropping tables through spark_catalog + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); + } + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + String dataLocation = table.location() + "/data"; + + // produce orphan files in the data location using parquet + sql("CREATE TABLE p (id bigint) USING parquet LOCATION '%s'", dataLocation); + sql("INSERT INTO TABLE p VALUES (1)"); + sql("INSERT INTO TABLE p VALUES (10)"); + sql("INSERT INTO TABLE p VALUES (100)"); + sql("INSERT INTO TABLE p VALUES (1000)"); + + // wait to ensure files are old enough + waitUntilAfter(System.currentTimeMillis()); + + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + + // check for orphans in the table location + List output = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "max_concurrent_deletes => %s," + + "older_than => TIMESTAMP '%s')", + catalogName, tableIdent, 4, currentTimestamp); + assertThat(output).as("Should be orphan files in the data folder").hasSize(4); + + // the previous call should have deleted all orphan files + List output3 = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "max_concurrent_deletes => %s," + + "older_than => TIMESTAMP '%s')", + catalogName, tableIdent, 4, currentTimestamp); + assertThat(output3).as("Should be no more orphan files in the data folder").isEmpty(); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testConcurrentRemoveOrphanFilesWithInvalidInput() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", + catalogName, tableIdent, 0)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("max_concurrent_deletes should have value > 0, value: 0"); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", + catalogName, tableIdent, -1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("max_concurrent_deletes should have value > 0, value: -1"); + + String tempViewName = "file_list_test"; + spark.emptyDataFrame().createOrReplaceTempView(tempViewName); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", + catalogName, tableIdent, tempViewName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("[FIELD_NOT_FOUND] No such struct field `file_path` in . SQLSTATE: 42704"); + + spark + .createDataset(Lists.newArrayList(), Encoders.tuple(Encoders.INT(), Encoders.TIMESTAMP())) + .toDF("file_path", "last_modified") + .createOrReplaceTempView(tempViewName); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", + catalogName, tableIdent, tempViewName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file_path column: IntegerType is not a string"); + + spark + .createDataset(Lists.newArrayList(), Encoders.tuple(Encoders.STRING(), Encoders.STRING())) + .toDF("file_path", "last_modified") + .createOrReplaceTempView(tempViewName); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", + catalogName, tableIdent, tempViewName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid last_modified column: StringType is not a timestamp"); + } + + @TestTemplate + public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { + sql( + "CREATE TABLE %s (id int, data string) USING iceberg TBLPROPERTIES" + + "('format-version'='2', 'write.delete.mode'='merge-on-read')", + tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + sql("DELETE FROM %s WHERE id=1", tableName); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + assertThat(TestHelpers.deleteManifests(table)).as("Should have 1 delete manifest").hasSize(1); + assertThat(TestHelpers.deleteFiles(table)).as("Should have 1 delete file").hasSize(1); + Path deleteManifestPath = new Path(TestHelpers.deleteManifests(table).iterator().next().path()); + Path deleteFilePath = new Path(TestHelpers.deleteFiles(table).iterator().next().location()); + + // wait to ensure files are old enough + waitUntilAfter(System.currentTimeMillis()); + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + + // delete orphans + List output = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s')", + catalogName, tableIdent, currentTimestamp); + assertThat(output).as("Should be no orphan files").isEmpty(); + + FileSystem localFs = FileSystem.getLocal(new Configuration()); + assertThat(localFs.exists(deleteManifestPath)) + .as("Delete manifest should still exist") + .isTrue(); + assertThat(localFs.exists(deleteFilePath)).as("Delete file should still exist").isTrue(); + + records.remove(new SimpleRecord(1, "a")); + Dataset resultDF = spark.read().format("iceberg").load(tableName); + List actualRecords = + resultDF.as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actualRecords).as("Rows must match").isEqualTo(records); + } + + @TestTemplate + public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { + sql( + "CREATE TABLE %s USING iceberg " + + "TBLPROPERTIES('format-version'='2') " + + "AS SELECT 10 int, 'abc' data", + tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + String statsFileName = "stats-file-" + UUID.randomUUID(); + String location = table.location(); + // not every catalog will return file proto for local directories + // i.e. Hadoop and Hive Catalog do, Jdbc and REST do not + if (!location.startsWith("file:")) { + location = "file:" + location; + } + File statsLocation = + new File(new URI(location)).toPath().resolve("data").resolve(statsFileName).toFile(); + StatisticsFile statisticsFile; + try (PuffinWriter puffinWriter = Puffin.write(Files.localOutput(statsLocation)).build()) { + long snapshotId = table.currentSnapshot().snapshotId(); + long snapshotSequenceNumber = table.currentSnapshot().sequenceNumber(); + puffinWriter.add( + new Blob( + "some-blob-type", + ImmutableList.of(1), + snapshotId, + snapshotSequenceNumber, + ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8)))); + puffinWriter.finish(); + statisticsFile = + new GenericStatisticsFile( + snapshotId, + statsLocation.toString(), + puffinWriter.fileSize(), + puffinWriter.footerSize(), + puffinWriter.writtenBlobsMetadata().stream() + .map(GenericBlobMetadata::from) + .collect(ImmutableList.toImmutableList())); + } + + Transaction transaction = table.newTransaction(); + transaction.updateStatistics().setStatistics(statisticsFile).commit(); + transaction.commitTransaction(); + + // wait to ensure files are old enough + waitUntilAfter(System.currentTimeMillis()); + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + + List output = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s')", + catalogName, tableIdent, currentTimestamp); + assertThat(output).as("Should be no orphan files").isEmpty(); + + assertThat(statsLocation).exists().hasSize(statisticsFile.fileSizeInBytes()); + + transaction = table.newTransaction(); + transaction.updateStatistics().removeStatistics(statisticsFile.snapshotId()).commit(); + transaction.commitTransaction(); + + output = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s')", + catalogName, tableIdent, currentTimestamp); + assertThat(output) + .hasSize(1) + .first() + .satisfies(files -> assertThat(files).containsExactly(statsLocation.toURI().toString())); + assertThat(statsLocation).doesNotExist(); + } + + @TestTemplate + public void testRemoveOrphanFilesWithPartitionStatisticFiles() throws Exception { + sql( + "CREATE TABLE %s USING iceberg " + + "TBLPROPERTIES('format-version'='2') " + + "AS SELECT 10 int, 'abc' data", + tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + String partitionStatsLocation = ProcedureUtil.statsFileLocation(table.location()); + PartitionStatisticsFile partitionStatisticsFile = + ProcedureUtil.writePartitionStatsFile( + table.currentSnapshot().snapshotId(), partitionStatsLocation, table.io()); + + commitPartitionStatsTxn(table, partitionStatisticsFile); + + // wait to ensure files are old enough + waitUntilAfter(System.currentTimeMillis()); + Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); + + List output = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s')", + catalogName, tableIdent, currentTimestamp); + assertThat(output).as("Should be no orphan files").isEmpty(); + + assertThat(new File(partitionStatsLocation)).as("partition stats file should exist").exists(); + + removePartitionStatsTxn(table, partitionStatisticsFile); + + output = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "older_than => TIMESTAMP '%s')", + catalogName, tableIdent, currentTimestamp); + assertThat(output) + .hasSize(1) + .first() + .satisfies(files -> assertThat(files).containsExactly("file:" + partitionStatsLocation)); + assertThat(new File(partitionStatsLocation)) + .as("partition stats file should be deleted") + .doesNotExist(); + } + + private static void removePartitionStatsTxn( + Table table, PartitionStatisticsFile partitionStatisticsFile) { + Transaction transaction = table.newTransaction(); + transaction + .updatePartitionStatistics() + .removePartitionStatistics(partitionStatisticsFile.snapshotId()) + .commit(); + transaction.commitTransaction(); + } + + private static void commitPartitionStatsTxn( + Table table, PartitionStatisticsFile partitionStatisticsFile) { + Transaction transaction = table.newTransaction(); + transaction + .updatePartitionStatistics() + .setPartitionStatistics(partitionStatisticsFile) + .commit(); + transaction.commitTransaction(); + } + + @TestTemplate + public void testRemoveOrphanFilesProcedureWithPrefixMode() + throws NoSuchTableException, ParseException, IOException { + if (catalogName.equals("testhadoop")) { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } else { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); + } + Table table = Spark3Util.loadIcebergTable(spark, tableName); + String location = table.location(); + Path originalPath = new Path(location); + + URI uri = originalPath.toUri(); + Path newParentPath = new Path("file1", uri.getAuthority(), uri.getPath()); + + DataFile dataFile1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(new Path(newParentPath, "path/to/data-a.parquet").toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + DataFile dataFile2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(new Path(newParentPath, "path/to/data-b.parquet").toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + table.newFastAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); + + Timestamp lastModifiedTimestamp = new Timestamp(10000); + + List allFiles = + Lists.newArrayList( + new FilePathLastModifiedRecord( + new Path(originalPath, "path/to/data-a.parquet").toString(), lastModifiedTimestamp), + new FilePathLastModifiedRecord( + new Path(originalPath, "path/to/data-b.parquet").toString(), lastModifiedTimestamp), + new FilePathLastModifiedRecord( + ReachableFileUtil.versionHintLocation(table), lastModifiedTimestamp)); + + for (String file : ReachableFileUtil.metadataFileLocations(table, true)) { + allFiles.add(new FilePathLastModifiedRecord(file, lastModifiedTimestamp)); + } + + for (ManifestFile manifest : TestHelpers.dataManifests(table)) { + allFiles.add(new FilePathLastModifiedRecord(manifest.path(), lastModifiedTimestamp)); + } + + Dataset compareToFileList = + spark + .createDataFrame(allFiles, FilePathLastModifiedRecord.class) + .withColumnRenamed("filePath", "file_path") + .withColumnRenamed("lastModified", "last_modified"); + String fileListViewName = "files_view"; + compareToFileList.createOrReplaceTempView(fileListViewName); + List orphanFiles = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "equal_schemes => map('file1', 'file')," + + "file_list_view => '%s')", + catalogName, tableIdent, fileListViewName); + assertThat(orphanFiles).isEmpty(); + + // Test with no equal schemes + assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "file_list_view => '%s')", + catalogName, tableIdent, fileListViewName)) + .isInstanceOf(ValidationException.class) + .hasMessageEndingWith("Conflicting authorities/schemes: [(file1, file)]."); + + // Drop table in afterEach has purge and fails due to invalid scheme "file1" used in this test + // Dropping the table here + sql("DROP TABLE %s", tableName); + } + + @TestTemplate + public void testRemoveOrphanFilesProcedureWithEqualAuthorities() + throws NoSuchTableException, ParseException, IOException { + if (catalogName.equals("testhadoop")) { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } else { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); + } + Table table = Spark3Util.loadIcebergTable(spark, tableName); + Path originalPath = new Path(table.location()); + + URI uri = originalPath.toUri(); + String originalAuthority = uri.getAuthority() == null ? "" : uri.getAuthority(); + Path newParentPath = new Path(uri.getScheme(), "localhost", uri.getPath()); + + DataFile dataFile1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(new Path(newParentPath, "path/to/data-a.parquet").toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + DataFile dataFile2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(new Path(newParentPath, "path/to/data-b.parquet").toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + table.newFastAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); + + Timestamp lastModifiedTimestamp = new Timestamp(10000); + + List allFiles = + Lists.newArrayList( + new FilePathLastModifiedRecord( + new Path(originalPath, "path/to/data-a.parquet").toString(), lastModifiedTimestamp), + new FilePathLastModifiedRecord( + new Path(originalPath, "path/to/data-b.parquet").toString(), lastModifiedTimestamp), + new FilePathLastModifiedRecord( + ReachableFileUtil.versionHintLocation(table), lastModifiedTimestamp)); + + for (String file : ReachableFileUtil.metadataFileLocations(table, true)) { + allFiles.add(new FilePathLastModifiedRecord(file, lastModifiedTimestamp)); + } + + for (ManifestFile manifest : TestHelpers.dataManifests(table)) { + allFiles.add(new FilePathLastModifiedRecord(manifest.path(), lastModifiedTimestamp)); + } + + Dataset compareToFileList = + spark + .createDataFrame(allFiles, FilePathLastModifiedRecord.class) + .withColumnRenamed("filePath", "file_path") + .withColumnRenamed("lastModified", "last_modified"); + String fileListViewName = "files_view"; + compareToFileList.createOrReplaceTempView(fileListViewName); + List orphanFiles = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "equal_authorities => map('localhost', '%s')," + + "file_list_view => '%s')", + catalogName, tableIdent, originalAuthority, fileListViewName); + assertThat(orphanFiles).isEmpty(); + + // Test with no equal authorities + assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "file_list_view => '%s')", + catalogName, tableIdent, fileListViewName)) + .isInstanceOf(ValidationException.class) + .hasMessageEndingWith("Conflicting authorities/schemes: [(localhost, null)]."); + + // Drop table in afterEach has purge and fails due to invalid authority "localhost" + // Dropping the table here + sql("DROP TABLE %s", tableName); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java new file mode 100644 index 000000000000..078df9d8aa8d --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java @@ -0,0 +1,270 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestReplaceBranch extends ExtensionsTestBase { + + private static final String[] TIME_UNITS = {"DAYS", "HOURS", "MINUTES"}; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties() + } + }; + } + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testReplaceBranchFailsForTag() throws NoSuchTableException { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + String tagName = "tag1"; + + List records = + ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.writeTo(tableName).append(); + Table table = validationCatalog.loadTable(tableIdent); + long first = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createTag(tagName, first).commit(); + df.writeTo(tableName).append(); + long second = table.currentSnapshot().snapshotId(); + + assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", + tableName, tagName, second)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref tag1 is a tag not a branch"); + } + + @TestTemplate + public void testReplaceBranch() throws NoSuchTableException { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + List records = + ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + long expectedMaxRefAgeMs = 1000; + int expectedMinSnapshotsToKeep = 2; + long expectedMaxSnapshotAgeMs = 1000; + table + .manageSnapshots() + .createBranch(branchName, first) + .setMaxRefAgeMs(branchName, expectedMaxRefAgeMs) + .setMinSnapshotsToKeep(branchName, expectedMinSnapshotsToKeep) + .setMaxSnapshotAgeMs(branchName, expectedMaxSnapshotAgeMs) + .commit(); + + df.writeTo(tableName).append(); + long second = table.currentSnapshot().snapshotId(); + + sql("ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", tableName, branchName, second); + + table.refresh(); + SnapshotRef ref = table.refs().get(branchName); + assertThat(ref).isNotNull(); + assertThat(ref.snapshotId()).isEqualTo(second); + assertThat(ref.minSnapshotsToKeep().intValue()).isEqualTo(expectedMinSnapshotsToKeep); + assertThat(ref.maxSnapshotAgeMs().longValue()).isEqualTo(expectedMaxSnapshotAgeMs); + assertThat(ref.maxRefAgeMs().longValue()).isEqualTo(expectedMaxRefAgeMs); + } + + @TestTemplate + public void testReplaceBranchDoesNotExist() throws NoSuchTableException { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + List records = + ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.writeTo(tableName).append(); + Table table = validationCatalog.loadTable(tableIdent); + + assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", + tableName, "someBranch", table.currentSnapshot().snapshotId())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Branch does not exist: someBranch"); + } + + @TestTemplate + public void testReplaceBranchWithRetain() throws NoSuchTableException { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + List records = + ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + table.manageSnapshots().createBranch(branchName, first).commit(); + df.writeTo(tableName).append(); + long second = table.currentSnapshot().snapshotId(); + + long maxRefAge = 10; + for (String timeUnit : TIME_UNITS) { + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d RETAIN %d %s", + tableName, branchName, second, maxRefAge, timeUnit); + + table.refresh(); + SnapshotRef ref = table.refs().get(branchName); + assertThat(ref).isNotNull(); + assertThat(ref.snapshotId()).isEqualTo(second); + assertThat(ref.minSnapshotsToKeep()).isNull(); + assertThat(ref.maxSnapshotAgeMs()).isNull(); + assertThat(ref.maxRefAgeMs().longValue()) + .isEqualTo(TimeUnit.valueOf(timeUnit).toMillis(maxRefAge)); + } + } + + @TestTemplate + public void testReplaceBranchWithSnapshotRetention() throws NoSuchTableException { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + List records = + ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.writeTo(tableName).append(); + String branchName = "b1"; + Table table = validationCatalog.loadTable(tableIdent); + long first = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createBranch(branchName, first).commit(); + df.writeTo(tableName).append(); + long second = table.currentSnapshot().snapshotId(); + + Integer minSnapshotsToKeep = 2; + long maxSnapshotAge = 2; + Long maxRefAgeMs = table.refs().get(branchName).maxRefAgeMs(); + for (String timeUnit : TIME_UNITS) { + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d WITH SNAPSHOT RETENTION %d SNAPSHOTS %d %s", + tableName, branchName, second, minSnapshotsToKeep, maxSnapshotAge, timeUnit); + + table.refresh(); + SnapshotRef ref = table.refs().get(branchName); + assertThat(ref).isNotNull(); + assertThat(ref.snapshotId()).isEqualTo(second); + assertThat(ref.minSnapshotsToKeep()).isEqualTo(minSnapshotsToKeep); + assertThat(ref.maxSnapshotAgeMs().longValue()) + .isEqualTo(TimeUnit.valueOf(timeUnit).toMillis(maxSnapshotAge)); + assertThat(ref.maxRefAgeMs()).isEqualTo(maxRefAgeMs); + } + } + + @TestTemplate + public void testReplaceBranchWithRetainAndSnapshotRetention() throws NoSuchTableException { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + List records = + ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + table.manageSnapshots().createBranch(branchName, first).commit(); + df.writeTo(tableName).append(); + long second = table.currentSnapshot().snapshotId(); + + Integer minSnapshotsToKeep = 2; + long maxSnapshotAge = 2; + long maxRefAge = 10; + for (String timeUnit : TIME_UNITS) { + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d RETAIN %d %s WITH SNAPSHOT RETENTION %d SNAPSHOTS %d %s", + tableName, + branchName, + second, + maxRefAge, + timeUnit, + minSnapshotsToKeep, + maxSnapshotAge, + timeUnit); + + table.refresh(); + SnapshotRef ref = table.refs().get(branchName); + assertThat(ref).isNotNull(); + assertThat(ref.snapshotId()).isEqualTo(second); + assertThat(ref.minSnapshotsToKeep()).isEqualTo(minSnapshotsToKeep); + assertThat(ref.maxSnapshotAgeMs().longValue()) + .isEqualTo(TimeUnit.valueOf(timeUnit).toMillis(maxSnapshotAge)); + assertThat(ref.maxRefAgeMs().longValue()) + .isEqualTo(TimeUnit.valueOf(timeUnit).toMillis(maxRefAge)); + } + } + + @TestTemplate + public void testCreateOrReplace() throws NoSuchTableException { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + List records = + ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + long first = table.currentSnapshot().snapshotId(); + String branchName = "b1"; + df.writeTo(tableName).append(); + long second = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createBranch(branchName, second).commit(); + + sql( + "ALTER TABLE %s CREATE OR REPLACE BRANCH %s AS OF VERSION %d", + tableName, branchName, first); + + table.refresh(); + SnapshotRef ref = table.refs().get(branchName); + assertThat(ref).isNotNull(); + assertThat(ref.snapshotId()).isEqualTo(first); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java new file mode 100644 index 000000000000..fe1c38482bed --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java @@ -0,0 +1,313 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.math.BigDecimal; +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.source.ThreeColumnRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRequiredDistributionAndOrdering extends ExtensionsTestBase { + + @AfterEach + public void dropTestTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testDefaultLocalSortWithBucketTransforms() throws NoSuchTableException { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, c1))", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(4, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(5, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(6, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(7, "BBBBBBBBBB", "A")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); + + // should insert a local sort by partition columns by default + inputDF.writeTo(tableName).append(); + + assertEquals( + "Row count must match", + ImmutableList.of(row(7L)), + sql("SELECT count(*) FROM %s", tableName)); + } + + @TestTemplate + public void testPartitionColumnsArePrependedForRangeDistribution() throws NoSuchTableException { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, c1))", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(4, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(5, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(6, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(7, "BBBBBBBBBB", "A")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); + + // should automatically prepend partition columns to the ordering + sql("ALTER TABLE %s WRITE ORDERED BY c1, c2", tableName); + + inputDF.writeTo(tableName).append(); + + assertEquals( + "Row count must match", + ImmutableList.of(row(7L)), + sql("SELECT count(*) FROM %s", tableName)); + } + + @TestTemplate + public void testSortOrderIncludesPartitionColumns() throws NoSuchTableException { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, c1))", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(4, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(5, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(6, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(7, "BBBBBBBBBB", "A")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); + + // should succeed with a correct sort order + sql("ALTER TABLE %s WRITE ORDERED BY bucket(2, c3), c1, c2", tableName); + + inputDF.writeTo(tableName).append(); + + assertEquals( + "Row count must match", + ImmutableList.of(row(7L)), + sql("SELECT count(*) FROM %s", tableName)); + } + + @TestTemplate + public void testHashDistributionOnBucketedColumn() throws NoSuchTableException { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, c1))", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(4, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(5, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(6, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(7, "BBBBBBBBBB", "A")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); + + // should automatically prepend partition columns to the local ordering after hash distribution + sql("ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION ORDERED BY c1, c2", tableName); + + inputDF.writeTo(tableName).append(); + + assertEquals( + "Row count must match", + ImmutableList.of(row(7L)), + sql("SELECT count(*) FROM %s", tableName)); + } + + @TestTemplate + public void testDisabledDistributionAndOrdering() { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, c1))", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(4, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(5, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(6, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(7, "BBBBBBBBBB", "A")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); + + // should fail if ordering is disabled + assertThatThrownBy( + () -> + inputDF + .writeTo(tableName) + .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") + .option(SparkWriteOptions.FANOUT_ENABLED, "false") + .append()) + .cause() + .isInstanceOf(IllegalStateException.class) + .hasMessageStartingWith( + "Incoming records violate the writer assumption that records are clustered by spec " + + "and by partition within each spec. Either cluster the incoming records or switch to fanout writers."); + } + + @TestTemplate + public void testDefaultSortOnDecimalBucketedColumn() { + sql( + "CREATE TABLE %s (c1 INT, c2 DECIMAL(20, 2)) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, c2))", + tableName); + + sql("INSERT INTO %s VALUES (1, 20.2), (2, 40.2), (3, 60.2)", tableName); + + List expected = + ImmutableList.of( + row(1, new BigDecimal("20.20")), + row(2, new BigDecimal("40.20")), + row(3, new BigDecimal("60.20"))); + + assertEquals("Rows must match", expected, sql("SELECT * FROM %s ORDER BY c1", tableName)); + } + + @TestTemplate + public void testDefaultSortOnStringBucketedColumn() { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, c2))", + tableName); + + sql("INSERT INTO %s VALUES (1, 'A'), (2, 'B')", tableName); + + List expected = ImmutableList.of(row(1, "A"), row(2, "B")); + + assertEquals("Rows must match", expected, sql("SELECT * FROM %s ORDER BY c1", tableName)); + } + + @TestTemplate + public void testDefaultSortOnBinaryBucketedColumn() { + sql( + "CREATE TABLE %s (c1 INT, c2 Binary) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, c2))", + tableName); + + sql("INSERT INTO %s VALUES (1, X'A1B1'), (2, X'A2B2')", tableName); + + byte[] bytes1 = new byte[] {-95, -79}; + byte[] bytes2 = new byte[] {-94, -78}; + List expected = ImmutableList.of(row(1, bytes1), row(2, bytes2)); + + assertEquals("Rows must match", expected, sql("SELECT * FROM %s ORDER BY c1", tableName)); + } + + @TestTemplate + public void testDefaultSortOnDecimalTruncatedColumn() { + sql( + "CREATE TABLE %s (c1 INT, c2 DECIMAL(20, 2)) " + + "USING iceberg " + + "PARTITIONED BY (truncate(2, c2))", + tableName); + + sql("INSERT INTO %s VALUES (1, 20.2), (2, 40.2)", tableName); + + List expected = + ImmutableList.of(row(1, new BigDecimal("20.20")), row(2, new BigDecimal("40.20"))); + + assertEquals("Rows must match", expected, sql("SELECT * FROM %s ORDER BY c1", tableName)); + } + + @TestTemplate + public void testDefaultSortOnLongTruncatedColumn() { + sql( + "CREATE TABLE %s (c1 INT, c2 BIGINT) " + + "USING iceberg " + + "PARTITIONED BY (truncate(2, c2))", + tableName); + + sql("INSERT INTO %s VALUES (1, 22222222222222), (2, 444444444444)", tableName); + + List expected = ImmutableList.of(row(1, 22222222222222L), row(2, 444444444444L)); + + assertEquals("Rows must match", expected, sql("SELECT * FROM %s ORDER BY c1", tableName)); + } + + @TestTemplate + public void testRangeDistributionWithQuotedColumnNames() throws NoSuchTableException { + sql( + "CREATE TABLE %s (`c.1` INT, c2 STRING, c3 STRING) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, `c.1`))", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(4, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(5, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(6, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(7, "BBBBBBBBBB", "A")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = + ds.selectExpr("c1 as `c.1`", "c2", "c3").coalesce(1).sortWithinPartitions("`c.1`"); + + sql("ALTER TABLE %s WRITE ORDERED BY `c.1`, c2", tableName); + + inputDF.writeTo(tableName).append(); + + assertEquals( + "Row count must match", + ImmutableList.of(row(7L)), + sql("SELECT count(*) FROM %s", tableName)); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java new file mode 100644 index 000000000000..3aabd635bb69 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -0,0 +1,1108 @@ +/* + * 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.extensions; + +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.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.LongStream; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.EnvironmentContext; +import org.apache.iceberg.Files; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionStatisticsFile; +import org.apache.iceberg.PartitionStats; +import org.apache.iceberg.PartitionStatsHandler; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.NamedReference; +import org.apache.iceberg.expressions.Zorder; +import org.apache.iceberg.io.CloseableIterable; +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.Lists; +import org.apache.iceberg.spark.ExtendedParser; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkTableCache; +import org.apache.iceberg.spark.SystemFunctionPushDownHelper; +import org.apache.iceberg.spark.source.ThreeColumnRecord; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.internal.SQLConf; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewriteDataFilesProcedure extends ExtensionsTestBase { + + private static final String QUOTED_SPECIAL_CHARS_TABLE_NAME = "`table:with.special:chars`"; + + @BeforeAll + public static void setupSpark() { + // disable AQE as tests assume that writes generate a particular number of files + spark.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), "false"); + } + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s", tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); + } + + @TestTemplate + public void testFilterCaseSensitivity() { + createTable(); + insertData(10); + sql("set %s = false", SQLConf.CASE_SENSITIVE().key()); + List expectedRecords = currentData(); + List output = + sql( + "CALL %s.system.rewrite_data_files(table=>'%s', where=>'C1 > 0')", + catalogName, tableIdent); + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + + @TestTemplate + public void testZOrderSortExpression() { + List order = + ExtendedParser.parseSortOrder(spark, "c1, zorder(c2, c3)"); + assertThat(order).as("Should parse 2 order fields").hasSize(2); + assertThat(((NamedReference) order.get(0).term()).name()) + .as("First field should be a ref") + .isEqualTo("c1"); + assertThat(order.get(1).term()).as("Second field should be zorder").isInstanceOf(Zorder.class); + } + + @TestTemplate + public void testRewriteDataFilesInEmptyTable() { + createTable(); + List output = sql("CALL %s.system.rewrite_data_files('%s')", catalogName, tableIdent); + assertEquals("Procedure output must match", ImmutableList.of(row(0, 0, 0L, 0, 0)), output); + } + + @TestTemplate + public void testRewriteDataFilesOnPartitionTable() { + createPartitionTable(); + // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') + insertData(10); + List expectedRecords = currentData(); + + List output = + sql("CALL %s.system.rewrite_data_files(table => '%s')", catalogName, tableIdent); + + assertEquals( + "Action should rewrite 10 data files and add 2 data files (one per partition) ", + row(10, 2), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + + @TestTemplate + public void testPartitionStatsIncrementalCompute() throws IOException { + createPartitionTable(); + // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') + insertData(10); + + Table table = validationCatalog.loadTable(tableIdent); + PartitionStatisticsFile statisticsFile = PartitionStatsHandler.computeAndWriteStatsFile(table); + table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); + + Schema dataSchema = PartitionStatsHandler.schema(Partitioning.partitionType(table), 2); + List statsBeforeCompaction; + try (CloseableIterable recordIterator = + PartitionStatsHandler.readPartitionStatsFile( + dataSchema, Files.localInput(statisticsFile.path()))) { + statsBeforeCompaction = Lists.newArrayList(recordIterator); + } + + sql("CALL %s.system.rewrite_data_files(table => '%s')", catalogName, tableIdent); + + table.refresh(); + statisticsFile = + PartitionStatsHandler.computeAndWriteStatsFile(table, table.currentSnapshot().snapshotId()); + table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); + List statsAfterCompaction; + try (CloseableIterable recordIterator = + PartitionStatsHandler.readPartitionStatsFile( + dataSchema, Files.localInput(statisticsFile.path()))) { + statsAfterCompaction = Lists.newArrayList(recordIterator); + } + + for (int index = 0; index < statsBeforeCompaction.size(); index++) { + PartitionStats statsAfter = statsAfterCompaction.get(index); + PartitionStats statsBefore = statsBeforeCompaction.get(index); + + assertThat(statsAfter.partition()).isEqualTo(statsBefore.partition()); + // data count should match after compaction + assertThat(statsAfter.dataRecordCount()).isEqualTo(statsBefore.dataRecordCount()); + // file count should not match as new file count will be one after compaction + assertThat(statsAfter.dataFileCount()).isNotEqualTo(statsBefore.dataFileCount()); + } + } + + @TestTemplate + public void testRewriteDataFilesOnNonPartitionTable() { + createTable(); + // create 10 files under non-partitioned table + insertData(10); + List expectedRecords = currentData(); + + List output = + sql("CALL %s.system.rewrite_data_files(table => '%s')", catalogName, tableIdent); + + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + + @TestTemplate + public void testRewriteDataFilesWithOptions() { + createTable(); + // create 10 files under non-partitioned table + insertData(10); + List expectedRecords = currentData(); + + // set the min-input-files = 12, instead of default 5 to skip compacting the files. + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s', options => map('min-input-files','12'))", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 0 data files and add 0 data files", + ImmutableList.of(row(0, 0, 0L, 0, 0)), + output); + + List actualRecords = currentData(); + assertEquals("Data should not change", expectedRecords, actualRecords); + } + + @TestTemplate + public void testRewriteDataFilesWithSortStrategy() { + createTable(); + // create 10 files under non-partitioned table + insertData(10); + List expectedRecords = currentData(); + + // set sort_order = c1 DESC LAST + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s', " + + "strategy => 'sort', sort_order => 'c1 DESC NULLS LAST')", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + + @TestTemplate + public void testRewriteDataFilesWithSortStrategyAndMultipleShufflePartitionsPerFile() { + createTable(); + insertData(10 /* file count */); + + List output = + sql( + "CALL %s.system.rewrite_data_files(" + + " table => '%s', " + + " strategy => 'sort', " + + " sort_order => 'c1', " + + " options => map('shuffle-partitions-per-file', '2'))", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + + // as there is only one small output file, validate the query ordering (it will not change) + ImmutableList expectedRows = + ImmutableList.of( + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null)); + assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); + } + + @TestTemplate + public void testRewriteDataFilesWithZOrder() { + createTable(); + // create 10 files under non-partitioned table + insertData(10); + + // set z_order = c1,c2 + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s', " + + "strategy => 'sort', sort_order => 'zorder(c1,c2)')", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + + // Due to Z_order, the data written will be in the below order. + // As there is only one small output file, we can validate the query ordering (as it will not + // change). + ImmutableList expectedRows = + ImmutableList.of( + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null)); + assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); + } + + @TestTemplate + public void testRewriteDataFilesWithZOrderNullBinaryColumn() { + sql("CREATE TABLE %s (c1 int, c2 string, c3 binary) USING iceberg", tableName); + + for (int i = 0; i < 5; i++) { + sql("INSERT INTO %s values (1, 'foo', null), (2, 'bar', null)", tableName); + } + + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s', " + + "strategy => 'sort', sort_order => 'zorder(c2,c3)')", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + assertThat(output.get(0)).hasSize(5); + assertThat(snapshotSummary()) + .containsEntry(SnapshotSummary.REMOVED_FILE_SIZE_PROP, String.valueOf(output.get(0)[2])); + assertThat(sql("SELECT * FROM %s", tableName)) + .containsExactly( + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null)); + } + + @TestTemplate + public void testRewriteDataFilesWithZOrderAndMultipleShufflePartitionsPerFile() { + createTable(); + insertData(10 /* file count */); + + List output = + sql( + "CALL %s.system.rewrite_data_files(" + + " table => '%s', " + + "strategy => 'sort', " + + " sort_order => 'zorder(c1, c2)', " + + " options => map('shuffle-partitions-per-file', '2'))", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + + // due to z-ordering, the data will be written in the below order + // as there is only one small output file, validate the query ordering (it will not change) + ImmutableList expectedRows = + ImmutableList.of( + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null)); + assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); + } + + @TestTemplate + public void testRewriteDataFilesWithFilter() { + createTable(); + // create 10 files under non-partitioned table + insertData(10); + List expectedRecords = currentData(); + + // select only 5 files for compaction (files that may have c1 = 1) + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + + " where => 'c1 = 1 and c2 is not null')", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 5 data files (containing c1 = 1) and add 1 data files", + row(5, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + + @TestTemplate + public void testRewriteDataFilesWithDeterministicTrueFilter() { + createTable(); + // create 10 files under non-partitioned table + insertData(10); + List expectedRecords = currentData(); + // select all 10 files for compaction + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s', where => '1=1')", + catalogName, tableIdent); + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + + @TestTemplate + public void testRewriteDataFilesWithDeterministicFalseFilter() { + createTable(); + // create 10 files under non-partitioned table + insertData(10); + List expectedRecords = currentData(); + // select no files for compaction + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s', where => '0=1')", + catalogName, tableIdent); + assertEquals( + "Action should rewrite 0 data files and add 0 data files", + row(0, 0), + Arrays.copyOf(output.get(0), 2)); + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + + @TestTemplate + public void testRewriteDataFilesWithFilterOnPartitionTable() { + createPartitionTable(); + // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') + insertData(10); + List expectedRecords = currentData(); + + // select only 5 files for compaction (files in the partition c2 = 'bar') + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c2 = \"bar\"')", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 5 data files from single matching partition" + + "(containing c2 = bar) and add 1 data files", + row(5, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + + @TestTemplate + public void testRewriteDataFilesWithFilterOnOnBucketExpression() { + // currently spark session catalog only resolve to v1 functions instead of desired v2 functions + // https://github.com/apache/spark/blob/branch-3.4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala#L2070-L2083 + assumeThat(catalogName).isNotEqualTo(SparkCatalogConfig.SPARK_SESSION.catalogName()); + createBucketPartitionTable(); + // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') + insertData(10); + List expectedRecords = currentData(); + + // select only 5 files for compaction (files in the partition c2 = 'bar') + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + + " where => '%s.system.bucket(2, c2) = 0')", + catalogName, tableIdent, catalogName); + + assertEquals( + "Action should rewrite 5 data files from single matching partition" + + "(containing bucket(c2) = 0) and add 1 data files", + row(5, 1), + row(output.get(0)[0], output.get(0)[1])); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + + @TestTemplate + public void testRewriteDataFilesWithInFilterOnPartitionTable() { + createPartitionTable(); + // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') + insertData(10); + List expectedRecords = currentData(); + + // select only 5 files for compaction (files in the partition c2 in ('bar')) + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c2 in (\"bar\")')", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 5 data files from single matching partition" + + "(containing c2 = bar) and add 1 data files", + row(5, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + + @TestTemplate + public void testRewriteDataFilesWithAllPossibleFilters() { + createPartitionTable(); + // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') + insertData(10); + + // Pass the literal value which is not present in the data files. + // So that parsing can be tested on a same dataset without actually compacting the files. + + // EqualTo + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c1 = 3')", + catalogName, tableIdent); + // GreaterThan + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c1 > 3')", + catalogName, tableIdent); + // GreaterThanOrEqual + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c1 >= 3')", + catalogName, tableIdent); + // LessThan + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c1 < 0')", + catalogName, tableIdent); + // LessThanOrEqual + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c1 <= 0')", + catalogName, tableIdent); + // In + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c1 in (3,4,5)')", + catalogName, tableIdent); + // IsNull + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c1 is null')", + catalogName, tableIdent); + // IsNotNull + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c3 is not null')", + catalogName, tableIdent); + // And + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c1 = 3 and c2 = \"bar\"')", + catalogName, tableIdent); + // Or + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c1 = 3 or c1 = 5')", + catalogName, tableIdent); + // Not + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c1 not in (1,2)')", + catalogName, tableIdent); + // StringStartsWith + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + " where => 'c2 like \"%s\"')", + catalogName, tableIdent, "car%"); + // TODO: Enable when org.apache.iceberg.spark.SparkFilters have implementations for + // StringEndsWith & StringContains + // StringEndsWith + // sql("CALL %s.system.rewrite_data_files(table => '%s'," + + // " where => 'c2 like \"%s\"')", catalogName, tableIdent, "%car"); + // StringContains + // sql("CALL %s.system.rewrite_data_files(table => '%s'," + + // " where => 'c2 like \"%s\"')", catalogName, tableIdent, "%car%"); + } + + @TestTemplate + public void testRewriteDataFilesWithPossibleV2Filters() { + // currently spark session catalog only resolve to v1 functions instead of desired v2 functions + // https://github.com/apache/spark/blob/branch-3.4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala#L2070-L2083 + assumeThat(catalogName).isNotEqualTo(SparkCatalogConfig.SPARK_SESSION.catalogName()); + + SystemFunctionPushDownHelper.createPartitionedTable(spark, tableName, "id"); + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + + " where => '%s.system.bucket(2, data) >= 0')", + catalogName, tableIdent, catalogName); + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + + " where => '%s.system.truncate(4, id) >= 1')", + catalogName, tableIdent, catalogName); + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + + " where => '%s.system.years(ts) >= 1')", + catalogName, tableIdent, catalogName); + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + + " where => '%s.system.months(ts) >= 1')", + catalogName, tableIdent, catalogName); + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + + " where => '%s.system.days(ts) >= date(\"2023-01-01\")')", + catalogName, tableIdent, catalogName); + sql( + "CALL %s.system.rewrite_data_files(table => '%s'," + + " where => '%s.system.hours(ts) >= 1')", + catalogName, tableIdent, catalogName); + } + + @TestTemplate + public void testRewriteDataFilesWithInvalidInputs() { + createTable(); + // create 2 files under non-partitioned table + insertData(2); + + // Test for invalid strategy + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', options => map('min-input-files','2'), " + + "strategy => 'temp')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("unsupported strategy: temp. Only binpack or sort is supported"); + + // Test for sort_order with binpack strategy + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'binpack', " + + "sort_order => 'c1 ASC NULLS FIRST')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot set rewrite mode, it has already been set to "); + + // Test for sort strategy without any (default/user defined) sort_order + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot sort data without a valid sort order"); + + // Test for sort_order with invalid null order + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'c1 ASC none')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unable to parse sortOrder: c1 ASC none"); + + // Test for sort_order with invalid sort direction + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'c1 none NULLS FIRST')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unable to parse sortOrder: c1 none NULLS FIRST"); + + // Test for sort_order with invalid column name + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'col1 DESC NULLS FIRST')", + catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Cannot find field 'col1' in struct:"); + + // Test with invalid filter column col1 + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', " + "where => 'col1 = 3')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse predicates in where option: col1 = 3"); + + // Test for z_order with invalid column name + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'zorder(col1)')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot find column 'col1' in table schema (case sensitive = false): " + + "struct<1: c1: optional int, 2: c2: optional string, 3: c3: optional string>"); + + // Test for z_order with sort_order + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'c1,zorder(c2,c3)')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot mix identity sort columns and a Zorder sort expression:" + " c1,zorder(c2,c3)"); + } + + @TestTemplate + public void testInvalidCasesForRewriteDataFiles() { + assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_data_files('n', table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.BOTH_POSITIONAL_AND_NAMED] Call to routine `rewrite_data_files` is invalid because it includes multiple argument assignments to the same parameter name `table`. A positional argument and named argument both referred to the same parameter. Please remove the named argument referring to this parameter. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.custom.rewrite_data_files('n', 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[FAILED_TO_LOAD_ROUTINE] Failed to load routine `%s`.`custom`.`rewrite_data_files`. SQLSTATE: 38000", + catalogName); + + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files()", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rewrite_data_files` because the parameter named `table` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_data_files(table => 't', table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageEndingWith( + "[DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE] Call to routine `rewrite_data_files` is invalid because it includes multiple argument assignments to the same parameter name `table`. More than one named argument referred to the same parameter. Please assign a value only once. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files('')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for parameter 'table'"); + } + + @TestTemplate + public void testBinPackTableWithSpecialChars() { + assumeThat(catalogName).isEqualTo(SparkCatalogConfig.HADOOP.catalogName()); + + TableIdentifier identifier = + TableIdentifier.of("default", QUOTED_SPECIAL_CHARS_TABLE_NAME.replaceAll("`", "")); + sql( + "CREATE TABLE %s (c1 int, c2 string, c3 string) USING iceberg", + tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); + + insertData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME), 10); + + List expectedRecords = currentData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); + + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s', where => 'c2 is not null')", + catalogName, tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); + + assertEquals( + "Action should rewrite 10 data files and add 1 data file", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isEqualTo( + Long.valueOf(snapshotSummary(identifier).get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + + List actualRecords = currentData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + + assertThat(SparkTableCache.get().size()).as("Table cache must be empty").isZero(); + } + + @TestTemplate + public void testSortTableWithSpecialChars() { + assumeThat(catalogName).isEqualTo(SparkCatalogConfig.HADOOP.catalogName()); + + TableIdentifier identifier = + TableIdentifier.of("default", QUOTED_SPECIAL_CHARS_TABLE_NAME.replaceAll("`", "")); + sql( + "CREATE TABLE %s (c1 int, c2 string, c3 string) USING iceberg", + tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); + + insertData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME), 10); + + List expectedRecords = currentData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); + + List output = + sql( + "CALL %s.system.rewrite_data_files(" + + " table => '%s'," + + " strategy => 'sort'," + + " sort_order => 'c1'," + + " where => 'c2 is not null')", + catalogName, tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); + + assertEquals( + "Action should rewrite 10 data files and add 1 data file", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo( + Long.valueOf(snapshotSummary(identifier).get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + + List actualRecords = currentData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + + assertThat(SparkTableCache.get().size()).as("Table cache must be empty").isZero(); + } + + @TestTemplate + public void testZOrderTableWithSpecialChars() { + assumeThat(catalogName).isEqualTo(SparkCatalogConfig.HADOOP.catalogName()); + + TableIdentifier identifier = + TableIdentifier.of("default", QUOTED_SPECIAL_CHARS_TABLE_NAME.replaceAll("`", "")); + sql( + "CREATE TABLE %s (c1 int, c2 string, c3 string) USING iceberg", + tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); + + insertData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME), 10); + + List expectedRecords = currentData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); + + List output = + sql( + "CALL %s.system.rewrite_data_files(" + + " table => '%s'," + + " strategy => 'sort'," + + " sort_order => 'zorder(c1, c2)'," + + " where => 'c2 is not null')", + catalogName, tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); + + assertEquals( + "Action should rewrite 10 data files and add 1 data file", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo( + Long.valueOf(snapshotSummary(identifier).get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + + List actualRecords = currentData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + + assertThat(SparkTableCache.get().size()).as("Table cache must be empty").isZero(); + } + + @TestTemplate + public void testDefaultSortOrder() { + createTable(); + // add a default sort order for a table + sql("ALTER TABLE %s WRITE ORDERED BY c2", tableName); + + // this creates 2 files under non-partitioned table due to sort order. + insertData(10); + List expectedRecords = currentData(); + + // When the strategy is set to 'sort' but the sort order is not specified, + // use table's default sort order. + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s', " + + "strategy => 'sort', " + + "options => map('min-input-files','2'))", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 2 data files and add 1 data files", + row(2, 1), + Arrays.copyOf(output.get(0), 2)); + // verify rewritten bytes separately + assertThat(output.get(0)).hasSize(5); + assertThat(output.get(0)[2]) + .isInstanceOf(Long.class) + .isEqualTo(Long.valueOf(snapshotSummary().get(SnapshotSummary.REMOVED_FILE_SIZE_PROP))); + + List actualRecords = currentData(); + assertEquals("Data after compaction should not change", expectedRecords, actualRecords); + } + + @TestTemplate + public void testRewriteWithUntranslatedOrUnconvertedFilter() { + createTable(); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', where => 'substr(encode(c2, \"utf-8\"), 2) = \"fo\"')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot translate Spark expression"); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', where => 'substr(c2, 2) = \"fo\"')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot convert Spark filter"); + } + + @TestTemplate + public void testRewriteDataFilesSummary() { + createTable(); + // create 10 files under non-partitioned table + insertData(10); + sql("CALL %s.system.rewrite_data_files(table => '%s')", catalogName, tableIdent); + + Map summary = snapshotSummary(); + assertThat(summary) + .containsKey(CatalogProperties.APP_ID) + .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") + .hasEntrySatisfying( + EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("4.0")); + } + + @TestTemplate + public void testRewriteDataFilesPreservesLineage() throws NoSuchTableException { + sql( + "CREATE TABLE %s (c1 int, c2 string, c3 string) USING iceberg TBLPROPERTIES('format-version' = '3')", + tableName); + List records = Lists.newArrayList(); + int numRecords = 10; + for (int i = 0; i < numRecords; i++) { + records.add(new ThreeColumnRecord(i, null, null)); + } + + spark + .createDataFrame(records, ThreeColumnRecord.class) + .repartition(10) + .writeTo(tableName) + .append(); + List expectedRowsWithLineage = + sql( + "SELECT c1, _row_id, _last_updated_sequence_number FROM %s ORDER BY _row_id", + tableName); + List rowIds = + expectedRowsWithLineage.stream() + .map(record -> (Long) record[1]) + .collect(Collectors.toList()); + List sequenceNumbers = + expectedRowsWithLineage.stream() + .map(record -> (Long) record[2]) + .collect(Collectors.toList()); + assertThat(rowIds) + .isEqualTo(LongStream.range(0, numRecords).boxed().collect(Collectors.toList())); + assertThat(sequenceNumbers).isEqualTo(Collections.nCopies(numRecords, 1L)); + + List output = + sql("CALL %s.system.rewrite_data_files(table => '%s')", catalogName, tableIdent); + + assertEquals( + "Action should rewrite 10 data files and add 1 data file", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + + List rowsWithLineageAfterRewrite = + sql( + "SELECT c1, _row_id, _last_updated_sequence_number FROM %s ORDER BY _row_id", + tableName); + assertEquals( + "Rows with lineage before rewrite should equal rows with lineage after rewrite", + expectedRowsWithLineage, + rowsWithLineageAfterRewrite); + } + + private void createTable() { + sql("CREATE TABLE %s (c1 int, c2 string, c3 string) USING iceberg", tableName); + } + + private void createPartitionTable() { + createPartitionTable( + ImmutableMap.of( + TableProperties.WRITE_DISTRIBUTION_MODE, TableProperties.WRITE_DISTRIBUTION_MODE_NONE)); + } + + private void createPartitionTable(Map properties) { + sql( + "CREATE TABLE %s (c1 int, c2 string, c3 string) " + + "USING iceberg " + + "PARTITIONED BY (c2)", + tableName); + properties.forEach( + (prop, value) -> + this.sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + new Object[] {this.tableName, prop, value})); + } + + private void createBucketPartitionTable() { + sql( + "CREATE TABLE %s (c1 int, c2 string, c3 string) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, c2)) " + + "TBLPROPERTIES ('%s' '%s')", + tableName, + TableProperties.WRITE_DISTRIBUTION_MODE, + TableProperties.WRITE_DISTRIBUTION_MODE_NONE); + } + + private void insertData(int filesCount) { + insertData(tableName, filesCount); + } + + private void insertData(String table, int filesCount) { + ThreeColumnRecord record1 = new ThreeColumnRecord(1, "foo", null); + ThreeColumnRecord record2 = new ThreeColumnRecord(2, "bar", null); + + List records = Lists.newArrayList(); + IntStream.range(0, filesCount / 2) + .forEach( + i -> { + records.add(record1); + records.add(record2); + }); + + Dataset df = + spark.createDataFrame(records, ThreeColumnRecord.class).repartition(filesCount); + try { + df.writeTo(table).append(); + } catch (org.apache.spark.sql.catalyst.analysis.NoSuchTableException e) { + throw new RuntimeException(e); + } + } + + private Map snapshotSummary() { + return snapshotSummary(tableIdent); + } + + private Map snapshotSummary(TableIdentifier tableIdentifier) { + return validationCatalog.loadTable(tableIdentifier).currentSnapshot().summary(); + } + + private List currentData() { + return currentData(tableName); + } + + private List currentData(String table) { + return rowsToJava(spark.sql("SELECT * FROM " + table + " order by c1, c2, c3").collectAsList()); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java new file mode 100644 index 000000000000..b8dca4b2cd18 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -0,0 +1,448 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.List; +import org.apache.iceberg.Files; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionStatisticsFile; +import org.apache.iceberg.PartitionStats; +import org.apache.iceberg.PartitionStatsHandler; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +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.Lists; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewriteManifestsProcedure extends ExtensionsTestBase { + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testRewriteManifestsInEmptyTable() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + assertEquals("Procedure output must match", ImmutableList.of(row(0, 0)), output); + } + + @TestTemplate + public void testRewriteLargeManifests() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); + + sql("ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest.target-size-bytes' '1')", tableName); + + List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + assertEquals("Procedure output must match", ImmutableList.of(row(1, 4)), output); + + table.refresh(); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 4 manifests") + .hasSize(4); + } + + @TestTemplate + public void testRewriteManifestsNoOp() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); + + List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + // should not rewrite any manifests for no-op (output of rewrite is same as before and after) + assertEquals("Procedure output must match", ImmutableList.of(row(0, 0)), output); + + table.refresh(); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); + } + + @TestTemplate + public void testRewriteLargeManifestsOnDatePartitionedTableWithJava8APIEnabled() { + withSQLConf( + ImmutableMap.of("spark.sql.datetime.java8API.enabled", "true"), + () -> { + sql( + "CREATE TABLE %s (id INTEGER, name STRING, dept STRING, ts DATE) USING iceberg PARTITIONED BY (ts)", + tableName); + try { + spark + .createDataFrame( + ImmutableList.of( + RowFactory.create(1, "John Doe", "hr", Date.valueOf("2021-01-01")), + RowFactory.create(2, "Jane Doe", "hr", Date.valueOf("2021-01-02")), + RowFactory.create(3, "Matt Doe", "hr", Date.valueOf("2021-01-03")), + RowFactory.create(4, "Will Doe", "facilities", Date.valueOf("2021-01-04"))), + spark.table(tableName).schema()) + .writeTo(tableName) + .append(); + } catch (NoSuchTableException e) { + // not possible as we already created the table above. + throw new RuntimeException(e); + } + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); + + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest.target-size-bytes' '1')", + tableName); + + List output = + sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + assertEquals("Procedure output must match", ImmutableList.of(row(1, 4)), output); + + table.refresh(); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 4 manifests") + .hasSize(4); + }); + } + + @TestTemplate + public void testRewriteLargeManifestsOnTimestampPartitionedTableWithJava8APIEnabled() { + withSQLConf( + ImmutableMap.of("spark.sql.datetime.java8API.enabled", "true"), + () -> { + sql( + "CREATE TABLE %s (id INTEGER, name STRING, dept STRING, ts TIMESTAMP) USING iceberg PARTITIONED BY (ts)", + tableName); + try { + spark + .createDataFrame( + ImmutableList.of( + RowFactory.create( + 1, "John Doe", "hr", Timestamp.valueOf("2021-01-01 00:00:00")), + RowFactory.create( + 2, "Jane Doe", "hr", Timestamp.valueOf("2021-01-02 00:00:00")), + RowFactory.create( + 3, "Matt Doe", "hr", Timestamp.valueOf("2021-01-03 00:00:00")), + RowFactory.create( + 4, "Will Doe", "facilities", Timestamp.valueOf("2021-01-04 00:00:00"))), + spark.table(tableName).schema()) + .writeTo(tableName) + .append(); + } catch (NoSuchTableException e) { + // not possible as we already created the table above. + throw new RuntimeException(e); + } + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); + + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest.target-size-bytes' '1')", + tableName); + + List output = + sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + assertEquals("Procedure output must match", ImmutableList.of(row(1, 4)), output); + + table.refresh(); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 4 manifests") + .hasSize(4); + }); + } + + @TestTemplate + public void testRewriteSmallManifestsWithSnapshotIdInheritance() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", + tableName, SNAPSHOT_ID_INHERITANCE_ENABLED, "true"); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + sql("INSERT INTO TABLE %s VALUES (3, 'c')", tableName); + sql("INSERT INTO TABLE %s VALUES (4, 'd')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 4 manifests") + .hasSize(4); + + List output = + sql("CALL %s.system.rewrite_manifests(table => '%s')", catalogName, tableIdent); + assertEquals("Procedure output must match", ImmutableList.of(row(4, 1)), output); + + table.refresh(); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); + } + + @TestTemplate + public void testRewriteSmallManifestsWithoutCaching() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 2 manifest") + .hasSize(2); + + List output = + sql( + "CALL %s.system.rewrite_manifests(use_caching => false, table => '%s')", + catalogName, tableIdent); + assertEquals("Procedure output must match", ImmutableList.of(row(2, 1)), output); + + table.refresh(); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); + } + + @Disabled("Spark SQL does not support case insensitive for named arguments") + public void testRewriteManifestsCaseInsensitiveArgs() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 2 manifests") + .hasSize(2); + + List output = + sql( + "CALL %s.system.rewrite_manifests(usE_cAcHiNg => false, tAbLe => '%s')", + catalogName, tableIdent); + assertEquals("Procedure output must match", ImmutableList.of(row(2, 1)), output); + + table.refresh(); + + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); + } + + @TestTemplate + public void testInvalidRewriteManifestsCases() { + assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_manifests('n', table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.BOTH_POSITIONAL_AND_NAMED] Call to routine `rewrite_manifests` is invalid because it includes multiple argument assignments to the same parameter name `table`. A positional argument and named argument both referred to the same parameter. Please remove the named argument referring to this parameter. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[FAILED_TO_LOAD_ROUTINE] Failed to load routine `%s`.`custom`.`rewrite_manifests`. SQLSTATE: 38000", + catalogName); + + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests()", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rewrite_manifests` because the parameter named `table` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('n', 2.2)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "[DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE] Cannot resolve CALL due to data type mismatch: The second parameter requires the \"BOOLEAN\" type, however \"2.2\" has the type \"DECIMAL(2,1)\". SQLSTATE: 42K09"); + + assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_manifests(table => 't', tAbLe => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[UNRECOGNIZED_PARAMETER_NAME] Cannot invoke routine `rewrite_manifests` because the routine call included a named argument reference for the argument named `tAbLe`, but this routine does not include any signature containing an argument with this name. Did you mean one of the following? [`table` `spec_id` `use_caching`]. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for parameter 'table'"); + } + + @TestTemplate + public void testReplacePartitionField() { + sql( + "CREATE TABLE %s (id int, ts timestamp, day_of_ts date) USING iceberg PARTITIONED BY (day_of_ts)", + tableName); + + sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version' = '2')", tableName); + sql("ALTER TABLE %s REPLACE PARTITION FIELD day_of_ts WITH days(ts)\n", tableName); + sql( + "INSERT INTO %s VALUES (1, CAST('2022-01-01 10:00:00' AS TIMESTAMP), CAST('2022-01-01' AS DATE))", + tableName); + sql( + "INSERT INTO %s VALUES (2, CAST('2022-01-01 11:00:00' AS TIMESTAMP), CAST('2022-01-01' AS DATE))", + tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01")), + row(2, Timestamp.valueOf("2022-01-01 11:00:00"), Date.valueOf("2022-01-01"))), + sql("SELECT * FROM %s WHERE ts < current_timestamp() order by 1 asc", tableName)); + + List output = + sql("CALL %s.system.rewrite_manifests(table => '%s')", catalogName, tableName); + assertEquals("Procedure output must match", ImmutableList.of(row(2, 1)), output); + + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(1, Timestamp.valueOf("2022-01-01 10:00:00"), Date.valueOf("2022-01-01")), + row(2, Timestamp.valueOf("2022-01-01 11:00:00"), Date.valueOf("2022-01-01"))), + sql("SELECT * FROM %s WHERE ts < current_timestamp() order by 1 asc", tableName)); + } + + @TestTemplate + public void testWriteManifestWithSpecId() { + sql( + "CREATE TABLE %s (id int, dt string, hr string) USING iceberg PARTITIONED BY (dt)", + tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest-merge.enabled' = 'false')", tableName); + + sql("INSERT INTO %s VALUES (1, '2024-01-01', '00')", tableName); + sql("INSERT INTO %s VALUES (2, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0", + ImmutableList.of(row(0), row(0)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + sql("ALTER TABLE %s ADD PARTITION FIELD hr", tableName); + sql("INSERT INTO %s VALUES (3, '2024-01-01', '00')", tableName); + assertEquals( + "Should have 3 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + + List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); + assertEquals("Nothing should be rewritten", ImmutableList.of(row(0, 0)), output); + + output = + sql( + "CALL %s.system.rewrite_manifests(table => '%s', spec_id => 0)", + catalogName, tableIdent); + assertEquals("There should be 2 manifests rewriten", ImmutableList.of(row(2, 1)), output); + assertEquals( + "Should have 2 manifests and their partition spec id should be 0 and 1", + ImmutableList.of(row(0), row(1)), + sql("SELECT partition_spec_id FROM %s.manifests order by 1 asc", tableName)); + } + + @TestTemplate + public void testPartitionStatsIncrementalCompute() throws IOException { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", + tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + PartitionStatisticsFile statisticsFile = PartitionStatsHandler.computeAndWriteStatsFile(table); + table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); + + Schema dataSchema = PartitionStatsHandler.schema(Partitioning.partitionType(table), 2); + List statsBeforeRewrite; + try (CloseableIterable recordIterator = + PartitionStatsHandler.readPartitionStatsFile( + dataSchema, Files.localInput(statisticsFile.path()))) { + statsBeforeRewrite = Lists.newArrayList(recordIterator); + } + + sql( + "CALL %s.system.rewrite_manifests(use_caching => false, table => '%s')", + catalogName, tableIdent); + + table.refresh(); + statisticsFile = + PartitionStatsHandler.computeAndWriteStatsFile(table, table.currentSnapshot().snapshotId()); + table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); + List statsAfterRewrite; + try (CloseableIterable recordIterator = + PartitionStatsHandler.readPartitionStatsFile( + dataSchema, Files.localInput(statisticsFile.path()))) { + statsAfterRewrite = Lists.newArrayList(recordIterator); + } + + for (int index = 0; index < statsBeforeRewrite.size(); index++) { + PartitionStats statsAfter = statsAfterRewrite.get(index); + PartitionStats statsBefore = statsBeforeRewrite.get(index); + + assertThat(statsAfter.partition()).isEqualTo(statsBefore.partition()); + // data count should match + assertThat(statsAfter.dataRecordCount()).isEqualTo(statsBefore.dataRecordCount()); + // file count should match + assertThat(statsAfter.dataFileCount()).isEqualTo(statsBefore.dataFileCount()); + } + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java new file mode 100644 index 000000000000..ad49da87bedd --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFiles.java @@ -0,0 +1,418 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; +import static org.apache.iceberg.data.FileHelpers.encrypt; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.Closeable; +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.time.LocalDateTime; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Files; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupRewriteResult; +import org.apache.iceberg.actions.RewritePositionDeleteFiles.Result; +import org.apache.iceberg.actions.SizeBasedFileRewritePlanner; +import org.apache.iceberg.data.GenericFileWriterFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestRewritePositionDeleteFiles extends ExtensionsTestBase { + + private static final Map CATALOG_PROPS = + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "cache-enabled", "false"); + + private static final String PARTITION_COL = "partition_col"; + private static final int NUM_DATA_FILES = 5; + private static final int ROWS_PER_DATA_FILE = 100; + private static final int DELETE_FILES_PER_PARTITION = 2; + private static final int DELETE_FILE_SIZE = 10; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, formatVersion = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + CATALOG_PROPS, + 2 + } + }; + } + + @Parameter(index = 3) + private int formatVersion; + + @AfterEach + public void cleanup() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testDatePartition() throws Exception { + createTable("date"); + Date baseDate = Date.valueOf("2023-01-01"); + insertData(i -> Date.valueOf(baseDate.toLocalDate().plusDays(i))); + testDanglingDelete(); + } + + @TestTemplate + public void testBooleanPartition() throws Exception { + createTable("boolean"); + insertData(i -> i % 2 == 0, 2); + testDanglingDelete(2); + } + + @TestTemplate + public void testTimestampPartition() throws Exception { + createTable("timestamp"); + Timestamp baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00"); + insertData(i -> Timestamp.valueOf(baseTimestamp.toLocalDateTime().plusDays(i))); + testDanglingDelete(); + } + + @TestTemplate + public void testTimestampNtz() throws Exception { + createTable("timestamp_ntz"); + LocalDateTime baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00").toLocalDateTime(); + insertData(baseTimestamp::plusDays); + testDanglingDelete(); + } + + @TestTemplate + public void testBytePartition() throws Exception { + createTable("byte"); + insertData(i -> i); + testDanglingDelete(); + } + + @TestTemplate + public void testDecimalPartition() throws Exception { + createTable("decimal(18, 10)"); + BigDecimal baseDecimal = new BigDecimal("1.0"); + insertData(i -> baseDecimal.add(new BigDecimal(i))); + testDanglingDelete(); + } + + @TestTemplate + public void testBinaryPartition() throws Exception { + createTable("binary"); + insertData(i -> java.nio.ByteBuffer.allocate(4).putInt(i).array()); + testDanglingDelete(); + } + + @TestTemplate + public void testCharPartition() throws Exception { + createTable("char(10)"); + insertData(Object::toString); + testDanglingDelete(); + } + + @TestTemplate + public void testVarcharPartition() throws Exception { + createTable("varchar(10)"); + insertData(Object::toString); + testDanglingDelete(); + } + + @TestTemplate + public void testIntPartition() throws Exception { + createTable("int"); + insertData(i -> i); + testDanglingDelete(); + } + + @TestTemplate + public void testDaysPartitionTransform() throws Exception { + createTable("timestamp", PARTITION_COL, String.format("days(%s)", PARTITION_COL)); + Timestamp baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00"); + insertData(i -> Timestamp.valueOf(baseTimestamp.toLocalDateTime().plusDays(i))); + testDanglingDelete(); + } + + @TestTemplate + public void testNullTransform() throws Exception { + createTable("int"); + insertData(i -> i == 0 ? null : 1, 2); + testDanglingDelete(2); + } + + @TestTemplate + public void testPartitionColWithDot() throws Exception { + String partitionColWithDot = "`partition.col`"; + createTable("int", partitionColWithDot, partitionColWithDot); + insertData(partitionColWithDot, i -> i, NUM_DATA_FILES); + testDanglingDelete(partitionColWithDot, NUM_DATA_FILES); + } + + private void testDanglingDelete() throws Exception { + testDanglingDelete(NUM_DATA_FILES); + } + + private void testDanglingDelete(int numDataFiles) throws Exception { + testDanglingDelete(PARTITION_COL, numDataFiles); + } + + private void testDanglingDelete(String partitionCol, int numDataFiles) throws Exception { + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + List dataFiles = dataFiles(table); + assertThat(dataFiles).hasSize(numDataFiles); + + SparkActions.get(spark) + .rewriteDataFiles(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + + // write dangling delete files for 'old data files' + writePosDeletesForFiles(table, dataFiles); + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(numDataFiles * DELETE_FILES_PER_PARTITION); + + List expectedRecords = records(tableName, partitionCol); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).as("Remaining dangling deletes").isEmpty(); + checkResult(result, deleteFiles, Lists.newArrayList(), numDataFiles); + + List actualRecords = records(tableName, partitionCol); + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + private void createTable(String partitionType) { + createTable(partitionType, PARTITION_COL, PARTITION_COL); + } + + private void createTable(String partitionType, String partitionCol, String partitionTransform) { + sql( + "CREATE TABLE %s (id long, %s %s, c1 string, c2 string) " + + "USING iceberg " + + "PARTITIONED BY (%s) " + + "TBLPROPERTIES('format-version'='%d')", + tableName, partitionCol, partitionType, partitionTransform, formatVersion); + } + + private void insertData(Function partitionValueFunction) throws Exception { + insertData(partitionValueFunction, NUM_DATA_FILES); + } + + private void insertData(Function partitionValueFunction, int numDataFiles) + throws Exception { + insertData(PARTITION_COL, partitionValueFunction, numDataFiles); + } + + private void insertData( + String partitionCol, Function partitionValue, int numDataFiles) throws Exception { + for (int i = 0; i < numDataFiles; i++) { + Dataset df = + spark + .range(0, ROWS_PER_DATA_FILE) + .withColumn(partitionCol, lit(partitionValue.apply(i))) + .withColumn("c1", expr("CAST(id AS STRING)")) + .withColumn("c2", expr("CAST(id AS STRING)")); + appendAsFile(df); + } + } + + private void appendAsFile(Dataset df) throws Exception { + // ensure the schema is precise + StructType sparkSchema = spark.table(tableName).schema(); + spark.createDataFrame(df.rdd(), sparkSchema).coalesce(1).writeTo(tableName).append(); + } + + private void writePosDeletesForFiles(Table table, List files) throws IOException { + + Map> filesByPartition = + files.stream().collect(Collectors.groupingBy(ContentFile::partition)); + List deleteFiles = + Lists.newArrayListWithCapacity(DELETE_FILES_PER_PARTITION * filesByPartition.size()); + + for (Map.Entry> filesByPartitionEntry : + filesByPartition.entrySet()) { + + StructLike partition = filesByPartitionEntry.getKey(); + List partitionFiles = filesByPartitionEntry.getValue(); + + int deletesForPartition = partitionFiles.size() * DELETE_FILE_SIZE; + assertThat(deletesForPartition % DELETE_FILE_SIZE) + .as("Number of delete files per partition modulo number of data files in this partition") + .isEqualTo(0); + int deleteFileSize = deletesForPartition / DELETE_FILES_PER_PARTITION; + + int counter = 0; + List> deletes = Lists.newArrayList(); + for (DataFile partitionFile : partitionFiles) { + for (int deletePos = 0; deletePos < DELETE_FILE_SIZE; deletePos++) { + deletes.add(Pair.of(partitionFile.location(), (long) deletePos)); + counter++; + if (counter == deleteFileSize) { + // Dump to file and reset variables + OutputFile output = + Files.localOutput(temp.resolve(UUID.randomUUID().toString()).toFile()); + deleteFiles.add(writeDeleteFile(table, output, partition, deletes)); + counter = 0; + deletes.clear(); + } + } + } + } + + RowDelta rowDelta = table.newRowDelta(); + deleteFiles.forEach(rowDelta::addDeletes); + rowDelta.commit(); + } + + private DeleteFile writeDeleteFile( + Table table, OutputFile out, StructLike partition, List> deletes) + throws IOException { + FileFormat format = defaultFormat(table.properties()); + + PositionDeleteWriter writer = + new GenericFileWriterFactory.Builder(table) + .deleteFileFormat(format) + .build() + .newPositionDeleteWriter(encrypt(out), table.spec(), partition); + PositionDelete posDelete = PositionDelete.create(); + try (Closeable toClose = writer) { + for (Pair delete : deletes) { + writer.write(posDelete.set(delete.first(), delete.second(), null)); + } + } + + return writer.toDeleteFile(); + } + + private static FileFormat defaultFormat(Map properties) { + String formatString = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); + return FileFormat.fromString(formatString); + } + + private List records(String table, String partitionCol) { + return rowsToJava( + spark.read().format("iceberg").load(table).sort(partitionCol, "id").collectAsList()); + } + + private long size(List deleteFiles) { + return deleteFiles.stream().mapToLong(DeleteFile::fileSizeInBytes).sum(); + } + + private List dataFiles(Table table) { + CloseableIterable tasks = table.newScan().includeColumnStats().planFiles(); + return Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + } + + private List deleteFiles(Table table) { + Table deletesTable = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); + CloseableIterable tasks = deletesTable.newBatchScan().planFiles(); + return Lists.newArrayList( + CloseableIterable.transform(tasks, t -> ((PositionDeletesScanTask) t).file())); + } + + private void checkResult( + Result result, + List rewrittenDeletes, + List newDeletes, + int expectedGroups) { + assertThat(result.rewrittenDeleteFilesCount()) + .as("Rewritten delete files") + .isEqualTo(rewrittenDeletes.size()); + assertThat(result.addedDeleteFilesCount()) + .as("Added delete files") + .isEqualTo(newDeletes.size()); + assertThat(result.rewrittenBytesCount()) + .as("Rewritten delete bytes") + .isEqualTo(size(rewrittenDeletes)); + assertThat(result.addedBytesCount()).as("New Delete byte count").isEqualTo(size(newDeletes)); + + assertThat(result.rewriteResults()).as("Rewritten group count").hasSize(expectedGroups); + assertThat( + result.rewriteResults().stream() + .mapToInt(FileGroupRewriteResult::rewrittenDeleteFilesCount) + .sum()) + .as("Rewritten delete file count in all groups") + .isEqualTo(rewrittenDeletes.size()); + assertThat( + result.rewriteResults().stream() + .mapToInt(FileGroupRewriteResult::addedDeleteFilesCount) + .sum()) + .as("Added delete file count in all groups") + .isEqualTo(newDeletes.size()); + assertThat( + result.rewriteResults().stream() + .mapToLong(FileGroupRewriteResult::rewrittenBytesCount) + .sum()) + .as("Rewritten delete bytes in all groups") + .isEqualTo(size(rewrittenDeletes)); + assertThat( + result.rewriteResults().stream() + .mapToLong(FileGroupRewriteResult::addedBytesCount) + .sum()) + .as("Added delete bytes in all groups") + .isEqualTo(size(newDeletes)); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java new file mode 100644 index 000000000000..4a6d2ff1979e --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java @@ -0,0 +1,250 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.SnapshotSummary.ADDED_FILE_SIZE_PROP; +import static org.apache.iceberg.SnapshotSummary.REMOVED_FILE_SIZE_PROP; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.EnvironmentContext; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.Encoders; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewritePositionDeleteFilesProcedure extends ExtensionsTestBase { + + private void createTable() throws Exception { + createTable(false); + } + + private void createTable(boolean partitioned) throws Exception { + String partitionStmt = partitioned ? "PARTITIONED BY (id)" : ""; + sql( + "CREATE TABLE %s (id bigint, data string) USING iceberg %s TBLPROPERTIES" + + "('format-version'='2', 'write.delete.mode'='merge-on-read', 'write.delete.granularity'='partition')", + tableName, partitionStmt); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(1, "b"), + new SimpleRecord(1, "c"), + new SimpleRecord(2, "d"), + new SimpleRecord(2, "e"), + new SimpleRecord(2, "f"), + new SimpleRecord(3, "g"), + new SimpleRecord(3, "h"), + new SimpleRecord(3, "i"), + new SimpleRecord(4, "j"), + new SimpleRecord(4, "k"), + new SimpleRecord(4, "l"), + new SimpleRecord(5, "m"), + new SimpleRecord(5, "n"), + new SimpleRecord(5, "o"), + new SimpleRecord(6, "p"), + new SimpleRecord(6, "q"), + new SimpleRecord(6, "r")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testExpireDeleteFilesAll() throws Exception { + createTable(); + + sql("DELETE FROM %s WHERE id=1", tableName); + sql("DELETE FROM %s WHERE id=2", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(TestHelpers.deleteFiles(table)).hasSize(2); + + List output = + sql( + "CALL %s.system.rewrite_position_delete_files(" + + "table => '%s'," + + "options => map(" + + "'rewrite-all','true'))", + catalogName, tableIdent); + table.refresh(); + + Map snapshotSummary = snapshotSummary(); + assertEquals( + "Should delete 2 delete files and add 1", + ImmutableList.of( + row( + 2, + 1, + Long.valueOf(snapshotSummary.get(REMOVED_FILE_SIZE_PROP)), + Long.valueOf(snapshotSummary.get(ADDED_FILE_SIZE_PROP)))), + output); + + assertThat(TestHelpers.deleteFiles(table)).hasSize(1); + } + + @TestTemplate + public void testExpireDeleteFilesNoOption() throws Exception { + createTable(); + + sql("DELETE FROM %s WHERE id=1", tableName); + sql("DELETE FROM %s WHERE id=2", tableName); + sql("DELETE FROM %s WHERE id=3", tableName); + sql("DELETE FROM %s WHERE id=4", tableName); + sql("DELETE FROM %s WHERE id=5", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(TestHelpers.deleteFiles(table)).hasSize(5); + + List output = + sql( + "CALL %s.system.rewrite_position_delete_files(" + "table => '%s')", + catalogName, tableIdent); + table.refresh(); + + Map snapshotSummary = snapshotSummary(); + assertEquals( + "Should replace 5 delete files with 1", + ImmutableList.of( + row( + 5, + 1, + Long.valueOf(snapshotSummary.get(REMOVED_FILE_SIZE_PROP)), + Long.valueOf(snapshotSummary.get(ADDED_FILE_SIZE_PROP)))), + output); + } + + @TestTemplate + public void testExpireDeleteFilesFilter() throws Exception { + createTable(true); + + sql("DELETE FROM %s WHERE id = 1 and data='a'", tableName); + sql("DELETE FROM %s WHERE id = 1 and data='b'", tableName); + sql("DELETE FROM %s WHERE id = 2 and data='d'", tableName); + sql("DELETE FROM %s WHERE id = 2 and data='e'", tableName); + sql("DELETE FROM %s WHERE id = 3 and data='g'", tableName); + sql("DELETE FROM %s WHERE id = 3 and data='h'", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(TestHelpers.deleteFiles(table)).hasSize(6); + + List output = + sql( + "CALL %s.system.rewrite_position_delete_files(" + + "table => '%s'," + // data filter is ignored as it cannot be applied to position deletes + + "where => 'id IN (1, 2) AND data=\"bar\"'," + + "options => map(" + + "'rewrite-all','true'))", + catalogName, tableIdent); + table.refresh(); + + Map snapshotSummary = snapshotSummary(); + assertEquals( + "Should delete 4 delete files and add 2", + ImmutableList.of( + row( + 4, + 2, + Long.valueOf(snapshotSummary.get(REMOVED_FILE_SIZE_PROP)), + Long.valueOf(snapshotSummary.get(ADDED_FILE_SIZE_PROP)))), + output); + + assertThat(TestHelpers.deleteFiles(table)).hasSize(4); + } + + @TestTemplate + public void testInvalidOption() throws Exception { + createTable(); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_position_delete_files(" + + "table => '%s'," + + "options => map(" + + "'foo', 'bar'))", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot use options [foo], they are not supported by the action or the rewriter BIN-PACK"); + } + + @TestTemplate + public void testRewriteWithUntranslatedOrUnconvertedFilter() throws Exception { + createTable(); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_position_delete_files(table => '%s', where => 'substr(encode(data, \"utf-8\"), 2) = \"fo\"')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot translate Spark expression"); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_position_delete_files(table => '%s', where => 'substr(data, 2) = \"fo\"')", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot convert Spark filter"); + } + + @TestTemplate + public void testRewriteSummary() throws Exception { + createTable(); + sql("DELETE FROM %s WHERE id=1", tableName); + + sql( + "CALL %s.system.rewrite_position_delete_files(" + + "table => '%s'," + + "options => map(" + + "'rewrite-all','true'))", + catalogName, tableIdent); + + Map summary = snapshotSummary(); + assertThat(summary) + .containsKey(CatalogProperties.APP_ID) + .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") + .hasEntrySatisfying( + EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("4.0")); + } + + private Map snapshotSummary() { + return validationCatalog.loadTable(tableIdent).currentSnapshot().summary(); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java new file mode 100644 index 000000000000..0c83f34219b1 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteTablePathProcedure.java @@ -0,0 +1,270 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.atIndex; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.RewriteTablePathUtil; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableUtil; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.AnalysisException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewriteTablePathProcedure extends ExtensionsTestBase { + @TempDir private Path staging; + @TempDir private Path targetTableDir; + + @BeforeEach + public void setupTableLocation() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testRewriteTablePathWithPositionalArgument() { + String location = targetTableDir.toFile().toURI().toString(); + Table table = validationCatalog.loadTable(tableIdent); + String metadataJson = TableUtil.metadataFileLocation(table); + + List result = + sql( + "CALL %s.system.rewrite_table_path('%s', '%s', '%s')", + catalogName, tableIdent, table.location(), location); + assertThat(result).hasSize(1); + assertThat(result.get(0)[0]) + .as("Should return correct latest version") + .isEqualTo(RewriteTablePathUtil.fileName(metadataJson)); + assertThat(result.get(0)[1]) + .as("Should return file_list_location") + .asString() + .startsWith(table.location()) + .endsWith("file-list"); + checkFileListLocationCount((String) result.get(0)[1], 1); + } + + @TestTemplate + public void testRewriteTablePathWithNamedArgument() { + Table table = validationCatalog.loadTable(tableIdent); + String v0Metadata = RewriteTablePathUtil.fileName(TableUtil.metadataFileLocation(table)); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + String v1Metadata = + RewriteTablePathUtil.fileName( + (((HasTableOperations) table).operations()).refresh().metadataFileLocation()); + + String targetLocation = targetTableDir.toFile().toURI().toString(); + String stagingLocation = staging.toFile().toURI().toString(); + String expectedFileListLocation = stagingLocation + "file-list"; + + List result = + sql( + "CALL %s.system.rewrite_table_path(" + + "table => '%s', " + + "target_prefix => '%s', " + + "source_prefix => '%s', " + + "end_version => '%s', " + + "start_version => '%s', " + + "staging_location => '%s')", + catalogName, + tableIdent, + targetLocation, + table.location(), + v1Metadata, + v0Metadata, + stagingLocation); + assertThat(result) + .singleElement() + .satisfies( + objects -> { + assertThat(objects).contains(v1Metadata, atIndex(0)); + assertThat(objects).contains(expectedFileListLocation, atIndex(1)); + }); + checkFileListLocationCount((String) result.get(0)[1], 4); + } + + @TestTemplate + public void testProcedureWithInvalidInput() { + String targetLocation = targetTableDir.toFile().toURI().toString(); + + assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_table_path('%s')", catalogName, tableIdent)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rewrite_table_path` because the parameter named `source_prefix` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_table_path('%s','%s')", + catalogName, tableIdent, targetLocation)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rewrite_table_path` because the parameter named `target_prefix` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_table_path('%s', '%s','%s')", + catalogName, "notExists", targetLocation, targetLocation)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Couldn't load table"); + + Table table = validationCatalog.loadTable(tableIdent); + String v0Metadata = RewriteTablePathUtil.fileName(TableUtil.metadataFileLocation(table)); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_table_path(" + + "table => '%s', " + + "source_prefix => '%s', " + + "target_prefix => '%s', " + + "start_version => '%s')", + catalogName, tableIdent, table.location(), targetLocation, "v20.metadata.json")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot find provided version file %s in metadata log.", "v20.metadata.json"); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_table_path(" + + "table => '%s', " + + "source_prefix => '%s', " + + "target_prefix => '%s', " + + "start_version => '%s'," + + "end_version => '%s')", + catalogName, + tableIdent, + table.location(), + targetLocation, + v0Metadata, + "v11.metadata.json")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot find provided version file %s in metadata log.", "v11.metadata.json"); + } + + @TestTemplate + public void testRewriteTablePathWithoutFileList() { + String location = targetTableDir.toFile().toURI().toString(); + Table table = validationCatalog.loadTable(tableIdent); + String metadataJson = TableUtil.metadataFileLocation(table); + + List result = + sql( + "CALL %s.system.rewrite_table_path(table => '%s', source_prefix => '%s', target_prefix => '%s', create_file_list => false)", + catalogName, tableIdent, table.location(), location); + assertThat(result).hasSize(1); + assertThat(result.get(0)[0]) + .as("Should return correct latest version") + .isEqualTo(RewriteTablePathUtil.fileName(metadataJson)); + assertThat(result.get(0)[1]) + .as("Check if file list location is correctly marked as N/A when not generated") + .asString() + .isEqualTo("N/A"); + } + + private void checkFileListLocationCount(String fileListLocation, long expectedFileCount) { + long fileCount = spark.read().format("text").load(fileListLocation).count(); + assertThat(fileCount).isEqualTo(expectedFileCount); + } + + @TestTemplate + public void testRewriteTablePathWithManifestAndDeleteCounts() throws IOException { + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + sql("INSERT INTO %s VALUES (3, 'c')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + List> rowsToDelete = + Lists.newArrayList( + Pair.of( + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().location(), + 0L)); + + File file = new File(removePrefix(table.location()) + "/data/deletes.parquet"); + String filePath = file.toURI().toString(); + if (SparkCatalogConfig.REST.catalogName().equals(catalogName)) { + // We applied this special handling because the base path for + // matching the RESTCATALOG's Hive BaseLocation is represented + // in the form of an AbsolutePath. + filePath = file.getAbsolutePath().toString(); + } + + DeleteFile positionDeletes = + FileHelpers.writeDeleteFile(table, table.io().newOutputFile(filePath), rowsToDelete) + .first(); + + table.newRowDelta().addDeletes(positionDeletes).commit(); + + sql("INSERT INTO %s VALUES (4, 'd')", tableName); + + String targetLocation = targetTableDir.toFile().toURI().toString(); + String stagingLocation = staging.toFile().toURI().toString(); + + List result = + sql( + "CALL %s.system.rewrite_table_path(" + + "table => '%s', " + + "source_prefix => '%s', " + + "target_prefix => '%s', " + + "staging_location => '%s', create_file_list => false)", + catalogName, tableIdent, table.location(), targetLocation, stagingLocation); + + assertThat(result).hasSize(1); + Object[] row = result.get(0); + + int rewrittenManifestFilesCount = ((Number) row[2]).intValue(); + int rewrittenDeleteFilesCount = ((Number) row[3]).intValue(); + + assertThat(rewrittenDeleteFilesCount) + .as( + "Expected exactly 1 delete file to be rewritten, but found " + + rewrittenDeleteFilesCount) + .isEqualTo(1); + + assertThat(rewrittenManifestFilesCount) + .as( + "Expected exactly 5 manifest files to be rewritten, but found " + + rewrittenManifestFilesCount) + .isEqualTo(5); + } + + private String removePrefix(String path) { + return path.substring(path.lastIndexOf(":") + 1); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java new file mode 100644 index 000000000000..cffc65f5dceb --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToSnapshotProcedure.java @@ -0,0 +1,286 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRollbackToSnapshotProcedure extends ExtensionsTestBase { + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testRollbackToSnapshotUsingPositionalArgs() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + List output = + sql( + "CALL %s.system.rollback_to_snapshot('%s', %dL)", + catalogName, tableIdent, firstSnapshot.snapshotId()); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "Rollback must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testRollbackToSnapshotUsingNamedArgs() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + List output = + sql( + "CALL %s.system.rollback_to_snapshot(snapshot_id => %dL, table => '%s')", + catalogName, firstSnapshot.snapshotId(), tableIdent); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "Rollback must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testRollbackToSnapshotRefreshesRelationCache() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + Dataset query = spark.sql("SELECT * FROM " + tableName + " WHERE id = 1"); + query.createOrReplaceTempView("tmp"); + + spark.sql("CACHE TABLE tmp"); + + assertEquals( + "View should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM tmp")); + + List output = + sql( + "CALL %s.system.rollback_to_snapshot(table => '%s', snapshot_id => %dL)", + catalogName, tableIdent, firstSnapshot.snapshotId()); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "View cache must be invalidated", ImmutableList.of(row(1L, "a")), sql("SELECT * FROM tmp")); + + sql("UNCACHE TABLE tmp"); + } + + @TestTemplate + public void testRollbackToSnapshotWithQuotedIdentifiers() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + StringBuilder quotedNamespaceBuilder = new StringBuilder(); + for (String level : tableIdent.namespace().levels()) { + quotedNamespaceBuilder.append("`"); + quotedNamespaceBuilder.append(level); + quotedNamespaceBuilder.append("`"); + } + String quotedNamespace = quotedNamespaceBuilder.toString(); + + List output = + sql( + "CALL %s.system.rollback_to_snapshot('%s', %d)", + catalogName, + quotedNamespace + ".`" + tableIdent.name() + "`", + firstSnapshot.snapshotId()); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "Rollback must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testRollbackToSnapshotWithoutExplicitCatalog() { + assumeThat(catalogName).as("Working only with the session catalog").isEqualTo("spark_catalog"); + + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + // use camel case intentionally to test case sensitivity + List output = + sql("CALL SyStEm.rOLlBaCk_to_SnApShOt('%s', %dL)", tableIdent, firstSnapshot.snapshotId()); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "Rollback must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testRollbackToInvalidSnapshot() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_snapshot('%s', -1L)", catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot roll back to unknown snapshot id: -1"); + } + + @TestTemplate + public void testInvalidRollbackToSnapshotCases() { + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_snapshot(namespace => 'n1', table => 't', 1L)", + catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[UNEXPECTED_POSITIONAL_ARGUMENT] Cannot invoke routine `rollback_to_snapshot` because it contains positional argument(s) following the named argument assigned to `table`; please rearrange them so the positional arguments come first and then retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.custom.rollback_to_snapshot('n', 't', 1L)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[FAILED_TO_LOAD_ROUTINE] Failed to load routine `%s`.`custom`.`rollback_to_snapshot`. SQLSTATE: 38000", + catalogName); + + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rollback_to_snapshot` because the parameter named `snapshot_id` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot(1L)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rollback_to_snapshot` because the parameter named `snapshot_id` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot(table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rollback_to_snapshot` because the parameter named `snapshot_id` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 1 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('t', '2.2')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "[CAST_INVALID_INPUT] The value '2.2' of the type \"STRING\" cannot be cast to \"BIGINT\" because it is malformed."); + + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('', 1L)", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for parameter 'table'"); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java new file mode 100644 index 000000000000..b554061d2c39 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRollbackToTimestampProcedure.java @@ -0,0 +1,339 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDateTime; +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRollbackToTimestampProcedure extends ExtensionsTestBase { + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testRollbackToTimestampUsingPositionalArgs() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + String firstSnapshotTimestamp = LocalDateTime.now().toString(); + + waitUntilAfter(firstSnapshot.timestampMillis()); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + List output = + sql( + "CALL %s.system.rollback_to_timestamp('%s',TIMESTAMP '%s')", + catalogName, tableIdent, firstSnapshotTimestamp); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "Rollback must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testRollbackToTimestampUsingNamedArgs() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + String firstSnapshotTimestamp = LocalDateTime.now().toString(); + + waitUntilAfter(firstSnapshot.timestampMillis()); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + List output = + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, firstSnapshotTimestamp, tableIdent); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "Rollback must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testRollbackToTimestampRefreshesRelationCache() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + String firstSnapshotTimestamp = LocalDateTime.now().toString(); + + waitUntilAfter(firstSnapshot.timestampMillis()); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + Dataset query = spark.sql("SELECT * FROM " + tableName + " WHERE id = 1"); + query.createOrReplaceTempView("tmp"); + + spark.sql("CACHE TABLE tmp"); + + assertEquals( + "View should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM tmp")); + + List output = + sql( + "CALL %s.system.rollback_to_timestamp(table => '%s', timestamp => TIMESTAMP '%s')", + catalogName, tableIdent, firstSnapshotTimestamp); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "View cache must be invalidated", ImmutableList.of(row(1L, "a")), sql("SELECT * FROM tmp")); + + sql("UNCACHE TABLE tmp"); + } + + @TestTemplate + public void testRollbackToTimestampWithQuotedIdentifiers() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + String firstSnapshotTimestamp = LocalDateTime.now().toString(); + + waitUntilAfter(firstSnapshot.timestampMillis()); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + StringBuilder quotedNamespaceBuilder = new StringBuilder(); + for (String level : tableIdent.namespace().levels()) { + quotedNamespaceBuilder.append("`"); + quotedNamespaceBuilder.append(level); + quotedNamespaceBuilder.append("`"); + } + String quotedNamespace = quotedNamespaceBuilder.toString(); + + List output = + sql( + "CALL %s.system.rollback_to_timestamp('%s', TIMESTAMP '%s')", + catalogName, quotedNamespace + ".`" + tableIdent.name() + "`", firstSnapshotTimestamp); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "Rollback must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testRollbackToTimestampWithoutExplicitCatalog() { + assumeThat(catalogName).as("Working only with the session catalog").isEqualTo("spark_catalog"); + + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + String firstSnapshotTimestamp = LocalDateTime.now().toString(); + + waitUntilAfter(firstSnapshot.timestampMillis()); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + // use camel case intentionally to test case sensitivity + List output = + sql( + "CALL SyStEm.rOLlBaCk_to_TiMeStaMp('%s', TIMESTAMP '%s')", + tableIdent, firstSnapshotTimestamp); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "Rollback must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testRollbackToTimestampBeforeOrEqualToOldestSnapshot() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + Timestamp beforeFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis() - 1)); + Timestamp exactFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis())); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, beforeFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + beforeFirstSnapshot.toInstant().toEpochMilli()); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, exactFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + exactFirstSnapshot.toInstant().toEpochMilli()); + } + + @TestTemplate + public void testInvalidRollbackToTimestampCases() { + String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(namespace => 'n1', 't', %s)", + catalogName, timestamp)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[UNEXPECTED_POSITIONAL_ARGUMENT] Cannot invoke routine `rollback_to_timestamp` because it contains positional argument(s) following the named argument assigned to `namespace`; please rearrange them so the positional arguments come first and then retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy( + () -> sql("CALL %s.custom.rollback_to_timestamp('n', 't', %s)", catalogName, timestamp)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[FAILED_TO_LOAD_ROUTINE] Failed to load routine `%s`.`custom`.`rollback_to_timestamp`. SQLSTATE: 38000", + catalogName); + + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rollback_to_timestamp` because the parameter named `timestamp` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => %s)", + catalogName, timestamp)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rollback_to_timestamp` because the parameter named `table` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp(table => 't')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `rollback_to_timestamp` because the parameter named `timestamp` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 1 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp('n', 't', %s, 1L)", + catalogName, timestamp)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "[WRONG_NUM_ARGS.WITHOUT_SUGGESTION] The `rollback_to_timestamp` requires 2 parameters but the actual number is 4."); + + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp('t', 2.2)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "[DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE] Cannot resolve CALL due to data type mismatch: The second parameter requires the \"TIMESTAMP\" type, however \"2.2\" has the type \"DECIMAL(2,1)\". SQLSTATE: 42K09"); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java new file mode 100644 index 000000000000..77303685235d --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRowLevelOperationsWithLineage.java @@ -0,0 +1,631 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.MetadataColumns.schemaWithRowLineage; +import static org.apache.iceberg.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE; +import static org.apache.iceberg.spark.Spark3Util.loadIcebergTable; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.data.GenericFileWriterFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.EncryptionUtil; +import org.apache.iceberg.io.DataWriter; +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.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.spark.functions.BucketFunction; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.PartitionMap; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; + +public abstract class TestRowLevelOperationsWithLineage extends SparkRowLevelOperationsTestBase { + static final Function BUCKET_PARTITION_GENERATOR = + record -> + TestHelpers.Row.of(BucketFunction.BucketInt.invoke(2, record.get(0, Integer.class))); + + static final Schema SCHEMA = + new Schema( + ImmutableList.of( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get()), + MetadataColumns.ROW_ID, + MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER)); + + static final List INITIAL_RECORDS = + ImmutableList.of( + createRecord(SCHEMA, 100, "a", 0L, 1L), + createRecord(SCHEMA, 101, "b", 1L, 1L), + createRecord(SCHEMA, 102, "c", 2L, 1L), + createRecord(SCHEMA, 103, "d", 3L, 1L), + createRecord(SCHEMA, 104, "e", 4L, 1L)); + + @Parameters( + name = + "catalogName = {0}, implementation = {1}, config = {2}," + + " format = {3}, vectorized = {4}, distributionMode = {5}," + + " fanout = {6}, branch = {7}, planningMode = {8}, formatVersion = {9}") + public static Object[][] parameters() { + return new Object[][] { + { + "testhadoop", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "hadoop"), + FileFormat.PARQUET, + true, + WRITE_DISTRIBUTION_MODE_HASH, + true, + null, + LOCAL, + 3 + }, + { + "testhadoop", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "hadoop"), + FileFormat.PARQUET, + false, + WRITE_DISTRIBUTION_MODE_RANGE, + true, + null, + DISTRIBUTED, + 3 + }, + { + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", + "hive", + "default-namespace", + "default", + "clients", + "1", + "parquet-enabled", + "false", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + ), + FileFormat.AVRO, + false, + WRITE_DISTRIBUTION_MODE_RANGE, + false, + null, + DISTRIBUTED, + 3 + }, + }; + } + + @BeforeAll + public static void setupSparkConf() { + spark.conf().set("spark.sql.shuffle.partitions", "4"); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS source"); + } + + @TestTemplate + public void testMergeIntoWithBothMatchedAndNonMatched() + throws NoSuchTableException, ParseException, IOException { + createAndInitTable("id INT, data STRING", null); + createBranchIfNeeded(); + Table table = loadIcebergTable(spark, tableName); + appendUnpartitionedRecords(table, INITIAL_RECORDS); + createOrReplaceView( + "source", + "id int, data string", + "{ \"id\": 101, \"data\": \"updated_b\" }\n " + "{ \"id\": 200, \"data\": \"f\" }\n"); + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET t.data = s.data " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + commitTarget()); + + Snapshot updateSnapshot = latestSnapshot(table); + long updateSnapshotFirstRowId = updateSnapshot.firstRowId(); + List allRows = rowsWithLineageAndFilePos(); + List carriedOverAndUpdatedRows = + allRows.stream() + .filter(row -> (long) row[3] < updateSnapshotFirstRowId) + .collect(Collectors.toList()); + + // Project sequence numbers first for easier comparison on the added row + assertEquals( + "Rows which are carried over or updated should have expected lineage", + ImmutableList.of( + row(1L, 100, "a", 0L, ANY, ANY), + row(updateSnapshot.sequenceNumber(), 101, "updated_b", 1L, ANY, ANY), + row(1L, 102, "c", 2L, ANY, ANY), + row(1L, 103, "d", 3L, ANY, ANY), + row(1L, 104, "e", 4L, ANY, ANY)), + carriedOverAndUpdatedRows); + + Object[] newRow = + Iterables.getOnlyElement( + allRows.stream() + .filter(row -> (long) row[3] >= updateSnapshotFirstRowId) + .collect(Collectors.toList())); + assertAddedRowLineage(row(updateSnapshot.sequenceNumber(), 200, "f"), newRow); + } + + @TestTemplate + public void testMergeIntoWithBothMatchedAndNonMatchedPartitioned() + throws NoSuchTableException, ParseException, IOException { + createAndInitTable("id INT, data STRING", "PARTITIONED BY (bucket(2, id))", null); + createBranchIfNeeded(); + Table table = loadIcebergTable(spark, tableName); + appendRecords( + table, partitionRecords(INITIAL_RECORDS, table.spec(), BUCKET_PARTITION_GENERATOR)); + createOrReplaceView( + "source", + "id int, data string", + "{ \"id\": 101, \"data\": \"updated_b\" }\n " + "{ \"id\": 200, \"data\": \"f\" }\n"); + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET t.data = s.data " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + commitTarget()); + + Snapshot updateSnapshot = latestSnapshot(table); + long updateSnapshotFirstRowId = updateSnapshot.firstRowId(); + List allRows = rowsWithLineageAndFilePos(); + + List carriedOverAndUpdatedRows = + allRows.stream() + .filter(row -> (long) row[3] < updateSnapshotFirstRowId) + .collect(Collectors.toList()); + + // Project sequence numbers first for easier comparison on the added row + assertEquals( + "Rows which are carried over or updated should have expected lineage", + ImmutableList.of( + row(1L, 100, "a", 0L, ANY, ANY), + row(updateSnapshot.sequenceNumber(), 101, "updated_b", 1L, ANY, ANY), + row(1L, 102, "c", 2L, ANY, ANY), + row(1L, 103, "d", 3L, ANY, ANY), + row(1L, 104, "e", 4L, ANY, ANY)), + carriedOverAndUpdatedRows); + + Object[] newRow = + Iterables.getOnlyElement( + allRows.stream() + .filter(row -> (long) row[3] >= updateSnapshotFirstRowId) + .collect(Collectors.toList())); + assertAddedRowLineage(row(updateSnapshot.sequenceNumber(), 200, "f"), newRow); + } + + @TestTemplate + public void testMergeIntoWithOnlyNonMatched() + throws NoSuchTableException, ParseException, IOException { + createAndInitTable("id INT, data string", null); + createBranchIfNeeded(); + Table table = loadIcebergTable(spark, tableName); + appendUnpartitionedRecords(table, INITIAL_RECORDS); + createOrReplaceView( + "source", + "id INT, data STRING", + "{ \"id\": 101, \"data\": \"updated_b\" }\n " + "{ \"id\": 200, \"data\": \"f\" }\n"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN NOT MATCHED THEN " + + "INSERT *", + commitTarget()); + + Snapshot updateSnapshot = latestSnapshot(table); + long updateSnapshotFirstRowId = updateSnapshot.firstRowId(); + + List allRows = rowsWithLineageAndFilePos(); + List carriedOverAndUpdatedRows = + allRows.stream() + .filter(row -> (long) row[3] < updateSnapshotFirstRowId) + .collect(Collectors.toList()); + + // Project sequence numbers first for easier comparison on the added row + assertEquals( + "Rows which are carried over or updated should have expected lineage", + ImmutableList.of( + row(1L, 100, "a", 0L, ANY, ANY), + row(1L, 101, "b", 1L, ANY, ANY), + row(1L, 102, "c", 2L, ANY, ANY), + row(1L, 103, "d", 3L, ANY, ANY), + row(1L, 104, "e", 4L, ANY, ANY)), + carriedOverAndUpdatedRows); + + Object[] newRow = + Iterables.getOnlyElement( + allRows.stream() + .filter(row -> (long) row[3] >= updateSnapshotFirstRowId) + .collect(Collectors.toList())); + assertAddedRowLineage(row(updateSnapshot.sequenceNumber(), 200, "f"), newRow); + } + + @TestTemplate + public void testMergeIntoWithOnlyMatched() + throws IOException, NoSuchTableException, ParseException { + createAndInitTable("id INT, data STRING", null); + createBranchIfNeeded(); + Table table = loadIcebergTable(spark, tableName); + appendUnpartitionedRecords(table, INITIAL_RECORDS); + createOrReplaceView( + "source", + "id INT, data string", + "{ \"id\": 101, \"data\": \"updated_b\" }\n " + + "{ \"id\": 102, \"data\": \"updated_c\" }\n"); + + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET t.data = s.data ", + commitTarget()); + + long updateSequenceNumber = latestSnapshot(table).sequenceNumber(); + assertEquals( + "Rows which are carried over or updated should have expected lineage", + ImmutableList.of( + row(100, "a", 0L, 1L), + row(101, "updated_b", 1L, updateSequenceNumber), + row(102, "updated_c", 2L, updateSequenceNumber), + row(103, "d", 3L, 1L), + row(104, "e", 4L, 1L)), + rowsWithLineage()); + } + + @TestTemplate + public void testMergeMatchedDelete() throws NoSuchTableException, ParseException, IOException { + createAndInitTable("id INT, data STRING", null); + createBranchIfNeeded(); + Table table = loadIcebergTable(spark, tableName); + appendUnpartitionedRecords(table, INITIAL_RECORDS); + createOrReplaceView( + "source", + "id INT, data string", + "{ \"id\": 101, \"data\": \"delete_101\" }\n " + + "{ \"id\": 102, \"data\": \"delete_102\" }\n"); + sql( + "MERGE INTO %s AS t USING source AS s " + "ON t.id == s.id " + "WHEN MATCHED THEN DELETE", + commitTarget()); + + assertEquals( + "Rows which are carried over or updated should have expected lineage", + ImmutableList.of(row(100, "a", 0L, 1L), row(103, "d", 3L, 1L), row(104, "e", 4L, 1L)), + rowsWithLineage()); + } + + @TestTemplate + public void testMergeWhenNotMatchedBySource() + throws NoSuchTableException, ParseException, IOException { + createAndInitTable("id INT, data STRING", null); + createBranchIfNeeded(); + Table table = loadIcebergTable(spark, tableName); + appendUnpartitionedRecords(table, INITIAL_RECORDS); + createOrReplaceView( + "source", + "id INT, data STRING", + "{ \"id\": 101, \"data\": \"updated_b\" }\n " + "{ \"id\": 200, \"data\": \"f\" }\n"); + + sql( + "MERGE INTO %s AS t USING source AS s ON t.id == s.id" + + " WHEN MATCHED THEN UPDATE set t.data = s.data " + + "WHEN NOT MATCHED BY SOURCE THEN UPDATE set data = 'not_matched_by_source'", + commitTarget()); + + long updateSequenceNumber = latestSnapshot(table).sequenceNumber(); + + assertEquals( + "Rows which are carried over or updated should have expected lineage", + ImmutableList.of( + row(100, "not_matched_by_source", 0L, updateSequenceNumber), + row(101, "updated_b", 1L, updateSequenceNumber), + row(102, "not_matched_by_source", 2L, updateSequenceNumber), + row(103, "not_matched_by_source", 3L, updateSequenceNumber), + row(104, "not_matched_by_source", 4L, updateSequenceNumber)), + rowsWithLineage()); + } + + @TestTemplate + public void testMergeWhenNotMatchedBySourceDelete() + throws NoSuchTableException, ParseException, IOException { + createAndInitTable("id INT, data STRING", null); + createBranchIfNeeded(); + Table table = loadIcebergTable(spark, tableName); + appendUnpartitionedRecords(table, INITIAL_RECORDS); + createOrReplaceView( + "source", + "id INT, data STRING", + "{ \"id\": 101, \"data\": \"updated_b\" }\n " + + "{ \"id\": 102, \"data\": \"updated_c\" }\n"); + + sql( + "MERGE INTO %s AS t USING source AS s ON t.id == s.id" + + " WHEN MATCHED THEN UPDATE set t.data = s.data " + + "WHEN NOT MATCHED BY SOURCE THEN DELETE", + commitTarget()); + + long updateSequenceNumber = latestSnapshot(table).sequenceNumber(); + assertEquals( + "Rows which are carried over or updated should have expected lineage", + ImmutableList.of( + row(101, "updated_b", 1L, updateSequenceNumber), + row(102, "updated_c", 2L, updateSequenceNumber)), + rowsWithLineage()); + } + + @TestTemplate + public void testUpdate() throws NoSuchTableException, ParseException, IOException { + createAndInitTable("id INT, data STRING", null); + createBranchIfNeeded(); + Table table = loadIcebergTable(spark, tableName); + appendUnpartitionedRecords(table, INITIAL_RECORDS); + + sql("UPDATE %s AS t set data = 'updated_b' WHERE id = 101", commitTarget()); + long updateSequenceNumber = latestSnapshot(table).sequenceNumber(); + + assertEquals( + "Rows which are carried over or updated should have expected lineage", + ImmutableList.of( + row(100, "a", 0L, 1L), + row(101, "updated_b", 1L, updateSequenceNumber), + row(102, "c", 2L, 1L), + row(103, "d", 3L, 1L), + row(104, "e", 4L, 1L)), + rowsWithLineage()); + } + + @TestTemplate + public void testDelete() throws NoSuchTableException, ParseException, IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + createAndInitTable("id int, data STRING", null); + createBranchIfNeeded(); + Table table = loadIcebergTable(spark, tableName); + appendUnpartitionedRecords(table, INITIAL_RECORDS); + + sql("DELETE FROM %s WHERE id = 101", commitTarget()); + + assertEquals( + "Rows which are carried over or updated should have expected lineage", + ImmutableList.of( + row(100, "a", 0L, 1L), + row(102, "c", 2L, 1L), + row(103, "d", 3L, 1L), + row(104, "e", 4L, 1L)), + rowsWithLineage()); + } + + @TestTemplate + public void testMergeWithManyRecords() throws NoSuchTableException, ParseException, IOException { + createAndInitTable("id INT, data STRING", null); + createBranchIfNeeded(); + Table table = loadIcebergTable(spark, tableName); + + int numRecords = 25000; + int startingId = 100; + + List initialRecords = Lists.newArrayList(); + int rowId = 0; + for (int id = 100; id < startingId + numRecords; id++) { + initialRecords.add(createRecord(SCHEMA, id, "data_" + id, rowId++, 1L)); + } + + appendUnpartitionedRecords(table, initialRecords); + createOrReplaceView( + "source", + "id int, data string", + "{ \"id\": 101, \"data\": \"updated_data_101\" }\n " + + "{ \"id\": 26000, \"data\": \"data_26000\" }\n"); + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET t.data = s.data " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + commitTarget()); + + Snapshot updateSnapshot = latestSnapshot(table); + long updateSnapshotFirstRowId = updateSnapshot.firstRowId(); + List allRows = rowsWithLineageAndFilePos(); + List carriedOverAndUpdatedRows = + allRows.stream() + .filter(row -> (long) row[3] < updateSnapshotFirstRowId) + .collect(Collectors.toList()); + + int newlyInsertedId = 26000; + int updatedId = 101; + List expectedCarriedOverAndUpdatedRows = + ImmutableList.builder() + .add(row(1L, 100, "data_100", 0L, ANY, ANY)) + .add(row(updateSnapshot.sequenceNumber(), updatedId, "updated_data_101", 1L, ANY, ANY)) + .addAll( + // Every record with higher ids than the updated excluding the new row should be a + // carry over + initialRecords.stream() + .filter( + initialRecord -> { + int id = initialRecord.get(0, Integer.class); + return id > updatedId && id != newlyInsertedId; + }) + .map(this::recordToExpectedRow) + .collect(Collectors.toList())) + .build(); + + assertEquals( + "Rows which are carried over or updated should have expected lineage", + expectedCarriedOverAndUpdatedRows, + carriedOverAndUpdatedRows); + + Object[] newRow = + Iterables.getOnlyElement( + allRows.stream() + .filter(row -> (long) row[3] >= updateSnapshotFirstRowId) + .collect(Collectors.toList())); + assertAddedRowLineage(row(updateSnapshot.sequenceNumber(), 26000, "data_26000"), newRow); + } + + private Object[] recordToExpectedRow(Record record) { + int id = record.get(0, Integer.class); + String data = record.get(1, String.class); + long rowId = record.get(2, Long.class); + long lastUpdated = record.get(3, Long.class); + return row(lastUpdated, id, data, rowId, ANY, ANY); + } + + private List rowsWithLineageAndFilePos() { + return sql( + "SELECT s._last_updated_sequence_number, s.id, s.data, s._row_id, files.first_row_id, s._pos FROM %s" + + " AS s JOIN %s.files AS files ON files.file_path = s._file ORDER BY s._row_id", + selectTarget(), selectTarget()); + } + + private List rowsWithLineage() { + return sql( + "SELECT id, data, _row_id, _last_updated_sequence_number FROM %s ORDER BY _row_id", + selectTarget()); + } + + /** + * Partitions the provided records based on the spec and partition function + * + * @return a partitioned map + */ + protected PartitionMap> partitionRecords( + List records, + PartitionSpec spec, + Function partitionGenerator) { + PartitionMap> recordsByPartition = + PartitionMap.create(Map.of(spec.specId(), spec)); + for (Record record : records) { + StructLike partition = partitionGenerator != null ? partitionGenerator.apply(record) : null; + List recordsForPartition = recordsByPartition.get(spec.specId(), partition); + if (recordsForPartition == null) { + recordsForPartition = Lists.newArrayList(); + } + + recordsForPartition.add(record); + recordsByPartition.put(spec.specId(), partition, recordsForPartition); + } + + return recordsByPartition; + } + + protected void appendUnpartitionedRecords(Table table, List records) throws IOException { + appendRecords(table, partitionRecords(records, table.spec(), record -> null)); + } + + protected void appendRecords(Table table, PartitionMap> partitionedRecords) + throws IOException { + AppendFiles append = table.newAppend(); + + for (Map.Entry, List> entry : partitionedRecords.entrySet()) { + OutputFile file = Files.localOutput(temp.resolve(UUID.randomUUID().toString()).toFile()); + DataWriter writer = + new GenericFileWriterFactory.Builder(table) + .dataSchema(schemaWithRowLineage(table.schema())) + .dataFileFormat(fileFormat) + .build() + .newDataWriter( + EncryptionUtil.plainAsEncryptedOutput(file), + table.spec(), + entry.getKey().second()); + List recordsForPartition = entry.getValue(); + writer.write(recordsForPartition); + writer.close(); + append = + append + .appendFile(writer.toDataFile()) + .toBranch(branch != null ? branch : SnapshotRef.MAIN_BRANCH); + } + + append.commit(); + } + + protected static Record createRecord( + Schema schema, int id, String data, long rowId, long lastUpdatedSequenceNumber) { + Record record = GenericRecord.create(schema); + record.set(0, id); + record.set(1, data); + record.set(2, rowId); + record.set(3, lastUpdatedSequenceNumber); + return record; + } + + private Snapshot latestSnapshot(Table table) { + table.refresh(); + return branch != null ? table.snapshot(branch) : table.currentSnapshot(); + } + + // Expected should have last updated sequence number followed by data columns + // Actual should have the contents of expected followed by the file first row ID and position + private void assertAddedRowLineage(Object[] expected, Object[] actual) { + // validate the sequence number and all the data columns + for (int pos = 0; pos < expected.length; pos++) { + assertThat(actual[pos]).isEqualTo(expected[pos]); + } + + int rowIdPos = expected.length; + int firstRowIdPos = rowIdPos + 1; + int positionPos = firstRowIdPos + 1; + long expectedRowId = (Long) actual[firstRowIdPos] + (Long) actual[positionPos]; + assertThat(actual[rowIdPos]).isEqualTo(expectedRowId); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java new file mode 100644 index 000000000000..ab0eca78d53b --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetCurrentSnapshotProcedure.java @@ -0,0 +1,291 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.spark.sql.AnalysisException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSetCurrentSnapshotProcedure extends ExtensionsTestBase { + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testSetCurrentSnapshotUsingPositionalArgs() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + List output = + sql( + "CALL %s.system.set_current_snapshot('%s', %dL)", + catalogName, tableIdent, firstSnapshot.snapshotId()); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "Set must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testSetCurrentSnapshotUsingNamedArgs() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + List output = + sql( + "CALL %s.system.set_current_snapshot(snapshot_id => %dL, table => '%s')", + catalogName, firstSnapshot.snapshotId(), tableIdent); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "Set must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testSetCurrentSnapshotWap() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); + + spark.conf().set("spark.wap.id", "1"); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should not see rows from staged snapshot", + ImmutableList.of(), + sql("SELECT * FROM %s", tableName)); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot wapSnapshot = Iterables.getOnlyElement(table.snapshots()); + + List output = + sql( + "CALL %s.system.set_current_snapshot(table => '%s', snapshot_id => %dL)", + catalogName, tableIdent, wapSnapshot.snapshotId()); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(null, wapSnapshot.snapshotId())), + output); + + assertEquals( + "Current snapshot must be set correctly", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s", tableName)); + } + + @TestTemplate + public void tesSetCurrentSnapshotWithoutExplicitCatalog() { + assumeThat(catalogName).as("Working only with the session catalog").isEqualTo("spark_catalog"); + + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + // use camel case intentionally to test case sensitivity + List output = + sql("CALL SyStEm.sEt_cuRrEnT_sNaPsHot('%s', %dL)", tableIdent, firstSnapshot.snapshotId()); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "Set must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testSetCurrentSnapshotToInvalidSnapshot() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + + assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot('%s', -1L)", catalogName, tableIdent)) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot roll back to unknown snapshot id: -1"); + } + + @TestTemplate + public void testInvalidRollbackToSnapshotCases() { + assertThatThrownBy( + () -> + sql( + "CALL %s.system.set_current_snapshot(namespace => 'n1', table => 't', 1L)", + catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[UNEXPECTED_POSITIONAL_ARGUMENT] Cannot invoke routine `set_current_snapshot` because it contains positional argument(s) following the named argument assigned to `table`; please rearrange them so the positional arguments come first and then retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.custom.set_current_snapshot('n', 't', 1L)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[FAILED_TO_LOAD_ROUTINE] Failed to load routine `%s`.`custom`.`set_current_snapshot`. SQLSTATE: 38000", + catalogName); + + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Either snapshot_id or ref must be provided, not both"); + + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot(1L)", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse identifier for parameter 'table': 1"); + + assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot(snapshot_id => 1L)", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `set_current_snapshot` because the parameter named `table` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot(table => 't')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Either snapshot_id or ref must be provided, not both"); + + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('t', '2.2')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "[CAST_INVALID_INPUT] The value '2.2' of the type \"STRING\" cannot be cast to \"BIGINT\" because it is malformed."); + + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('', 1L)", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for parameter 'table'"); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.set_current_snapshot(table => 't', snapshot_id => 1L, ref => 's1')", + catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Either snapshot_id or ref must be provided, not both"); + } + + @TestTemplate + public void testSetCurrentSnapshotToRef() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + String ref = "s1"; + sql("ALTER TABLE %s CREATE TAG %s", tableName, ref); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + table.refresh(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + List output = + sql( + "CALL %s.system.set_current_snapshot(table => '%s', ref => '%s')", + catalogName, tableIdent, ref); + + assertEquals( + "Procedure output must match", + ImmutableList.of(row(secondSnapshot.snapshotId(), firstSnapshot.snapshotId())), + output); + + assertEquals( + "Set must be successful", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + String notExistRef = "s2"; + assertThatThrownBy( + () -> + sql( + "CALL %s.system.set_current_snapshot(table => '%s', ref => '%s')", + catalogName, tableIdent, notExistRef)) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot find matching snapshot ID for ref " + notExistRef); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java new file mode 100644 index 000000000000..2bdeb197126a --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java @@ -0,0 +1,339 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.expressions.Expressions.bucket; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.spark.sql.internal.SQLConf; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSetWriteDistributionAndOrdering extends ExtensionsTestBase { + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testSetWriteOrderByColumn() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string, ts timestamp, data string) USING iceberg", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE ORDERED BY category, id", tableName); + + table.refresh(); + + String distributionMode = table.properties().get(TableProperties.WRITE_DISTRIBUTION_MODE); + assertThat(distributionMode).as("Distribution mode must match").isEqualTo("range"); + + SortOrder expected = + SortOrder.builderFor(table.schema()) + .withOrderId(1) + .asc("category", NullOrder.NULLS_FIRST) + .asc("id", NullOrder.NULLS_FIRST) + .build(); + assertThat(table.sortOrder()).as("Should have expected order").isEqualTo(expected); + } + + @TestTemplate + public void testSetWriteOrderWithCaseSensitiveColumnNames() { + sql( + "CREATE TABLE %s (Id bigint NOT NULL, Category string, ts timestamp, data string) USING iceberg", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + sql("SET %s=true", SQLConf.CASE_SENSITIVE().key()); + assertThatThrownBy( + () -> { + sql("ALTER TABLE %s WRITE ORDERED BY category, id", tableName); + }) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot find field 'category' in struct"); + + sql("SET %s=false", SQLConf.CASE_SENSITIVE().key()); + sql("ALTER TABLE %s WRITE ORDERED BY category, id", tableName); + table = validationCatalog.loadTable(tableIdent); + SortOrder expected = + SortOrder.builderFor(table.schema()).withOrderId(1).asc("Category").asc("Id").build(); + assertThat(table.sortOrder()).as("Should have expected order").isEqualTo(expected); + } + + @TestTemplate + public void testSetWriteOrderByColumnWithDirection() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string, ts timestamp, data string) USING iceberg", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE ORDERED BY category ASC, id DESC", tableName); + + table.refresh(); + + assertThat(table.properties()).containsEntry(TableProperties.WRITE_DISTRIBUTION_MODE, "range"); + + SortOrder expected = + SortOrder.builderFor(table.schema()) + .withOrderId(1) + .asc("category", NullOrder.NULLS_FIRST) + .desc("id", NullOrder.NULLS_LAST) + .build(); + assertThat(table.sortOrder()).as("Should have expected order").isEqualTo(expected); + } + + @TestTemplate + public void testSetWriteOrderByColumnWithDirectionAndNullOrder() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string, ts timestamp, data string) USING iceberg", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE ORDERED BY category ASC NULLS LAST, id DESC NULLS FIRST", tableName); + + table.refresh(); + + assertThat(table.properties()).containsEntry(TableProperties.WRITE_DISTRIBUTION_MODE, "range"); + + SortOrder expected = + SortOrder.builderFor(table.schema()) + .withOrderId(1) + .asc("category", NullOrder.NULLS_LAST) + .desc("id", NullOrder.NULLS_FIRST) + .build(); + assertThat(table.sortOrder()).as("Should have expected order").isEqualTo(expected); + } + + @TestTemplate + public void testSetWriteOrderByTransform() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string, ts timestamp, data string) USING iceberg", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).isTrue(); + + sql("ALTER TABLE %s WRITE ORDERED BY category DESC, bucket(16, id), id", tableName); + + table.refresh(); + + assertThat(table.properties()).containsEntry(TableProperties.WRITE_DISTRIBUTION_MODE, "range"); + + SortOrder expected = + SortOrder.builderFor(table.schema()) + .withOrderId(1) + .desc("category") + .asc(bucket("id", 16)) + .asc("id") + .build(); + assertThat(table.sortOrder()).as("Should have expected order").isEqualTo(expected); + } + + @TestTemplate + public void testSetWriteUnordered() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string, ts timestamp, data string) USING iceberg", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE ORDERED BY category DESC, bucket(16, id), id", tableName); + + table.refresh(); + + assertThat(table.properties()).containsEntry(TableProperties.WRITE_DISTRIBUTION_MODE, "range"); + + assertThat(table.sortOrder()).as("Table must be sorted").isNotEqualTo(SortOrder.unsorted()); + + sql("ALTER TABLE %s WRITE UNORDERED", tableName); + + table.refresh(); + + assertThat(table.properties()).containsEntry(TableProperties.WRITE_DISTRIBUTION_MODE, "none"); + + assertThat(table.sortOrder()).as("New sort order must match").isEqualTo(SortOrder.unsorted()); + } + + @TestTemplate + public void testSetWriteLocallyOrdered() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string, ts timestamp, data string) USING iceberg", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE LOCALLY ORDERED BY category DESC, bucket(16, id), id", tableName); + + table.refresh(); + + assertThat(table.properties()).doesNotContainKey(TableProperties.WRITE_DISTRIBUTION_MODE); + + SortOrder expected = + SortOrder.builderFor(table.schema()) + .withOrderId(1) + .desc("category") + .asc(bucket("id", 16)) + .asc("id") + .build(); + assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(expected); + } + + @TestTemplate + public void testSetWriteLocallyOrderedToPartitionedTable() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string) USING iceberg PARTITIONED BY (id)", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE LOCALLY ORDERED BY category DESC", tableName); + + table.refresh(); + + assertThat(table.properties()).doesNotContainKey(TableProperties.WRITE_DISTRIBUTION_MODE); + + SortOrder expected = + SortOrder.builderFor(table.schema()).withOrderId(1).desc("category").build(); + assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(expected); + } + + @TestTemplate + public void testSetWriteDistributedByWithSort() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string) USING iceberg PARTITIONED BY (category)", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION ORDERED BY id", tableName); + + table.refresh(); + + assertThat(table.properties()).containsEntry(TableProperties.WRITE_DISTRIBUTION_MODE, "hash"); + + SortOrder expected = SortOrder.builderFor(table.schema()).withOrderId(1).asc("id").build(); + assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(expected); + } + + @TestTemplate + public void testSetWriteDistributedByWithLocalSort() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string) USING iceberg PARTITIONED BY (category)", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION LOCALLY ORDERED BY id", tableName); + + table.refresh(); + + assertThat(table.properties()).containsEntry(TableProperties.WRITE_DISTRIBUTION_MODE, "hash"); + + SortOrder expected = SortOrder.builderFor(table.schema()).withOrderId(1).asc("id").build(); + assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(expected); + + sql("ALTER TABLE %s WRITE LOCALLY ORDERED BY id", tableName); + + table.refresh(); + + assertThat(table.properties()).containsEntry(TableProperties.WRITE_DISTRIBUTION_MODE, "hash"); + } + + @TestTemplate + public void testSetWriteDistributedByAndUnordered() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string) USING iceberg PARTITIONED BY (category)", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION UNORDERED", tableName); + + table.refresh(); + + assertThat(table.properties()).containsEntry(TableProperties.WRITE_DISTRIBUTION_MODE, "hash"); + + assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(SortOrder.unsorted()); + } + + @TestTemplate + public void testSetWriteDistributedByOnly() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string) USING iceberg PARTITIONED BY (category)", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION UNORDERED", tableName); + + table.refresh(); + + assertThat(table.properties()).containsEntry(TableProperties.WRITE_DISTRIBUTION_MODE, "hash"); + + assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(SortOrder.unsorted()); + } + + @TestTemplate + public void testSetWriteDistributedAndUnorderedInverted() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string) USING iceberg PARTITIONED BY (category)", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE UNORDERED DISTRIBUTED BY PARTITION", tableName); + + table.refresh(); + + assertThat(table.properties()).containsEntry(TableProperties.WRITE_DISTRIBUTION_MODE, "hash"); + + assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(SortOrder.unsorted()); + } + + @TestTemplate + public void testSetWriteDistributedAndLocallyOrderedInverted() { + sql( + "CREATE TABLE %s (id bigint NOT NULL, category string) USING iceberg PARTITIONED BY (category)", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); + + sql("ALTER TABLE %s WRITE ORDERED BY id DISTRIBUTED BY PARTITION", tableName); + + table.refresh(); + + assertThat(table.properties()).containsEntry(TableProperties.WRITE_DISTRIBUTION_MODE, "hash"); + + SortOrder expected = SortOrder.builderFor(table.schema()).withOrderId(1).asc("id").build(); + assertThat(table.sortOrder()).as("Sort order must match").isEqualTo(expected); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java new file mode 100644 index 000000000000..19800c2f4666 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -0,0 +1,378 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.entry; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.util.List; +import java.util.Map; +import org.apache.avro.generic.GenericData; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.avro.GenericAvroReader; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.AnalysisException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSnapshotTableProcedure extends ExtensionsTestBase { + private static final String SOURCE_NAME = "spark_catalog.default.source"; + + // Currently we can only Snapshot only out of the Spark Session Catalog + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s PURGE", SOURCE_NAME); + } + + @TestTemplate + public void testSnapshot() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + Object result = + scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, SOURCE_NAME, tableName); + + assertThat(result).as("Should have added one file").isEqualTo(1L); + + Table createdTable = validationCatalog.loadTable(tableIdent); + String tableLocation = createdTable.location(); + assertThat(tableLocation) + .as("Table should not have the original location") + .isNotEqualTo(location); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testSnapshotWithProperties() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + Object result = + scalarSql( + "CALL %s.system.snapshot(source_table => '%s', table => '%s', properties => map('foo','bar'))", + catalogName, SOURCE_NAME, tableName); + + assertThat(result).as("Should have added one file").isEqualTo(1L); + + Table createdTable = validationCatalog.loadTable(tableIdent); + + String tableLocation = createdTable.location(); + assertThat(tableLocation) + .as("Table should not have the original location") + .isNotEqualTo(location); + + Map props = createdTable.properties(); + assertThat(props).as("Should have extra property set").containsEntry("foo", "bar"); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testSnapshotWithAlternateLocation() throws IOException { + assumeThat(catalogName) + .as("No Snapshoting with Alternate locations with Hadoop Catalogs") + .doesNotContain("hadoop"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + String snapshotLocation = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + Object[] result = + sql( + "CALL %s.system.snapshot(source_table => '%s', table => '%s', location => '%s')", + catalogName, SOURCE_NAME, tableName, snapshotLocation) + .get(0); + + assertThat(result[0]).as("Should have added one file").isEqualTo(1L); + + String storageLocation = validationCatalog.loadTable(tableIdent).location(); + assertThat(storageLocation) + .as("Snapshot should be made at specified location") + .isEqualTo(snapshotLocation); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testDropTable() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + + Object result = + scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, SOURCE_NAME, tableName); + assertThat(result).as("Should have added one file").isEqualTo(1L); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s", tableName)); + + sql("DROP TABLE %s", tableName); + + assertEquals( + "Source table should be intact", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s", SOURCE_NAME)); + } + + @TestTemplate + public void testSnapshotWithConflictingProps() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + + Object result = + scalarSql( + "CALL %s.system.snapshot(" + + "source_table => '%s'," + + "table => '%s'," + + "properties => map('%s', 'true', 'snapshot', 'false'))", + catalogName, SOURCE_NAME, tableName, TableProperties.GC_ENABLED); + assertThat(result).as("Should have added one file").isEqualTo(1L); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a")), + sql("SELECT * FROM %s", tableName)); + + Table table = validationCatalog.loadTable(tableIdent); + Map props = table.properties(); + assertThat(props) + .contains(entry("snapshot", "true"), entry(TableProperties.GC_ENABLED, "false")); + } + + @TestTemplate + public void testInvalidSnapshotsCases() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, location); + + assertThatThrownBy(() -> sql("CALL %s.system.snapshot('foo')", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessage( + "[REQUIRED_PARAMETER_NOT_FOUND] Cannot invoke routine `snapshot` because the parameter named `table` is required, but the routine call did not supply a value. Please update the routine call to supply an argument value (either positionally at index 0 or by name) and retry the query again. SQLSTATE: 4274K"); + + assertThatThrownBy( + () -> sql("CALL %s.system.snapshot('n', 't', map('foo', 'bar'))", catalogName)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "[DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE] Cannot resolve CALL due to data type mismatch: The third parameter requires the \"STRING\" type, however \"map(foo, bar)\" has the type \"MAP\". SQLSTATE: 42K09"); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.snapshot('%s', 'fable', 'loc', map(2, 1, 1))", + catalogName, SOURCE_NAME)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "The `map` requires 2n (n > 0) parameters but the actual number is 3"); + + assertThatThrownBy(() -> sql("CALL %s.system.snapshot('', 'dest')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument source_table"); + + assertThatThrownBy(() -> sql("CALL %s.system.snapshot('src', '')", catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot handle an empty identifier for argument table"); + } + + @TestTemplate + public void testSnapshotWithParallelism() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + + List result = + sql( + "CALL %s.system.snapshot(source_table => '%s', table => '%s', parallelism => %d)", + catalogName, SOURCE_NAME, tableName, 2); + assertEquals("Procedure output must match", ImmutableList.of(row(2L)), result); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testSnapshotWithInvalidParallelism() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.snapshot(source_table => '%s', table => '%s', parallelism => %d)", + catalogName, SOURCE_NAME, tableName, -1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Parallelism should be larger than 0"); + } + + private static final Schema SNAPSHOT_ID_READ_SCHEMA = + new Schema( + Types.NestedField.required("snapshot_id") + .withId(1) + .ofType(Types.LongType.get()) + .asOptional() + .build()); + + @TestTemplate + public void testSnapshotPartitionedWithParallelism() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s (id, data) VALUES (1, 'a'), (2, 'b')", SOURCE_NAME); + List result = + sql( + "CALL %s.system.snapshot(source_table => '%s', table => '%s', parallelism => %d)", + catalogName, SOURCE_NAME, tableName, 2); + assertEquals("Procedure output must match", ImmutableList.of(row(2L)), result); + assertEquals( + "Should have expected rows", + ImmutableList.of(row("a", 1L), row("b", 2L)), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testSnapshotPartitioned() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s (id, data) VALUES (1, 'a'), (2, 'b')", SOURCE_NAME); + assertThat( + sql( + "CALL %s.system.snapshot(source_table => '%s', table => '%s')", + catalogName, SOURCE_NAME, tableName)) + .containsExactly(row(2L)); + assertThat(sql("SELECT * FROM %s ORDER BY id", tableName)) + .containsExactly(row("a", 1L), row("b", 2L)); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row("a", 1L), row("b", 2L)), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + Table createdTable = validationCatalog.loadTable(tableIdent); + + for (ManifestFile manifest : createdTable.currentSnapshot().dataManifests(createdTable.io())) { + try (AvroIterable reader = + Avro.read(org.apache.iceberg.Files.localInput(manifest.path())) + .project(SNAPSHOT_ID_READ_SCHEMA) + .createResolvingReader(GenericAvroReader::create) + .build()) { + + assertThat(reader.getMetadata()).containsEntry("format-version", "2"); + + List records = Lists.newArrayList(reader.iterator()); + for (GenericData.Record row : records) { + assertThat(row.get(0)).as("Field-ID should be inherited").isNull(); + } + } + } + } + + @TestTemplate + public void testSnapshotPartitionedV1() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet PARTITIONED BY (id) LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s (id, data) VALUES (1, 'a'), (2, 'b')", SOURCE_NAME); + assertThat( + sql( + "CALL %s.system.snapshot(source_table => '%s', table => '%s', properties => map('format-version', '1'))", + catalogName, SOURCE_NAME, tableName)) + .containsExactly(row(2L)); + assertThat(sql("SELECT * FROM %s ORDER BY id", tableName)) + .containsExactly(row("a", 1L), row("b", 2L)); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row("a", 1L), row("b", 2L)), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + Table createdTable = validationCatalog.loadTable(tableIdent); + + for (ManifestFile manifest : createdTable.currentSnapshot().dataManifests(createdTable.io())) { + try (AvroIterable reader = + Avro.read(org.apache.iceberg.Files.localInput(manifest.path())) + .project(SNAPSHOT_ID_READ_SCHEMA) + .createResolvingReader(GenericAvroReader::create) + .build()) { + + assertThat(reader.getMetadata()).containsEntry("format-version", "1"); + + List records = Lists.newArrayList(reader.iterator()); + for (GenericData.Record row : records) { + assertThat(row.get(0)).as("Field-ID should not be inherited").isNotNull(); + } + } + } + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java new file mode 100644 index 000000000000..ce609450c097 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java @@ -0,0 +1,316 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.apache.commons.lang3.StringUtils; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.TableProperties; +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.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkSQLProperties; +import org.apache.spark.sql.execution.SparkPlan; +import org.apache.spark.sql.internal.SQLConf; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestStoragePartitionedJoinsInRowLevelOperations extends ExtensionsTestBase { + + private static final String OTHER_TABLE_NAME = "other_table"; + + // open file cost and split size are set as 16 MB to produce a split per file + private static final Map COMMON_TABLE_PROPERTIES = + ImmutableMap.of( + TableProperties.FORMAT_VERSION, + "2", + TableProperties.SPLIT_SIZE, + "16777216", + TableProperties.SPLIT_OPEN_FILE_COST, + "16777216"); + + // only v2 bucketing and preserve data grouping properties have to be enabled to trigger SPJ + // other properties are only to simplify testing and validation + private static final Map ENABLED_SPJ_SQL_CONF = + ImmutableMap.of( + SQLConf.V2_BUCKETING_ENABLED().key(), + "true", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED().key(), + "true", + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(), + "false", + SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), + "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(), + "-1", + SparkSQLProperties.PRESERVE_DATA_GROUPING, + "true"); + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties() + } + }; + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME)); + } + + @TestTemplate + public void testCopyOnWriteDeleteWithoutShuffles() { + checkDelete(COPY_ON_WRITE); + } + + @TestTemplate + public void testMergeOnReadDeleteWithoutShuffles() { + checkDelete(MERGE_ON_READ); + } + + private void checkDelete(RowLevelOperationMode mode) { + String createTableStmt = + "CREATE TABLE %s (id INT, salary INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep) " + + "TBLPROPERTIES (%s)"; + + sql(createTableStmt, tableName, tablePropsAsString(COMMON_TABLE_PROPERTIES)); + + append(tableName, "{ \"id\": 1, \"salary\": 100, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 2, \"salary\": 200, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 3, \"salary\": 300, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 4, \"salary\": 400, \"dep\": \"hardware\" }"); + + sql(createTableStmt, tableName(OTHER_TABLE_NAME), tablePropsAsString(COMMON_TABLE_PROPERTIES)); + + append(tableName(OTHER_TABLE_NAME), "{ \"id\": 1, \"salary\": 110, \"dep\": \"hr\" }"); + append(tableName(OTHER_TABLE_NAME), "{ \"id\": 5, \"salary\": 500, \"dep\": \"hr\" }"); + + Map deleteTableProps = + ImmutableMap.of( + TableProperties.DELETE_MODE, + mode.modeName(), + TableProperties.DELETE_DISTRIBUTION_MODE, + "none"); + + sql("ALTER TABLE %s SET TBLPROPERTIES(%s)", tableName, tablePropsAsString(deleteTableProps)); + + withSQLConf( + ENABLED_SPJ_SQL_CONF, + () -> { + SparkPlan plan = + executeAndKeepPlan( + "DELETE FROM %s t WHERE " + + "EXISTS (SELECT 1 FROM %s s WHERE t.id = s.id AND t.dep = s.dep)", + tableName, tableName(OTHER_TABLE_NAME)); + String planAsString = plan.toString(); + if (mode == COPY_ON_WRITE) { + int actualNumShuffles = StringUtils.countMatches(planAsString, "Exchange"); + assertThat(actualNumShuffles).as("Should be 1 shuffle with SPJ").isEqualTo(1); + assertThat(planAsString).contains("Exchange hashpartitioning(_file"); + } else { + assertThat(planAsString).doesNotContain("Exchange"); + } + }); + + ImmutableList expectedRows = + ImmutableList.of( + row(2, 200, "hr"), // remaining + row(3, 300, "hr"), // remaining + row(4, 400, "hardware")); // remaining + + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id, salary", tableName)); + } + + @TestTemplate + public void testCopyOnWriteUpdateWithoutShuffles() { + checkUpdate(COPY_ON_WRITE); + } + + @TestTemplate + public void testMergeOnReadUpdateWithoutShuffles() { + checkUpdate(MERGE_ON_READ); + } + + private void checkUpdate(RowLevelOperationMode mode) { + String createTableStmt = + "CREATE TABLE %s (id INT, salary INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep) " + + "TBLPROPERTIES (%s)"; + + sql(createTableStmt, tableName, tablePropsAsString(COMMON_TABLE_PROPERTIES)); + + append(tableName, "{ \"id\": 1, \"salary\": 100, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 2, \"salary\": 200, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 3, \"salary\": 300, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 4, \"salary\": 400, \"dep\": \"hardware\" }"); + + sql(createTableStmt, tableName(OTHER_TABLE_NAME), tablePropsAsString(COMMON_TABLE_PROPERTIES)); + + append(tableName(OTHER_TABLE_NAME), "{ \"id\": 1, \"salary\": 110, \"dep\": \"hr\" }"); + append(tableName(OTHER_TABLE_NAME), "{ \"id\": 5, \"salary\": 500, \"dep\": \"hr\" }"); + + Map updateTableProps = + ImmutableMap.of( + TableProperties.UPDATE_MODE, + mode.modeName(), + TableProperties.UPDATE_DISTRIBUTION_MODE, + "none"); + + sql("ALTER TABLE %s SET TBLPROPERTIES(%s)", tableName, tablePropsAsString(updateTableProps)); + + withSQLConf( + ENABLED_SPJ_SQL_CONF, + () -> { + SparkPlan plan = + executeAndKeepPlan( + "UPDATE %s t SET salary = -1 WHERE " + + "EXISTS (SELECT 1 FROM %s s WHERE t.id = s.id AND t.dep = s.dep)", + tableName, tableName(OTHER_TABLE_NAME)); + String planAsString = plan.toString(); + if (mode == COPY_ON_WRITE) { + int actualNumShuffles = StringUtils.countMatches(planAsString, "Exchange"); + assertThat(actualNumShuffles).as("Should be 1 shuffle with SPJ").isEqualTo(1); + assertThat(planAsString).contains("Exchange hashpartitioning(_file"); + } else { + assertThat(planAsString).doesNotContain("Exchange"); + } + }); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, -1, "hr"), // updated + row(2, 200, "hr"), // existing + row(3, 300, "hr"), // existing + row(4, 400, "hardware")); // existing + + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id, salary", tableName)); + } + + @TestTemplate + public void testCopyOnWriteMergeWithoutShuffles() { + checkMerge(COPY_ON_WRITE, false /* with ON predicate */); + } + + @TestTemplate + public void testCopyOnWriteMergeWithoutShufflesWithPredicate() { + checkMerge(COPY_ON_WRITE, true /* with ON predicate */); + } + + @TestTemplate + public void testMergeOnReadMergeWithoutShuffles() { + checkMerge(MERGE_ON_READ, false /* with ON predicate */); + } + + @TestTemplate + public void testMergeOnReadMergeWithoutShufflesWithPredicate() { + checkMerge(MERGE_ON_READ, true /* with ON predicate */); + } + + private void checkMerge(RowLevelOperationMode mode, boolean withPredicate) { + String createTableStmt = + "CREATE TABLE %s (id INT, salary INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep) " + + "TBLPROPERTIES (%s)"; + + sql(createTableStmt, tableName, tablePropsAsString(COMMON_TABLE_PROPERTIES)); + + append(tableName, "{ \"id\": 1, \"salary\": 100, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 2, \"salary\": 200, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 3, \"salary\": 300, \"dep\": \"hr\" }"); + append(tableName, "{ \"id\": 4, \"salary\": 400, \"dep\": \"hardware\" }"); + append(tableName, "{ \"id\": 6, \"salary\": 600, \"dep\": \"software\" }"); + + sql(createTableStmt, tableName(OTHER_TABLE_NAME), tablePropsAsString(COMMON_TABLE_PROPERTIES)); + + append(tableName(OTHER_TABLE_NAME), "{ \"id\": 1, \"salary\": 110, \"dep\": \"hr\" }"); + append(tableName(OTHER_TABLE_NAME), "{ \"id\": 5, \"salary\": 500, \"dep\": \"hr\" }"); + append(tableName(OTHER_TABLE_NAME), "{ \"id\": 6, \"salary\": 300, \"dep\": \"software\" }"); + append(tableName(OTHER_TABLE_NAME), "{ \"id\": 10, \"salary\": 1000, \"dep\": \"ops\" }"); + + Map mergeTableProps = + ImmutableMap.of( + TableProperties.MERGE_MODE, + mode.modeName(), + TableProperties.MERGE_DISTRIBUTION_MODE, + "none"); + + sql("ALTER TABLE %s SET TBLPROPERTIES(%s)", tableName, tablePropsAsString(mergeTableProps)); + + withSQLConf( + ENABLED_SPJ_SQL_CONF, + () -> { + String predicate = withPredicate ? "AND t.dep IN ('hr', 'ops', 'software')" : ""; + SparkPlan plan = + executeAndKeepPlan( + "MERGE INTO %s AS t USING %s AS s " + + "ON t.id = s.id AND t.dep = s.dep %s " + + "WHEN MATCHED THEN " + + " UPDATE SET t.salary = s.salary " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + tableName, tableName(OTHER_TABLE_NAME), predicate); + String planAsString = plan.toString(); + if (mode == COPY_ON_WRITE) { + int actualNumShuffles = StringUtils.countMatches(planAsString, "Exchange"); + assertThat(actualNumShuffles).as("Should be 1 shuffle with SPJ").isEqualTo(1); + assertThat(planAsString).contains("Exchange hashpartitioning(_file"); + } else { + assertThat(planAsString).doesNotContain("Exchange"); + } + }); + + ImmutableList expectedRows = + ImmutableList.of( + row(1, 110, "hr"), // updated + row(2, 200, "hr"), // existing + row(3, 300, "hr"), // existing + row(4, 400, "hardware"), // existing + row(5, 500, "hr"), // new + row(6, 300, "software"), // updated + row(10, 1000, "ops")); // new + + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT * FROM %s ORDER BY id, salary", tableName)); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java new file mode 100644 index 000000000000..f6102bab69b0 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java @@ -0,0 +1,313 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.expressions.Expressions.bucket; +import static org.apache.iceberg.expressions.Expressions.day; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.hour; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.month; +import static org.apache.iceberg.expressions.Expressions.notEqual; +import static org.apache.iceberg.expressions.Expressions.truncate; +import static org.apache.iceberg.expressions.Expressions.year; +import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.STRUCT; +import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.createPartitionedTable; +import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.createUnpartitionedTable; +import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToDayOrdinal; +import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToHourOrdinal; +import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToMonthOrdinal; +import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToYearOrdinal; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.expressions.ExpressionUtil; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.source.PlanUtils; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.expressions.ApplyFunctionExpression; +import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSystemFunctionPushDownDQL extends ExtensionsTestBase { + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties(), + }, + }; + } + + @BeforeEach + public void before() { + super.before(); + sql("USE %s", catalogName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testYearsFunctionOnUnpartitionedTable() { + createUnpartitionedTable(spark, tableName); + testYearsFunction(false); + } + + @TestTemplate + public void testYearsFunctionOnPartitionedTable() { + createPartitionedTable(spark, tableName, "years(ts)"); + testYearsFunction(true); + } + + private void testYearsFunction(boolean partitioned) { + int targetYears = timestampStrToYearOrdinal("2017-11-22T00:00:00.000000+00:00"); + String query = + String.format( + "SELECT * FROM %s WHERE system.years(ts) = %s ORDER BY id", tableName, targetYears); + + Dataset df = spark.sql(query); + LogicalPlan optimizedPlan = df.queryExecution().optimizedPlan(); + + checkExpressions(optimizedPlan, partitioned, "years"); + checkPushedFilters(optimizedPlan, equal(year("ts"), targetYears)); + + List actual = rowsToJava(df.collectAsList()); + assertThat(actual).hasSize(5); + } + + @TestTemplate + public void testMonthsFunctionOnUnpartitionedTable() { + createUnpartitionedTable(spark, tableName); + testMonthsFunction(false); + } + + @TestTemplate + public void testMonthsFunctionOnPartitionedTable() { + createPartitionedTable(spark, tableName, "months(ts)"); + testMonthsFunction(true); + } + + private void testMonthsFunction(boolean partitioned) { + int targetMonths = timestampStrToMonthOrdinal("2017-11-22T00:00:00.000000+00:00"); + String query = + String.format( + "SELECT * FROM %s WHERE system.months(ts) > %s ORDER BY id", tableName, targetMonths); + + Dataset df = spark.sql(query); + LogicalPlan optimizedPlan = df.queryExecution().optimizedPlan(); + + checkExpressions(optimizedPlan, partitioned, "months"); + checkPushedFilters(optimizedPlan, greaterThan(month("ts"), targetMonths)); + + List actual = rowsToJava(df.collectAsList()); + assertThat(actual).hasSize(5); + } + + @TestTemplate + public void testDaysFunctionOnUnpartitionedTable() { + createUnpartitionedTable(spark, tableName); + testDaysFunction(false); + } + + @TestTemplate + public void testDaysFunctionOnPartitionedTable() { + createPartitionedTable(spark, tableName, "days(ts)"); + testDaysFunction(true); + } + + private void testDaysFunction(boolean partitioned) { + String timestamp = "2018-11-20T00:00:00.000000+00:00"; + int targetDays = timestampStrToDayOrdinal(timestamp); + String query = + String.format( + "SELECT * FROM %s WHERE system.days(ts) < date('%s') ORDER BY id", + tableName, timestamp); + + Dataset df = spark.sql(query); + LogicalPlan optimizedPlan = df.queryExecution().optimizedPlan(); + + checkExpressions(optimizedPlan, partitioned, "days"); + checkPushedFilters(optimizedPlan, lessThan(day("ts"), targetDays)); + + List actual = rowsToJava(df.collectAsList()); + assertThat(actual).hasSize(5); + } + + @TestTemplate + public void testHoursFunctionOnUnpartitionedTable() { + createUnpartitionedTable(spark, tableName); + testHoursFunction(false); + } + + @TestTemplate + public void testHoursFunctionOnPartitionedTable() { + createPartitionedTable(spark, tableName, "hours(ts)"); + testHoursFunction(true); + } + + private void testHoursFunction(boolean partitioned) { + int targetHours = timestampStrToHourOrdinal("2017-11-22T06:02:09.243857+00:00"); + String query = + String.format( + "SELECT * FROM %s WHERE system.hours(ts) >= %s ORDER BY id", tableName, targetHours); + + Dataset df = spark.sql(query); + LogicalPlan optimizedPlan = df.queryExecution().optimizedPlan(); + + checkExpressions(optimizedPlan, partitioned, "hours"); + checkPushedFilters(optimizedPlan, greaterThanOrEqual(hour("ts"), targetHours)); + + List actual = rowsToJava(df.collectAsList()); + assertThat(actual).hasSize(8); + } + + @TestTemplate + public void testBucketLongFunctionOnUnpartitionedTable() { + createUnpartitionedTable(spark, tableName); + testBucketLongFunction(false); + } + + @TestTemplate + public void testBucketLongFunctionOnPartitionedTable() { + createPartitionedTable(spark, tableName, "bucket(5, id)"); + testBucketLongFunction(true); + } + + private void testBucketLongFunction(boolean partitioned) { + int target = 2; + String query = + String.format( + "SELECT * FROM %s WHERE system.bucket(5, id) <= %s ORDER BY id", tableName, target); + + Dataset df = spark.sql(query); + LogicalPlan optimizedPlan = df.queryExecution().optimizedPlan(); + + checkExpressions(optimizedPlan, partitioned, "bucket"); + checkPushedFilters(optimizedPlan, lessThanOrEqual(bucket("id", 5), target)); + + List actual = rowsToJava(df.collectAsList()); + assertThat(actual).hasSize(5); + } + + @TestTemplate + public void testBucketStringFunctionOnUnpartitionedTable() { + createUnpartitionedTable(spark, tableName); + testBucketStringFunction(false); + } + + @TestTemplate + public void testBucketStringFunctionOnPartitionedTable() { + createPartitionedTable(spark, tableName, "bucket(5, data)"); + testBucketStringFunction(true); + } + + private void testBucketStringFunction(boolean partitioned) { + int target = 2; + String query = + String.format( + "SELECT * FROM %s WHERE system.bucket(5, data) != %s ORDER BY id", tableName, target); + + Dataset df = spark.sql(query); + LogicalPlan optimizedPlan = df.queryExecution().optimizedPlan(); + + checkExpressions(optimizedPlan, partitioned, "bucket"); + checkPushedFilters(optimizedPlan, notEqual(bucket("data", 5), target)); + + List actual = rowsToJava(df.collectAsList()); + assertThat(actual).hasSize(8); + } + + @TestTemplate + public void testTruncateFunctionOnUnpartitionedTable() { + createUnpartitionedTable(spark, tableName); + testTruncateFunction(false); + } + + @TestTemplate + public void testTruncateFunctionOnPartitionedTable() { + createPartitionedTable(spark, tableName, "truncate(4, data)"); + testTruncateFunction(true); + } + + private void testTruncateFunction(boolean partitioned) { + String target = "data"; + String query = + String.format( + "SELECT * FROM %s WHERE system.truncate(4, data) = '%s' ORDER BY id", + tableName, target); + + Dataset df = spark.sql(query); + LogicalPlan optimizedPlan = df.queryExecution().optimizedPlan(); + + checkExpressions(optimizedPlan, partitioned, "truncate"); + checkPushedFilters(optimizedPlan, equal(truncate("data", 4), target)); + + List actual = rowsToJava(df.collectAsList()); + assertThat(actual).hasSize(5); + } + + private void checkExpressions( + LogicalPlan optimizedPlan, boolean partitioned, String expectedFunctionName) { + List staticInvokes = + PlanUtils.collectSparkExpressions( + optimizedPlan, expression -> expression instanceof StaticInvoke); + assertThat(staticInvokes).isEmpty(); + + List applyExpressions = + PlanUtils.collectSparkExpressions( + optimizedPlan, expression -> expression instanceof ApplyFunctionExpression); + + if (partitioned) { + assertThat(applyExpressions).isEmpty(); + } else { + assertThat(applyExpressions).hasSize(1); + ApplyFunctionExpression expression = (ApplyFunctionExpression) applyExpressions.get(0); + assertThat(expression.name()).isEqualTo(expectedFunctionName); + } + } + + private void checkPushedFilters( + LogicalPlan optimizedPlan, org.apache.iceberg.expressions.Expression expected) { + List pushedFilters = + PlanUtils.collectPushDownFilters(optimizedPlan); + assertThat(pushedFilters).hasSize(1); + org.apache.iceberg.expressions.Expression actual = pushedFilters.get(0); + assertThat(ExpressionUtil.equivalent(expected, actual, STRUCT, true)) + .as("Pushed filter should match") + .isTrue(); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java new file mode 100644 index 000000000000..934220e5d31e --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java @@ -0,0 +1,365 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.catalyst.expressions.ApplyFunctionExpression; +import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke; +import org.apache.spark.sql.execution.CommandResultExec; +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSystemFunctionPushDownInRowLevelOperations extends ExtensionsTestBase { + + private static final String CHANGES_TABLE_NAME = "changes"; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties() + } + }; + } + + @BeforeEach + public void beforeEach() { + sql("USE %s", catalogName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s PURGE", tableName); + sql("DROP TABLE IF EXISTS %s PURGE", tableName(CHANGES_TABLE_NAME)); + } + + @TestTemplate + public void testCopyOnWriteDeleteBucketTransformInPredicate() { + initTable("bucket(4, dep)"); + checkDelete(COPY_ON_WRITE, "system.bucket(4, dep) IN (2, 3)"); + } + + @TestTemplate + public void testMergeOnReadDeleteBucketTransformInPredicate() { + initTable("bucket(4, dep)"); + checkDelete(MERGE_ON_READ, "system.bucket(4, dep) IN (2, 3)"); + } + + @TestTemplate + public void testCopyOnWriteDeleteBucketTransformEqPredicate() { + initTable("bucket(4, dep)"); + checkDelete(COPY_ON_WRITE, "system.bucket(4, dep) = 2"); + } + + @TestTemplate + public void testMergeOnReadDeleteBucketTransformEqPredicate() { + initTable("bucket(4, dep)"); + checkDelete(MERGE_ON_READ, "system.bucket(4, dep) = 2"); + } + + @TestTemplate + public void testCopyOnWriteDeleteYearsTransform() { + initTable("years(ts)"); + checkDelete(COPY_ON_WRITE, "system.years(ts) > 30"); + } + + @TestTemplate + public void testMergeOnReadDeleteYearsTransform() { + initTable("years(ts)"); + checkDelete(MERGE_ON_READ, "system.years(ts) <= 30"); + } + + @TestTemplate + public void testCopyOnWriteDeleteMonthsTransform() { + initTable("months(ts)"); + checkDelete(COPY_ON_WRITE, "system.months(ts) <= 250"); + } + + @TestTemplate + public void testMergeOnReadDeleteMonthsTransform() { + initTable("months(ts)"); + checkDelete(MERGE_ON_READ, "system.months(ts) > 250"); + } + + @TestTemplate + public void testCopyOnWriteDeleteDaysTransform() { + initTable("days(ts)"); + checkDelete(COPY_ON_WRITE, "system.days(ts) <= date('2000-01-03 00:00:00')"); + } + + @TestTemplate + public void testMergeOnReadDeleteDaysTransform() { + initTable("days(ts)"); + checkDelete(MERGE_ON_READ, "system.days(ts) > date('2000-01-03 00:00:00')"); + } + + @TestTemplate + public void testCopyOnWriteDeleteHoursTransform() { + initTable("hours(ts)"); + checkDelete(COPY_ON_WRITE, "system.hours(ts) <= 100000"); + } + + @TestTemplate + public void testMergeOnReadDeleteHoursTransform() { + initTable("hours(ts)"); + checkDelete(MERGE_ON_READ, "system.hours(ts) > 100000"); + } + + @TestTemplate + public void testCopyOnWriteDeleteTruncateTransform() { + initTable("truncate(1, dep)"); + checkDelete(COPY_ON_WRITE, "system.truncate(1, dep) = 'i'"); + } + + @TestTemplate + public void testMergeOnReadDeleteTruncateTransform() { + initTable("truncate(1, dep)"); + checkDelete(MERGE_ON_READ, "system.truncate(1, dep) = 'i'"); + } + + @TestTemplate + public void testCopyOnWriteUpdateBucketTransform() { + initTable("bucket(4, dep)"); + checkUpdate(COPY_ON_WRITE, "system.bucket(4, dep) IN (2, 3)"); + } + + @TestTemplate + public void testMergeOnReadUpdateBucketTransform() { + initTable("bucket(4, dep)"); + checkUpdate(MERGE_ON_READ, "system.bucket(4, dep) = 2"); + } + + @TestTemplate + public void testCopyOnWriteUpdateYearsTransform() { + initTable("years(ts)"); + checkUpdate(COPY_ON_WRITE, "system.years(ts) > 30"); + } + + @TestTemplate + public void testMergeOnReadUpdateYearsTransform() { + initTable("years(ts)"); + checkUpdate(MERGE_ON_READ, "system.years(ts) <= 30"); + } + + @TestTemplate + public void testCopyOnWriteMergeBucketTransform() { + initTable("bucket(4, dep)"); + checkMerge(COPY_ON_WRITE, "system.bucket(4, dep) IN (2, 3)"); + } + + @TestTemplate + public void testMergeOnReadMergeBucketTransform() { + initTable("bucket(4, dep)"); + checkMerge(MERGE_ON_READ, "system.bucket(4, dep) = 2"); + } + + @TestTemplate + public void testCopyOnWriteMergeYearsTransform() { + initTable("years(ts)"); + checkMerge(COPY_ON_WRITE, "system.years(ts) > 30"); + } + + @TestTemplate + public void testMergeOnReadMergeYearsTransform() { + initTable("years(ts)"); + checkMerge(MERGE_ON_READ, "system.years(ts) <= 30"); + } + + @TestTemplate + public void testCopyOnWriteMergeTruncateTransform() { + initTable("truncate(1, dep)"); + checkMerge(COPY_ON_WRITE, "system.truncate(1, dep) = 'i'"); + } + + @TestTemplate + public void testMergeOnReadMergeTruncateTransform() { + initTable("truncate(1, dep)"); + checkMerge(MERGE_ON_READ, "system.truncate(1, dep) = 'i'"); + } + + private void checkDelete(RowLevelOperationMode mode, String cond) { + withUnavailableLocations( + findIrrelevantFileLocations(cond), + () -> { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s', '%s' '%s')", + tableName, + TableProperties.DELETE_MODE, + mode.modeName(), + TableProperties.DELETE_DISTRIBUTION_MODE, + DistributionMode.NONE.modeName()); + + Dataset changeDF = spark.table(tableName).where(cond).limit(2).select("id"); + try { + changeDF.coalesce(1).writeTo(tableName(CHANGES_TABLE_NAME)).create(); + } catch (TableAlreadyExistsException e) { + throw new AlreadyExistsException( + "Cannot create table %s as it already exists", CHANGES_TABLE_NAME); + } + + List calls = + executeAndCollectFunctionCalls( + "DELETE FROM %s t WHERE %s AND t.id IN (SELECT id FROM %s)", + tableName, cond, tableName(CHANGES_TABLE_NAME)); + // CoW planning currently does not optimize post-scan filters in DELETE + int expectedCallCount = mode == COPY_ON_WRITE ? 1 : 0; + assertThat(calls).hasSize(expectedCallCount); + + assertEquals( + "Should have no matching rows", + ImmutableList.of(), + sql( + "SELECT * FROM %s WHERE %s AND id IN (SELECT * FROM %s)", + tableName, cond, tableName(CHANGES_TABLE_NAME))); + }); + } + + private void checkUpdate(RowLevelOperationMode mode, String cond) { + withUnavailableLocations( + findIrrelevantFileLocations(cond), + () -> { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s', '%s' '%s')", + tableName, + TableProperties.UPDATE_MODE, + mode.modeName(), + TableProperties.UPDATE_DISTRIBUTION_MODE, + DistributionMode.NONE.modeName()); + + Dataset changeDF = spark.table(tableName).where(cond).limit(2).select("id"); + try { + changeDF.coalesce(1).writeTo(tableName(CHANGES_TABLE_NAME)).create(); + } catch (TableAlreadyExistsException e) { + throw new AlreadyExistsException( + "Cannot create table %s as it already exists", CHANGES_TABLE_NAME); + } + + List calls = + executeAndCollectFunctionCalls( + "UPDATE %s t SET t.salary = -1 WHERE %s AND t.id IN (SELECT id FROM %s)", + tableName, cond, tableName(CHANGES_TABLE_NAME)); + // CoW planning currently does not optimize post-scan filters in UPDATE + int expectedCallCount = mode == COPY_ON_WRITE ? 2 : 0; + assertThat(calls).hasSize(expectedCallCount); + + assertEquals( + "Should have correct updates", + sql("SELECT id FROM %s", tableName(CHANGES_TABLE_NAME)), + sql("SELECT id FROM %s WHERE %s AND salary = -1", tableName, cond)); + }); + } + + private void checkMerge(RowLevelOperationMode mode, String cond) { + withUnavailableLocations( + findIrrelevantFileLocations(cond), + () -> { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s', '%s' '%s')", + tableName, + TableProperties.MERGE_MODE, + mode.modeName(), + TableProperties.MERGE_DISTRIBUTION_MODE, + DistributionMode.NONE.modeName()); + + Dataset changeDF = + spark.table(tableName).where(cond).limit(2).selectExpr("id + 1 as id"); + try { + changeDF.coalesce(1).writeTo(tableName(CHANGES_TABLE_NAME)).create(); + } catch (TableAlreadyExistsException e) { + throw new AlreadyExistsException( + "Cannot create table %s as it already exists", CHANGES_TABLE_NAME); + } + + List calls = + executeAndCollectFunctionCalls( + "MERGE INTO %s t USING %s s " + + "ON t.id == s.id AND %s " + + "WHEN MATCHED THEN " + + " UPDATE SET salary = -1 " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT (id, salary, dep, ts) VALUES (100, -1, 'hr', null)", + tableName, tableName(CHANGES_TABLE_NAME), cond); + assertThat(calls).isEmpty(); + + assertEquals( + "Should have correct updates", + sql("SELECT id FROM %s", tableName(CHANGES_TABLE_NAME)), + sql("SELECT id FROM %s WHERE %s AND salary = -1", tableName, cond)); + }); + } + + private List executeAndCollectFunctionCalls(String query, Object... args) { + CommandResultExec command = (CommandResultExec) executeAndKeepPlan(query, args); + V2TableWriteExec write = (V2TableWriteExec) command.commandPhysicalPlan(); + return SparkPlanUtil.collectExprs( + write.query(), + expr -> expr instanceof StaticInvoke || expr instanceof ApplyFunctionExpression); + } + + private List findIrrelevantFileLocations(String cond) { + return spark + .table(tableName) + .where("NOT " + cond) + .select(MetadataColumns.FILE_PATH.name()) + .distinct() + .as(Encoders.STRING()) + .collectAsList(); + } + + private void initTable(String transform) { + sql( + "CREATE TABLE %s (id BIGINT, salary INT, dep STRING, ts TIMESTAMP)" + + "USING iceberg " + + "PARTITIONED BY (%s)", + tableName, transform); + + append( + tableName, + "{ \"id\": 1, \"salary\": 100, \"dep\": \"hr\", \"ts\": \"1975-01-01 06:00:00\" }", + "{ \"id\": 2, \"salary\": 200, \"dep\": \"hr\", \"ts\": \"1975-01-01 06:00:00\" }", + "{ \"id\": 3, \"salary\": 300, \"dep\": \"hr\", \"ts\": \"1975-01-01 06:00:00\" }", + "{ \"id\": 4, \"salary\": 400, \"dep\": \"it\", \"ts\": \"2020-01-01 10:00:00\" }", + "{ \"id\": 5, \"salary\": 500, \"dep\": \"it\", \"ts\": \"2020-01-01 10:00:00\" }", + "{ \"id\": 6, \"salary\": 600, \"dep\": \"it\", \"ts\": \"2020-01-01 10:00:00\" }"); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java new file mode 100644 index 000000000000..79d5b0ab41f1 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestTagDDL.java @@ -0,0 +1,365 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import java.util.Locale; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.extensions.IcebergParseException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestTagDDL extends ExtensionsTestBase { + private static final String[] TIME_UNITS = {"DAYS", "HOURS", "MINUTES"}; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties() + } + }; + } + + @BeforeEach + public void createTable() { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + } + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testCreateTagWithRetain() throws NoSuchTableException { + Table table = insertRows(); + long firstSnapshotId = table.currentSnapshot().snapshotId(); + long maxRefAge = 10L; + + List records = + ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.writeTo(tableName).append(); + + for (String timeUnit : TIME_UNITS) { + String tagName = "t1" + timeUnit; + sql( + "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN %d %s", + tableName, tagName, firstSnapshotId, maxRefAge, timeUnit); + table.refresh(); + SnapshotRef ref = table.refs().get(tagName); + assertThat(ref.snapshotId()) + .as("The tag needs to point to a specific snapshot id.") + .isEqualTo(firstSnapshotId); + assertThat(ref.maxRefAgeMs().longValue()) + .as("The tag needs to have the correct max ref age.") + .isEqualTo(TimeUnit.valueOf(timeUnit.toUpperCase(Locale.ENGLISH)).toMillis(maxRefAge)); + } + + String tagName = "t1"; + assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN", + tableName, tagName, firstSnapshotId, maxRefAge)) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("no viable alternative at input ''"); + + assertThatThrownBy( + () -> sql("ALTER TABLE %s CREATE TAG %s RETAIN %s DAYS", tableName, tagName, "abc")) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("no viable alternative at input 'abc'"); + + assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN %d SECONDS", + tableName, tagName, firstSnapshotId, maxRefAge)) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input 'SECONDS' expecting {'DAYS', 'HOURS', 'MINUTES'}"); + } + + @TestTemplate + public void testCreateTagOnEmptyTable() { + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "abc")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot complete create or replace tag operation on %s, main has no snapshot", + tableName); + } + + @TestTemplate + public void testCreateTagUseDefaultConfig() throws NoSuchTableException { + Table table = insertRows(); + long snapshotId = table.currentSnapshot().snapshotId(); + String tagName = "t1"; + + assertThatThrownBy( + () -> sql("ALTER TABLE %s CREATE TAG %s AS OF VERSION %d", tableName, tagName, -1)) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot set " + tagName + " to unknown snapshot: -1"); + + sql("ALTER TABLE %s CREATE TAG %s", tableName, tagName); + table.refresh(); + SnapshotRef ref = table.refs().get(tagName); + assertThat(ref.snapshotId()) + .as("The tag needs to point to a specific snapshot id.") + .isEqualTo(snapshotId); + assertThat(ref.maxRefAgeMs()) + .as("The tag needs to have the default max ref age, which is null.") + .isNull(); + + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, tagName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("already exists"); + + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "123")) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("no viable alternative at input '123'"); + + table.manageSnapshots().removeTag(tagName).commit(); + List records = + ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.writeTo(tableName).append(); + snapshotId = table.currentSnapshot().snapshotId(); + sql("ALTER TABLE %s CREATE TAG %s AS OF VERSION %d", tableName, tagName, snapshotId); + table.refresh(); + ref = table.refs().get(tagName); + assertThat(ref.snapshotId()) + .as("The tag needs to point to a specific snapshot id.") + .isEqualTo(snapshotId); + assertThat(ref.maxRefAgeMs()) + .as("The tag needs to have the default max ref age, which is null.") + .isNull(); + } + + @TestTemplate + public void testCreateTagIfNotExists() throws NoSuchTableException { + long maxSnapshotAge = 2L; + Table table = insertRows(); + String tagName = "t1"; + sql("ALTER TABLE %s CREATE TAG %s RETAIN %d days", tableName, tagName, maxSnapshotAge); + sql("ALTER TABLE %s CREATE TAG IF NOT EXISTS %s", tableName, tagName); + + table.refresh(); + SnapshotRef ref = table.refs().get(tagName); + assertThat(ref.snapshotId()) + .as("The tag needs to point to a specific snapshot id.") + .isEqualTo(table.currentSnapshot().snapshotId()); + assertThat(ref.maxRefAgeMs().longValue()) + .as("The tag needs to have the correct max ref age.") + .isEqualTo(TimeUnit.DAYS.toMillis(maxSnapshotAge)); + } + + @TestTemplate + public void testReplaceTagFailsForBranch() throws NoSuchTableException { + String branchName = "branch1"; + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createBranch(branchName, first).commit(); + insertRows(); + long second = table.currentSnapshot().snapshotId(); + + assertThatThrownBy(() -> sql("ALTER TABLE %s REPLACE Tag %s", tableName, branchName, second)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Ref branch1 is a branch not a tag"); + } + + @TestTemplate + public void testReplaceTag() throws NoSuchTableException { + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + String tagName = "t1"; + long expectedMaxRefAgeMs = 1000; + table + .manageSnapshots() + .createTag(tagName, first) + .setMaxRefAgeMs(tagName, expectedMaxRefAgeMs) + .commit(); + + insertRows(); + long second = table.currentSnapshot().snapshotId(); + + sql("ALTER TABLE %s REPLACE Tag %s AS OF VERSION %d", tableName, tagName, second); + table.refresh(); + SnapshotRef ref = table.refs().get(tagName); + assertThat(ref.snapshotId()) + .as("The tag needs to point to a specific snapshot id.") + .isEqualTo(second); + assertThat(ref.maxRefAgeMs().longValue()) + .as("The tag needs to have the correct max ref age.") + .isEqualTo(expectedMaxRefAgeMs); + } + + @TestTemplate + public void testReplaceTagDoesNotExist() throws NoSuchTableException { + Table table = insertRows(); + + assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s REPLACE Tag %s AS OF VERSION %d", + tableName, "someTag", table.currentSnapshot().snapshotId())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Tag does not exist"); + } + + @TestTemplate + public void testReplaceTagWithRetain() throws NoSuchTableException { + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + String tagName = "t1"; + table.manageSnapshots().createTag(tagName, first).commit(); + insertRows(); + long second = table.currentSnapshot().snapshotId(); + + long maxRefAge = 10; + for (String timeUnit : TIME_UNITS) { + sql( + "ALTER TABLE %s REPLACE Tag %s AS OF VERSION %d RETAIN %d %s", + tableName, tagName, second, maxRefAge, timeUnit); + + table.refresh(); + SnapshotRef ref = table.refs().get(tagName); + assertThat(ref.snapshotId()) + .as("The tag needs to point to a specific snapshot id.") + .isEqualTo(second); + assertThat(ref.maxRefAgeMs().longValue()) + .as("The tag needs to have the correct max ref age.") + .isEqualTo(TimeUnit.valueOf(timeUnit).toMillis(maxRefAge)); + } + } + + @TestTemplate + public void testCreateOrReplace() throws NoSuchTableException { + Table table = insertRows(); + long first = table.currentSnapshot().snapshotId(); + String tagName = "t1"; + insertRows(); + long second = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createTag(tagName, second).commit(); + + sql("ALTER TABLE %s CREATE OR REPLACE TAG %s AS OF VERSION %d", tableName, tagName, first); + table.refresh(); + SnapshotRef ref = table.refs().get(tagName); + assertThat(ref.snapshotId()) + .as("The tag needs to point to a specific snapshot id.") + .isEqualTo(first); + } + + @TestTemplate + public void testDropTag() throws NoSuchTableException { + insertRows(); + Table table = validationCatalog.loadTable(tableIdent); + String tagName = "t1"; + table.manageSnapshots().createTag(tagName, table.currentSnapshot().snapshotId()).commit(); + SnapshotRef ref = table.refs().get(tagName); + assertThat(ref.snapshotId()).isEqualTo(table.currentSnapshot().snapshotId()); + + sql("ALTER TABLE %s DROP TAG %s", tableName, tagName); + table.refresh(); + assertThat(table.refs()).doesNotContainKey(tagName); + } + + @TestTemplate + public void testDropTagNonConformingName() { + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "123")) + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("no viable alternative at input '123'"); + } + + @TestTemplate + public void testDropTagDoesNotExist() { + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "nonExistingTag")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Tag does not exist: nonExistingTag"); + } + + @TestTemplate + public void testDropTagFailesForBranch() throws NoSuchTableException { + String branchName = "b1"; + Table table = insertRows(); + table.manageSnapshots().createBranch(branchName, table.currentSnapshot().snapshotId()).commit(); + + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, branchName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Ref b1 is a branch not a tag"); + } + + @TestTemplate + public void testDropTagIfExists() throws NoSuchTableException { + String tagName = "nonExistingTag"; + Table table = insertRows(); + assertThat(table.refs()).doesNotContainKey(tagName); + + sql("ALTER TABLE %s DROP TAG IF EXISTS %s", tableName, tagName); + table.refresh(); + assertThat(table.refs()).doesNotContainKey(tagName); + + table.manageSnapshots().createTag(tagName, table.currentSnapshot().snapshotId()).commit(); + assertThat(table.refs().get(tagName).snapshotId()) + .as("The tag has been created successfully.") + .isEqualTo(table.currentSnapshot().snapshotId()); + + sql("ALTER TABLE %s DROP TAG IF EXISTS %s", tableName, tagName); + table.refresh(); + assertThat(table.refs()).doesNotContainKey(tagName); + } + + @TestTemplate + public void createOrReplaceWithNonExistingTag() throws NoSuchTableException { + Table table = insertRows(); + String tagName = "t1"; + insertRows(); + long snapshotId = table.currentSnapshot().snapshotId(); + + sql("ALTER TABLE %s CREATE OR REPLACE TAG %s AS OF VERSION %d", tableName, tagName, snapshotId); + table.refresh(); + assertThat(table.refs().get(tagName).snapshotId()).isEqualTo(snapshotId); + } + + private Table insertRows() throws NoSuchTableException { + List records = + ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.writeTo(tableName).append(); + return validationCatalog.loadTable(tableIdent); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java new file mode 100644 index 000000000000..77011fd28cf0 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java @@ -0,0 +1,1518 @@ +/* + * 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.extensions; + +import static org.apache.iceberg.DataOperations.OVERWRITE; +import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; +import static org.apache.iceberg.SnapshotSummary.ADDED_FILES_PROP; +import static org.apache.iceberg.SnapshotSummary.CHANGED_PARTITION_COUNT_PROP; +import static org.apache.iceberg.SnapshotSummary.DELETED_FILES_PROP; +import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.iceberg.TableProperties.SPLIT_SIZE; +import static org.apache.iceberg.TableProperties.UPDATE_DISTRIBUTION_MODE; +import static org.apache.iceberg.TableProperties.UPDATE_ISOLATION_LEVEL; +import static org.apache.iceberg.TableProperties.UPDATE_MODE; +import static org.apache.iceberg.TableProperties.UPDATE_MODE_DEFAULT; +import static org.apache.spark.sql.functions.lit; +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.Arrays; +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.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.exceptions.ValidationException; +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.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; +import org.apache.iceberg.spark.SparkSQLProperties; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.SparkRuntimeException; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.execution.SparkPlan; +import org.apache.spark.sql.internal.SQLConf; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TestUpdate extends SparkRowLevelOperationsTestBase { + + @BeforeAll + public static void setupSparkConf() { + spark.conf().set("spark.sql.shuffle.partitions", "4"); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS updated_id"); + sql("DROP TABLE IF EXISTS updated_dep"); + sql("DROP TABLE IF EXISTS deleted_employee"); + } + + @TestTemplate + public void testUpdateWithVectorizedReads() { + assumeThat(supportsVectorization()).isTrue(); + + createAndInitTable( + "id INT, value INT, dep STRING", + "PARTITIONED BY (dep)", + "{ \"id\": 1, \"value\": 100, \"dep\": \"hr\" }"); + + SparkPlan plan = executeAndKeepPlan("UPDATE %s SET value = -1 WHERE id = 1", commitTarget()); + + assertAllBatchScansVectorized(plan); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, -1, "hr")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testCoalesceUpdate() { + createAndInitTable("id INT, dep STRING"); + + String[] records = new String[100]; + for (int index = 0; index < 100; index++) { + records[index] = String.format("{ \"id\": %d, \"dep\": \"hr\" }", index); + } + append(tableName, records); + append(tableName, records); + append(tableName, records); + append(tableName, records); + + // set the open file cost large enough to produce a separate scan task per file + // use range distribution to trigger a shuffle + // set partitioned scoped deletes so that 1 delete file is written as part of the output task + Map tableProps = + ImmutableMap.of( + SPLIT_OPEN_FILE_COST, + String.valueOf(Integer.MAX_VALUE), + UPDATE_DISTRIBUTION_MODE, + DistributionMode.RANGE.modeName(), + TableProperties.DELETE_GRANULARITY, + DeleteGranularity.PARTITION.toString()); + sql("ALTER TABLE %s SET TBLPROPERTIES (%s)", tableName, tablePropsAsString(tableProps)); + + createBranchIfNeeded(); + + // enable AQE and set the advisory partition size big enough to trigger combining + // set the number of shuffle partitions to 200 to distribute the work across reducers + // set the advisory partition size for shuffles small enough to ensure writes override it + withSQLConf( + ImmutableMap.of( + SQLConf.SHUFFLE_PARTITIONS().key(), + "200", + SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), + "true", + SQLConf.COALESCE_PARTITIONS_ENABLED().key(), + "true", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES().key(), + "100", + SparkSQLProperties.ADVISORY_PARTITION_SIZE, + String.valueOf(256 * 1024 * 1024)), + () -> { + SparkPlan plan = + executeAndKeepPlan("UPDATE %s SET id = -1 WHERE mod(id, 2) = 0", commitTarget()); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + }); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snapshot = SnapshotUtil.latestSnapshot(table, branch); + + if (mode(table) == COPY_ON_WRITE) { + // CoW UPDATE requests the updated records to be range distributed by `_file`, `_pos` + // every task has data for each of 200 reducers + // AQE detects that all shuffle blocks are small and processes them in 1 task + // otherwise, there would be 200 tasks writing to the table + validateProperty(snapshot, SnapshotSummary.ADDED_FILES_PROP, "1"); + } else if (mode(table) == MERGE_ON_READ && formatVersion >= 3) { + validateProperty(snapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "4"); + validateProperty(snapshot, SnapshotSummary.ADDED_DVS_PROP, "4"); + } else { + // MoR UPDATE requests the deleted records to be range distributed by partition and `_file` + // each task contains only 1 file and therefore writes only 1 shuffle block + // that means 4 shuffle blocks are distributed among 200 reducers + // AQE detects that all 4 shuffle blocks are small and processes them in 1 task + // otherwise, there would be 4 tasks processing 1 shuffle block each + validateProperty(snapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "1"); + } + + assertThat(scalarSql("SELECT COUNT(*) FROM %s WHERE id = -1", commitTarget())) + .as("Row count must match") + .isEqualTo(200L); + } + + @TestTemplate + public void testSkewUpdate() { + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + String[] records = new String[100]; + for (int index = 0; index < 100; index++) { + records[index] = String.format("{ \"id\": %d, \"dep\": \"hr\" }", index); + } + append(tableName, records); + append(tableName, records); + append(tableName, records); + append(tableName, records); + + // set the open file cost large enough to produce a separate scan task per file + // use hash distribution to trigger a shuffle + Map tableProps = + ImmutableMap.of( + SPLIT_OPEN_FILE_COST, + String.valueOf(Integer.MAX_VALUE), + UPDATE_DISTRIBUTION_MODE, + DistributionMode.HASH.modeName()); + sql("ALTER TABLE %s SET TBLPROPERTIES (%s)", tableName, tablePropsAsString(tableProps)); + + createBranchIfNeeded(); + + // enable AQE and set the advisory partition size small enough to trigger a split + // set the number of shuffle partitions to 2 to only have 2 reducers + // set the advisory partition size for shuffles big enough to ensure writes override it + withSQLConf( + ImmutableMap.of( + SQLConf.SHUFFLE_PARTITIONS().key(), + "2", + SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), + "true", + SQLConf.ADAPTIVE_OPTIMIZE_SKEWS_IN_REBALANCE_PARTITIONS_ENABLED().key(), + "true", + SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES().key(), + "256MB", + SparkSQLProperties.ADVISORY_PARTITION_SIZE, + "100"), + () -> { + SparkPlan plan = + executeAndKeepPlan("UPDATE %s SET id = -1 WHERE mod(id, 2) = 0", commitTarget()); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + }); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snapshot = SnapshotUtil.latestSnapshot(table, branch); + + if (mode(table) == COPY_ON_WRITE) { + // CoW UPDATE requests the updated records to be clustered by `_file` + // each task contains only 1 file and therefore writes only 1 shuffle block + // that means 4 shuffle blocks are distributed among 2 reducers + // AQE detects that all shuffle blocks are big and processes them in 4 independent tasks + // otherwise, there would be 2 tasks processing 2 shuffle blocks each + validateProperty(snapshot, SnapshotSummary.ADDED_FILES_PROP, "4"); + } else { + // MoR UPDATE requests the deleted records to be clustered by `_spec_id` and `_partition` + // all tasks belong to the same partition and therefore write only 1 shuffle block per task + // that means there are 4 shuffle blocks, all assigned to the same reducer + // AQE detects that all 4 shuffle blocks are big and processes them in 4 separate tasks + // otherwise, there would be 1 task processing 4 shuffle blocks + validateProperty(snapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "4"); + } + + assertThat(scalarSql("SELECT COUNT(*) FROM %s WHERE id = -1", commitTarget())) + .as("Row count must match") + .isEqualTo(200L); + } + + @TestTemplate + public void testExplain() { + createAndInitTable("id INT, dep STRING"); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); + createBranchIfNeeded(); + + sql("EXPLAIN UPDATE %s SET dep = 'invalid' WHERE id <=> 1", commitTarget()); + + sql("EXPLAIN UPDATE %s SET dep = 'invalid' WHERE true", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 1 snapshot").hasSize(1); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testUpdateEmptyTable() { + assumeThat(branch).as("Custom branch does not exist for empty table").isNotEqualTo("test"); + createAndInitTable("id INT, dep STRING"); + + sql("UPDATE %s SET dep = 'invalid' WHERE id IN (1)", commitTarget()); + sql("UPDATE %s SET id = -1 WHERE dep = 'hr'", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); + + assertEquals( + "Should have expected rows", + ImmutableList.of(), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testUpdateNonExistingCustomBranch() { + assumeThat(branch).as("Test only applicable to custom branch").isEqualTo("test"); + createAndInitTable("id INT, dep STRING"); + + assertThatThrownBy(() -> sql("UPDATE %s SET dep = 'invalid' WHERE id IN (1)", commitTarget())) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot use branch (does not exist): test"); + } + + @TestTemplate + public void testUpdateWithAlias() { + createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"a\" }"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + sql("UPDATE %s AS t SET t.dep = 'invalid'", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "invalid")), + sql("SELECT * FROM %s", selectTarget())); + } + + @TestTemplate + public void testUpdateAlignsAssignments() { + createAndInitTable("id INT, c1 INT, c2 INT"); + + sql("INSERT INTO TABLE %s VALUES (1, 11, 111), (2, 22, 222)", tableName); + createBranchIfNeeded(); + + sql("UPDATE %s SET `c2` = c2 - 2, c1 = `c1` - 1 WHERE id <=> 1", commitTarget()); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, 10, 109), row(2, 22, 222)), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testUpdateWithUnsupportedPartitionPredicate() { + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'software'), (2, 'hr')", tableName); + createBranchIfNeeded(); + + sql("UPDATE %s t SET `t`.`id` = -1 WHERE t.dep LIKE '%%r' ", commitTarget()); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(1, "software")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testUpdateWithDynamicFileFiltering() { + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 3, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + append( + commitTarget(), + "{ \"id\": 1, \"dep\": \"hardware\" }\n" + "{ \"id\": 2, \"dep\": \"hardware\" }"); + + sql("UPDATE %s SET id = cast('-1' AS INT) WHERE id = 2", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + if (mode(table) == COPY_ON_WRITE) { + validateCopyOnWrite(currentSnapshot, "1", "1", "1"); + } else { + validateMergeOnRead(currentSnapshot, "1", "1", "1"); + } + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hardware"), row(1, "hardware"), row(1, "hr"), row(3, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", commitTarget())); + } + + @TestTemplate + public void testUpdateNonExistingRecords() { + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); + createBranchIfNeeded(); + + sql("UPDATE %s SET id = -1 WHERE id > 10", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + if (mode(table) == COPY_ON_WRITE) { + validateCopyOnWrite(currentSnapshot, "0", null, null); + } else { + validateMergeOnRead(currentSnapshot, "0", null, null); + } + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testUpdateWithoutCondition() { + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + sql("ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); + createBranchIfNeeded(); + sql("INSERT INTO TABLE %s VALUES (2, 'hardware')", commitTarget()); + sql("INSERT INTO TABLE %s VALUES (null, 'hr')", commitTarget()); + + // set the num of shuffle partitions to 200 instead of default 4 to reduce the chance of hashing + // records for multiple source files to one writing task (needed for a predictable num of output + // files) + withSQLConf( + ImmutableMap.of(SQLConf.SHUFFLE_PARTITIONS().key(), "200"), + () -> { + sql("UPDATE %s SET id = -1", commitTarget()); + }); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 4 snapshots").hasSize(4); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + + assertThat(currentSnapshot.operation()).as("Operation must match").isEqualTo(OVERWRITE); + if (mode(table) == COPY_ON_WRITE) { + assertThat(currentSnapshot.operation()).as("Operation must match").isEqualTo(OVERWRITE); + validateProperty(currentSnapshot, CHANGED_PARTITION_COUNT_PROP, "2"); + validateProperty(currentSnapshot, DELETED_FILES_PROP, "3"); + validateProperty(currentSnapshot, ADDED_FILES_PROP, ImmutableSet.of("2", "3")); + } else { + validateMergeOnRead(currentSnapshot, "2", "3", "2"); + } + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hardware"), row(-1, "hr"), row(-1, "hr")), + sql("SELECT * FROM %s ORDER BY dep ASC", selectTarget())); + } + + @TestTemplate + public void testUpdateWithNullConditions() { + createAndInitTable("id INT, dep STRING"); + + append( + tableName, + "{ \"id\": 0, \"dep\": null }\n" + + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hardware\" }"); + createBranchIfNeeded(); + + // should not update any rows as null is never equal to null + sql("UPDATE %s SET id = -1 WHERE dep = NULL", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(0, null), row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + // should not update any rows the condition does not match any records + sql("UPDATE %s SET id = -1 WHERE dep = 'software'", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(0, null), row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + // should update one matching row with a null-safe condition + sql("UPDATE %s SET dep = 'invalid', id = -1 WHERE dep <=> NULL", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "invalid"), row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testUpdateWithInAndNotInConditions() { + createAndInitTable("id INT, dep STRING"); + + append( + tableName, + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hardware\" }\n" + + "{ \"id\": null, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + + sql("UPDATE %s SET id = -1 WHERE id IN (1, null)", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql("UPDATE %s SET id = 100 WHERE id NOT IN (null, 1)", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql("UPDATE %s SET id = 100 WHERE id NOT IN (1, 10)", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(100, "hardware"), row(100, "hr"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST, dep", selectTarget())); + } + + @TestTemplate + public void testUpdateWithMultipleRowGroupsParquet() throws NoSuchTableException { + assumeThat(fileFormat).isEqualTo(FileFormat.PARQUET); + + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%d')", + tableName, PARQUET_ROW_GROUP_SIZE_BYTES, 100); + sql("ALTER TABLE %s SET TBLPROPERTIES('%s' '%d')", tableName, SPLIT_SIZE, 100); + + List ids = Lists.newArrayListWithCapacity(200); + for (int id = 1; id <= 200; id++) { + ids.add(id); + } + Dataset df = + spark + .createDataset(ids, Encoders.INT()) + .withColumnRenamed("value", "id") + .withColumn("dep", lit("hr")); + df.coalesce(1).writeTo(tableName).append(); + createBranchIfNeeded(); + + assertThat(spark.table(commitTarget()).count()).isEqualTo(200); + + // update a record from one of two row groups and copy over the second one + sql("UPDATE %s SET id = -1 WHERE id IN (200, 201)", commitTarget()); + + assertThat(spark.table(commitTarget()).count()).isEqualTo(200); + } + + @TestTemplate + public void testUpdateNestedStructFields() { + createAndInitTable( + "id INT, s STRUCT,m:MAP>>", + "{ \"id\": 1, \"s\": { \"c1\": 2, \"c2\": { \"a\": [1,2], \"m\": { \"a\": \"b\"} } } } }"); + + // update primitive, array, map columns inside a struct + sql("UPDATE %s SET s.c1 = -1, s.c2.m = map('k', 'v'), s.c2.a = array(-1)", commitTarget()); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, row(-1, row(ImmutableList.of(-1), ImmutableMap.of("k", "v"))))), + sql("SELECT * FROM %s", selectTarget())); + + // set primitive, array, map columns to NULL (proper casts should be in place) + sql("UPDATE %s SET s.c1 = NULL, s.c2 = NULL WHERE id IN (1)", commitTarget()); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, row(null, null))), + sql("SELECT * FROM %s", selectTarget())); + + // update all fields in a struct + sql( + "UPDATE %s SET s = named_struct('c1', 1, 'c2', named_struct('a', array(1), 'm', null))", + commitTarget()); + + assertEquals( + "Output should match", + ImmutableList.of(row(1, row(1, row(ImmutableList.of(1), null)))), + sql("SELECT * FROM %s", selectTarget())); + } + + @TestTemplate + public void testUpdateWithUserDefinedDistribution() { + createAndInitTable("id INT, c2 INT, c3 INT"); + sql("ALTER TABLE %s ADD PARTITION FIELD bucket(8, c3)", tableName); + + append( + tableName, + "{ \"id\": 1, \"c2\": 11, \"c3\": 1 }\n" + + "{ \"id\": 2, \"c2\": 22, \"c3\": 1 }\n" + + "{ \"id\": 3, \"c2\": 33, \"c3\": 1 }"); + createBranchIfNeeded(); + + // request a global sort + sql("ALTER TABLE %s WRITE ORDERED BY c2", tableName); + sql("UPDATE %s SET c2 = -22 WHERE id NOT IN (1, 3)", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, 11, 1), row(2, -22, 1), row(3, 33, 1)), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + // request a local sort + sql("ALTER TABLE %s WRITE LOCALLY ORDERED BY id", tableName); + sql("UPDATE %s SET c2 = -33 WHERE id = 3", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, 11, 1), row(2, -22, 1), row(3, -33, 1)), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + // request a hash distribution + local sort + sql("ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION ORDERED BY id", tableName); + sql("UPDATE %s SET c2 = -11 WHERE id = 1", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, -11, 1), row(2, -22, 1), row(3, -33, 1)), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public synchronized void testUpdateWithSerializableIsolation() throws InterruptedException { + // cannot run tests with concurrency for Hadoop tables without atomic renames + assumeThat(catalogName).isNotEqualToIgnoringCase("testhadoop"); + // if caching is off, the table is eagerly refreshed during runtime filtering + // this can cause a validation exception as concurrent changes would be visible + assumeThat(cachingCatalogEnabled()).isTrue(); + + createAndInitTable("id INT, dep STRING"); + + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + tableName, UPDATE_ISOLATION_LEVEL, "serializable"); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); + createBranchIfNeeded(); + + 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( + () -> { + // load the table via the validation catalog to use another table instance + Table table = validationCatalog.loadTable(tableIdent); + + 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(ValidationException.class) + .hasMessageContaining("Found conflicting files that can contain"); + } finally { + shouldAppend.set(false); + appendFuture.cancel(true); + } + + executorService.shutdown(); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); + } + + @TestTemplate + public synchronized void testUpdateWithSnapshotIsolation() + throws InterruptedException, ExecutionException { + // cannot run tests with concurrency for Hadoop tables without atomic renames + assumeThat(catalogName).isNotEqualToIgnoringCase("testhadoop"); + // if caching is off, the table is eagerly refreshed during runtime filtering + // this can cause a validation exception as concurrent changes would be visible + assumeThat(cachingCatalogEnabled()).isTrue(); + + 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(); + + 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 < 20; 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", tableName); + + barrier.incrementAndGet(); + } + }); + + // append thread + Future appendFuture = + executorService.submit( + () -> { + // load the table via the validation catalog to use another table instance for inserts + Table table = validationCatalog.loadTable(tableIdent); + + GenericRecord record = GenericRecord.create(SnapshotUtil.schemaFor(table, branch)); + record.set(0, 1); // id + record.set(1, "hr"); // dep + + for (int numOperations = 0; numOperations < 20; 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 { + updateFuture.get(); + } finally { + shouldAppend.set(false); + appendFuture.cancel(true); + } + + executorService.shutdown(); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); + } + + @TestTemplate + public void testUpdateWithInferredCasts() { + createAndInitTable("id INT, s STRING", "{ \"id\": 1, \"s\": \"value\" }"); + + sql("UPDATE %s SET s = -1 WHERE id = 1", commitTarget()); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "-1")), + sql("SELECT * FROM %s", selectTarget())); + } + + @TestTemplate + public void testUpdateModifiesNullStruct() { + createAndInitTable("id INT, s STRUCT", "{ \"id\": 1, \"s\": null }"); + + sql("UPDATE %s SET s.n1 = -1 WHERE id = 1", commitTarget()); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, row(-1, null))), + sql("SELECT * FROM %s", selectTarget())); + } + + @TestTemplate + public void testUpdateRefreshesRelationCache() { + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 3, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + + append( + commitTarget(), + "{ \"id\": 1, \"dep\": \"hardware\" }\n" + "{ \"id\": 2, \"dep\": \"hardware\" }"); + + Dataset query = spark.sql("SELECT * FROM " + commitTarget() + " WHERE id = 1"); + query.createOrReplaceTempView("tmp"); + + spark.sql("CACHE TABLE tmp"); + + assertEquals( + "View should have correct data", + ImmutableList.of(row(1, "hardware"), row(1, "hr")), + sql("SELECT * FROM tmp ORDER BY id, dep")); + + sql("UPDATE %s SET id = -1 WHERE id = 1", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + if (mode(table) == COPY_ON_WRITE) { + validateCopyOnWrite(currentSnapshot, "2", "2", "2"); + } else { + validateMergeOnRead(currentSnapshot, "2", "2", "2"); + } + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hardware"), row(-1, "hr"), row(2, "hardware"), row(3, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", commitTarget())); + + assertEquals( + "Should refresh the relation cache", + ImmutableList.of(), + sql("SELECT * FROM tmp ORDER BY id, dep")); + + spark.sql("UNCACHE TABLE tmp"); + } + + @TestTemplate + public void testUpdateWithInSubquery() { + createAndInitTable("id INT, dep STRING"); + + append( + tableName, + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hardware\" }\n" + + "{ \"id\": null, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + + createOrReplaceView("updated_id", Arrays.asList(0, 1, null), Encoders.INT()); + createOrReplaceView("updated_dep", Arrays.asList("software", "hr"), Encoders.STRING()); + + sql( + "UPDATE %s SET id = -1 WHERE " + + "id IN (SELECT * FROM updated_id) AND " + + "dep IN (SELECT * from updated_dep)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql( + "UPDATE %s SET id = 5 WHERE id IS NULL OR id IN (SELECT value + 1 FROM updated_id)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(5, "hardware"), row(5, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); + + append( + commitTarget(), "{ \"id\": null, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"hr\" }"); + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(-1, "hr"), row(2, "hr"), row(5, "hardware"), row(5, "hr"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST, dep", selectTarget())); + + sql( + "UPDATE %s SET id = 10 WHERE id IN (SELECT value + 2 FROM updated_id) AND dep = 'hr'", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of( + row(-1, "hr"), row(5, "hardware"), row(5, "hr"), row(10, "hr"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST, dep", selectTarget())); + } + + @TestTemplate + public void testUpdateWithInSubqueryAndDynamicFileFiltering() { + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + sql("ALTER TABLE %s WRITE DISTRIBUTED BY PARTITION", tableName); + + append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 3, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + append( + commitTarget(), + "{ \"id\": 1, \"dep\": \"hardware\" }\n" + "{ \"id\": 2, \"dep\": \"hardware\" }"); + + createOrReplaceView("updated_id", Arrays.asList(-1, 2), Encoders.INT()); + + sql("UPDATE %s SET id = -1 WHERE id IN (SELECT * FROM updated_id)", commitTarget()); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); + + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + if (mode(table) == COPY_ON_WRITE) { + validateCopyOnWrite(currentSnapshot, "1", "1", "1"); + } else { + validateMergeOnRead(currentSnapshot, "1", "1", "1"); + } + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hardware"), row(1, "hardware"), row(1, "hr"), row(3, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", commitTarget())); + } + + @TestTemplate + public void testUpdateWithSelfSubquery() { + createAndInitTable("id INT, dep STRING"); + + append(tableName, "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + + sql( + "UPDATE %s SET dep = 'x' WHERE id IN (SELECT id + 1 FROM %s)", + commitTarget(), commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "x")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + + // TODO: Spark does not support AQE and DPP with aggregates at the moment + withSQLConf( + ImmutableMap.of(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), "false"), + () -> { + sql( + "UPDATE %s SET dep = 'y' WHERE " + + "id = (SELECT count(*) FROM (SELECT DISTINCT id FROM %s) AS t)", + commitTarget(), commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "y")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + }); + + sql("UPDATE %s SET id = (SELECT id - 2 FROM %s WHERE id = 1)", commitTarget(), commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(-1, "y")), + sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); + } + + @TestTemplate + public void testUpdateWithMultiColumnInSubquery() { + createAndInitTable("id INT, dep STRING"); + + append( + tableName, + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hardware\" }\n" + + "{ \"id\": null, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + + List deletedEmployees = + Arrays.asList(new Employee(null, "hr"), new Employee(1, "hr")); + createOrReplaceView("deleted_employee", deletedEmployees, Encoders.bean(Employee.class)); + + sql( + "UPDATE %s SET dep = 'x', id = -1 WHERE (id, dep) IN (SELECT id, dep FROM deleted_employee)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "x"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + } + + @TestTemplate + public void testUpdateWithNotInSubquery() { + createAndInitTable("id INT, dep STRING"); + + append( + tableName, + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hardware\" }\n" + + "{ \"id\": null, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + + createOrReplaceView("updated_id", Arrays.asList(-1, -2, null), Encoders.INT()); + createOrReplaceView("updated_dep", Arrays.asList("software", "hr"), Encoders.STRING()); + + // the file filter subquery (nested loop lef-anti join) returns 0 records + sql("UPDATE %s SET id = -1 WHERE id NOT IN (SELECT * FROM updated_id)", commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql( + "UPDATE %s SET id = -1 WHERE id NOT IN (SELECT * FROM updated_id WHERE value IS NOT NULL)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hardware"), row(-1, "hr"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST, dep", selectTarget())); + + sql( + "UPDATE %s SET id = 5 WHERE id NOT IN (SELECT * FROM updated_id) OR dep IN ('software', 'hr')", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hardware"), row(5, "hr"), row(5, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST, dep", selectTarget())); + } + + @TestTemplate + public void testUpdateWithExistSubquery() { + createAndInitTable("id INT, dep STRING"); + + append( + tableName, + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hardware\" }\n" + + "{ \"id\": null, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + + createOrReplaceView("updated_id", Arrays.asList(-1, -2, null), Encoders.INT()); + createOrReplaceView("updated_dep", Arrays.asList("hr", null), Encoders.STRING()); + + sql( + "UPDATE %s t SET id = -1 WHERE EXISTS (SELECT 1 FROM updated_id u WHERE t.id = u.value)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql( + "UPDATE %s t SET dep = 'x', id = -1 WHERE " + + "EXISTS (SELECT 1 FROM updated_id u WHERE t.id = u.value + 2)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "x"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + + sql( + "UPDATE %s t SET id = -2 WHERE " + + "EXISTS (SELECT 1 FROM updated_id u WHERE t.id = u.value) OR " + + "t.id IS NULL", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-2, "hr"), row(-2, "x"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); + + sql( + "UPDATE %s t SET id = 1 WHERE " + + "EXISTS (SELECT 1 FROM updated_id ui WHERE t.id = ui.value) AND " + + "EXISTS (SELECT 1 FROM updated_dep ud WHERE t.dep = ud.value)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-2, "x"), row(1, "hr"), row(2, "hardware")), + sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); + } + + @TestTemplate + public void testUpdateWithNotExistsSubquery() { + createAndInitTable("id INT, dep STRING"); + + append( + tableName, + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hardware\" }\n" + + "{ \"id\": null, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + + createOrReplaceView("updated_id", Arrays.asList(-1, -2, null), Encoders.INT()); + createOrReplaceView("updated_dep", Arrays.asList("hr", "software"), Encoders.STRING()); + + sql( + "UPDATE %s t SET id = -1 WHERE NOT EXISTS (SELECT 1 FROM updated_id u WHERE t.id = u.value + 2)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hardware"), row(-1, "hr"), row(1, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); + + sql( + "UPDATE %s t SET id = 5 WHERE " + + "NOT EXISTS (SELECT 1 FROM updated_id u WHERE t.id = u.value) OR " + + "t.id = 1", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hardware"), row(-1, "hr"), row(5, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); + + sql( + "UPDATE %s t SET id = 10 WHERE " + + "NOT EXISTS (SELECT 1 FROM updated_id ui WHERE t.id = ui.value) AND " + + "EXISTS (SELECT 1 FROM updated_dep ud WHERE t.dep = ud.value)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hardware"), row(-1, "hr"), row(10, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); + } + + @TestTemplate + public void testUpdateWithScalarSubquery() { + createAndInitTable("id INT, dep STRING"); + + append( + tableName, + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hardware\" }\n" + + "{ \"id\": null, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + + createOrReplaceView("updated_id", Arrays.asList(1, 100, null), Encoders.INT()); + + // TODO: Spark does not support AQE and DPP with aggregates at the moment + withSQLConf( + ImmutableMap.of(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), "false"), + () -> { + sql( + "UPDATE %s SET id = -1 WHERE id <= (SELECT min(value) FROM updated_id)", + commitTarget()); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(2, "hardware"), row(null, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); + }); + } + + @TestTemplate + public void testUpdateThatRequiresGroupingBeforeWrite() { + createAndInitTable("id INT, dep STRING"); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + append( + tableName, + "{ \"id\": 0, \"dep\": \"hr\" }\n" + + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + + append( + commitTarget(), + "{ \"id\": 0, \"dep\": \"ops\" }\n" + + "{ \"id\": 1, \"dep\": \"ops\" }\n" + + "{ \"id\": 2, \"dep\": \"ops\" }"); + + append( + commitTarget(), + "{ \"id\": 0, \"dep\": \"hr\" }\n" + + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hr\" }"); + + append( + commitTarget(), + "{ \"id\": 0, \"dep\": \"ops\" }\n" + + "{ \"id\": 1, \"dep\": \"ops\" }\n" + + "{ \"id\": 2, \"dep\": \"ops\" }"); + + createOrReplaceView("updated_id", Arrays.asList(1, 100), Encoders.INT()); + + String originalNumOfShufflePartitions = spark.conf().get("spark.sql.shuffle.partitions"); + try { + // set the num of shuffle partitions to 1 to ensure we have only 1 writing task + spark.conf().set("spark.sql.shuffle.partitions", "1"); + + sql("UPDATE %s t SET id = -1 WHERE id IN (SELECT * FROM updated_id)", commitTarget()); + assertThat(spark.table(commitTarget()).count()) + .as("Should have expected num of rows") + .isEqualTo(12L); + } finally { + spark.conf().set("spark.sql.shuffle.partitions", originalNumOfShufflePartitions); + } + } + + @TestTemplate + public void testUpdateWithVectorization() { + createAndInitTable("id INT, dep STRING"); + + append( + tableName, + "{ \"id\": 0, \"dep\": \"hr\" }\n" + + "{ \"id\": 1, \"dep\": \"hr\" }\n" + + "{ \"id\": 2, \"dep\": \"hr\" }"); + createBranchIfNeeded(); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.VECTORIZATION_ENABLED, "true"), + () -> { + sql("UPDATE %s t SET id = -1", commitTarget()); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(-1, "hr"), row(-1, "hr")), + sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); + }); + } + + @TestTemplate + public void testUpdateModifyPartitionSourceField() throws NoSuchTableException { + createAndInitTable("id INT, dep STRING, country STRING"); + + sql("ALTER TABLE %s ADD PARTITION FIELD bucket(4, id)", tableName); + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + List ids = Lists.newArrayListWithCapacity(100); + for (int id = 1; id <= 100; id++) { + ids.add(id); + } + + Dataset df1 = + spark + .createDataset(ids, Encoders.INT()) + .withColumnRenamed("value", "id") + .withColumn("dep", lit("hr")) + .withColumn("country", lit("usa")); + df1.coalesce(1).writeTo(tableName).append(); + createBranchIfNeeded(); + + Dataset df2 = + spark + .createDataset(ids, Encoders.INT()) + .withColumnRenamed("value", "id") + .withColumn("dep", lit("software")) + .withColumn("country", lit("usa")); + df2.coalesce(1).writeTo(commitTarget()).append(); + + Dataset df3 = + spark + .createDataset(ids, Encoders.INT()) + .withColumnRenamed("value", "id") + .withColumn("dep", lit("hardware")) + .withColumn("country", lit("usa")); + df3.coalesce(1).writeTo(commitTarget()).append(); + + sql( + "UPDATE %s SET id = -1 WHERE id IN (10, 11, 12, 13, 14, 15, 16, 17, 18, 19)", + commitTarget()); + assertThat(scalarSql("SELECT count(*) FROM %s WHERE id = -1", selectTarget())).isEqualTo(30L); + } + + @TestTemplate + public void testUpdateWithStaticPredicatePushdown() { + createAndInitTable("id INT, dep STRING"); + + sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); + + // add a data file to the 'software' partition + append(tableName, "{ \"id\": 1, \"dep\": \"software\" }"); + createBranchIfNeeded(); + + // add a data file to the 'hr' partition + append(commitTarget(), "{ \"id\": 1, \"dep\": \"hr\" }"); + + Table table = validationCatalog.loadTable(tableIdent); + + Snapshot snapshot = SnapshotUtil.latestSnapshot(table, branch); + String dataFilesCount = snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP); + assertThat(dataFilesCount).as("Must have 2 files before UPDATE").isEqualTo("2"); + + // remove the data file from the 'hr' partition to ensure it is not scanned + DataFile dataFile = Iterables.getOnlyElement(snapshot.addedDataFiles(table.io())); + table.io().deleteFile(dataFile.location()); + + // disable dynamic pruning and rely only on static predicate pushdown + withSQLConf( + ImmutableMap.of( + SQLConf.DYNAMIC_PARTITION_PRUNING_ENABLED().key(), "false", + SQLConf.RUNTIME_ROW_LEVEL_OPERATION_GROUP_FILTER_ENABLED().key(), "false"), + () -> { + sql("UPDATE %s SET id = -1 WHERE dep IN ('software') AND id == 1", commitTarget()); + }); + } + + @TestTemplate + public void testUpdateWithInvalidUpdates() { + createAndInitTable( + "id INT, a ARRAY>, m MAP", + "{ \"id\": 0, \"a\": null, \"m\": null }"); + + assertThatThrownBy(() -> sql("UPDATE %s SET a.c1 = 1", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updating nested fields is only supported for StructType"); + + assertThatThrownBy(() -> sql("UPDATE %s SET m.key = 'new_key'", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updating nested fields is only supported for StructType"); + } + + @TestTemplate + public void testUpdateWithConflictingAssignments() { + createAndInitTable( + "id INT, c STRUCT>", "{ \"id\": 0, \"s\": null }"); + + assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = 1, t.c.n1 = 2, t.id = 2", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Multiple assignments for 'id'"); + + assertThatThrownBy( + () -> sql("UPDATE %s t SET t.c.n1 = 1, t.id = 2, t.c.n1 = 2", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Multiple assignments for 'c.n1"); + + assertThatThrownBy( + () -> + sql( + "UPDATE %s SET c.n1 = 1, c = named_struct('n1', 1, 'n2', named_struct('dn1', 1, 'dn2', 2))", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Conflicting assignments for 'c'"); + } + + @TestTemplate + public void testUpdateWithInvalidAssignmentsAnsi() { + createAndInitTable( + "id INT NOT NULL, s STRUCT> NOT NULL", + "{ \"id\": 0, \"s\": { \"n1\": 1, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); + + withSQLConf( + ImmutableMap.of("spark.sql.storeAssignmentPolicy", "ansi"), + () -> { + assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = NULL", commitTarget())) + .isInstanceOf(SparkRuntimeException.class) + .hasMessageContaining( + "[NOT_NULL_ASSERT_VIOLATION] NULL value appeared in non-nullable field"); + + assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = NULL", commitTarget())) + .isInstanceOf(SparkRuntimeException.class) + .hasMessageContaining( + "[NOT_NULL_ASSERT_VIOLATION] NULL value appeared in non-nullable field"); + + assertThatThrownBy( + () -> sql("UPDATE %s t SET t.s = named_struct('n1', 1)", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot find data for the output column `s`.`n2`"); + + assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = 'str'", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot safely cast"); + + assertThatThrownBy( + () -> + sql( + "UPDATE %s t SET t.s.n2 = named_struct('dn3', 1, 'dn1', 2)", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot find data for the output column `s`.`n2`.`dn2`"); + }); + } + + @TestTemplate + public void testUpdateWithInvalidAssignmentsStrict() { + createAndInitTable( + "id INT NOT NULL, s STRUCT> NOT NULL", + "{ \"id\": 0, \"s\": { \"n1\": 1, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); + + withSQLConf( + ImmutableMap.of("spark.sql.storeAssignmentPolicy", "strict"), + () -> { + assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = NULL", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot safely cast `id` \"VOID\" to \"INT\""); + + assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = NULL", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot safely cast `s`.`n1` \"VOID\" to \"INT\""); + + assertThatThrownBy( + () -> sql("UPDATE %s t SET t.s = named_struct('n1', 1)", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot find data for the output column"); + + assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = 'str'", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot safely cast"); + + assertThatThrownBy( + () -> + sql( + "UPDATE %s t SET t.s.n2 = named_struct('dn3', 1, 'dn1', 2)", + commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot find data for the output column"); + }); + } + + @TestTemplate + public void testUpdateWithNonDeterministicCondition() { + createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"hr\" }"); + + assertThatThrownBy( + () -> sql("UPDATE %s SET id = -1 WHERE id = 1 AND rand() > 0.5", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The operator expects a deterministic expression"); + } + + @TestTemplate + public void testUpdateOnNonIcebergTableNotSupported() { + createOrReplaceView("testtable", "{ \"c1\": -100, \"c2\": -200 }"); + + assertThatThrownBy(() -> sql("UPDATE %s SET c1 = -1 WHERE c2 = 1", "testtable")) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("UPDATE TABLE is not supported temporarily."); + } + + @TestTemplate + public void testUpdateToWAPBranch() { + assumeThat(branch).as("WAP branch only works for table identifier without branch").isNull(); + + createAndInitTable( + "id INT, dep STRING", "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"a\" }"); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' = 'true')", + tableName, TableProperties.WRITE_AUDIT_PUBLISH_ENABLED); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), + () -> { + sql("UPDATE %s SET dep='hr' WHERE dep='a'", tableName); + assertThat(sql("SELECT * FROM %s WHERE dep='hr'", tableName)) + .as("Should have expected num of rows when reading table") + .hasSize(2); + assertThat(sql("SELECT * FROM %s.branch_wap WHERE dep='hr'", tableName)) + .as("Should have expected num of rows when reading WAP branch") + .hasSize(2); + assertThat(sql("SELECT * FROM %s.branch_main WHERE dep='hr'", tableName)) + .as("Should not modify main branch") + .hasSize(1); + }); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), + () -> { + sql("UPDATE %s SET dep='b' WHERE dep='hr'", tableName); + assertThat(sql("SELECT * FROM %s WHERE dep='b'", tableName)) + .as("Should have expected num of rows when reading table with multiple writes") + .hasSize(2); + assertThat(sql("SELECT * FROM %s.branch_wap WHERE dep='b'", tableName)) + .as("Should have expected num of rows when reading WAP branch with multiple writes") + .hasSize(2); + assertThat(sql("SELECT * FROM %s.branch_main WHERE dep='b'", tableName)) + .as("Should not modify main branch with multiple writes") + .hasSize(0); + }); + } + + @TestTemplate + public void testUpdateToWapBranchWithTableBranchIdentifier() { + assumeThat(branch).as("Test must have branch name part in table identifier").isNotNull(); + + createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"hr\" }"); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' = 'true')", + tableName, TableProperties.WRITE_AUDIT_PUBLISH_ENABLED); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), + () -> + assertThatThrownBy(() -> sql("UPDATE %s SET dep='hr' WHERE dep='a'", commitTarget())) + .isInstanceOf(ValidationException.class) + .hasMessage( + String.format( + "Cannot write to both branch and WAP branch, but got branch [%s] and WAP branch [wap]", + branch))); + } + + private RowLevelOperationMode mode(Table table) { + String modeName = table.properties().getOrDefault(UPDATE_MODE, UPDATE_MODE_DEFAULT); + return RowLevelOperationMode.fromName(modeName); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java new file mode 100644 index 000000000000..2406a909049e --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java @@ -0,0 +1,2121 @@ +/* + * 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.extensions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatNoException; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.nio.file.Paths; +import java.util.List; +import java.util.Locale; +import java.util.Random; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.IcebergBuild; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.catalog.ViewCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.view.ImmutableSQLViewRepresentation; +import org.apache.iceberg.view.SQLViewRepresentation; +import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewHistoryEntry; +import org.apache.iceberg.view.ViewProperties; +import org.apache.iceberg.view.ViewUtil; +import org.apache.iceberg.view.ViewVersion; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.catalog.SessionCatalog; +import org.assertj.core.api.InstanceOfAssertFactories; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestViews extends ExtensionsTestBase { + private static final Namespace NAMESPACE = Namespace.of("default"); + private static final String SPARK_CATALOG = "spark_catalog"; + private final String tableName = "table"; + + @BeforeEach + @Override + public void before() { + super.before(); + spark.conf().set("spark.sql.defaultCatalog", catalogName); + sql("USE %s", catalogName); + sql("CREATE NAMESPACE IF NOT EXISTS %s", NAMESPACE); + sql( + "CREATE TABLE IF NOT EXISTS %s.%s (id INT, data STRING)%s", + NAMESPACE, tableName, catalogName.equals(SPARK_CATALOG) ? " USING iceberg" : ""); + sql("USE %s.%s", catalogName, NAMESPACE); + } + + @AfterEach + public void removeTable() { + sql("USE %s", catalogName); + sql("DROP TABLE IF EXISTS %s.%s", NAMESPACE, tableName); + + // reset spark session catalog + spark.sessionState().catalogManager().reset(); + spark.conf().unset("spark.sql.catalog.spark_catalog"); + } + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.SPARK_WITH_VIEWS.catalogName(), + SparkCatalogConfig.SPARK_WITH_VIEWS.implementation(), + SparkCatalogConfig.SPARK_WITH_VIEWS.properties() + }, + { + SparkCatalogConfig.SPARK_SESSION_WITH_VIEWS.catalogName(), + SparkCatalogConfig.SPARK_SESSION_WITH_VIEWS.implementation(), + ImmutableMap.builder() + .putAll(SparkCatalogConfig.SPARK_SESSION_WITH_VIEWS.properties()) + .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) + .build() + } + }; + } + + @TestTemplate + public void readFromView() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("simpleView"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + // use non-existing column name to make sure only the SQL definition for spark is loaded + .withQuery("trino", String.format("SELECT non_existing FROM %s", tableName)) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + List expected = + IntStream.rangeClosed(1, 10).mapToObj(this::row).collect(Collectors.toList()); + + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(10) + .containsExactlyInAnyOrderElementsOf(expected); + } + + @TestTemplate + public void readFromTrinoView() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("trinoView"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("trino", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + List expected = + IntStream.rangeClosed(1, 10).mapToObj(this::row).collect(Collectors.toList()); + + // there's no explicit view defined for spark, so it will fall back to the defined trino view + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(10) + .containsExactlyInAnyOrderElementsOf(expected); + } + + @TestTemplate + public void readFromMultipleViews() throws NoSuchTableException { + insertRows(6); + String viewName = viewName("firstView"); + String secondView = viewName("secondView"); + String viewSQL = String.format("SELECT id FROM %s WHERE id <= 3", tableName); + String secondViewSQL = String.format("SELECT id FROM %s WHERE id > 3", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", viewSQL) + .withDefaultNamespace(NAMESPACE) + .withSchema(schema(viewSQL)) + .create(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, secondView)) + .withQuery("spark", secondViewSQL) + .withDefaultNamespace(NAMESPACE) + .withSchema(schema(secondViewSQL)) + .create(); + + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(1), row(2), row(3)); + + assertThat(sql("SELECT * FROM %s", secondView)) + .hasSize(3) + .containsExactlyInAnyOrder(row(4), row(5), row(6)); + } + + @TestTemplate + public void readFromViewUsingNonExistingTable() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("viewWithNonExistingTable"); + + ViewCatalog viewCatalog = viewCatalog(); + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.LongType.get())); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", "SELECT id FROM non_existing") + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema) + .create(); + + assertThatThrownBy(() -> sql("SELECT * FROM %s", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "The table or view `%s`.`%s`.`non_existing` cannot be found", catalogName, NAMESPACE); + } + + @TestTemplate + public void readFromViewUsingNonExistingTableColumn() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("viewWithNonExistingColumn"); + + ViewCatalog viewCatalog = viewCatalog(); + Schema schema = new Schema(Types.NestedField.required(1, "non_existing", Types.LongType.get())); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", String.format("SELECT non_existing FROM %s", tableName)) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema) + .create(); + + assertThatThrownBy(() -> sql("SELECT * FROM %s", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "A column, variable, or function parameter with name `non_existing` cannot be resolved"); + } + + @TestTemplate + public void readFromViewUsingInvalidSQL() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("viewWithInvalidSQL"); + + ViewCatalog viewCatalog = viewCatalog(); + Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", "invalid SQL") + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema) + .create(); + + assertThatThrownBy(() -> sql("SELECT * FROM %s", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Invalid view name: %s", viewName); + } + + @TestTemplate + public void readFromViewWithStaleSchema() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("staleView"); + String sql = String.format("SELECT id, data FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + // drop a column the view depends on + // note that this tests `data` because it has an invalid ordinal + sql("ALTER TABLE %s DROP COLUMN data", tableName); + + // reading from the view should now fail + assertThatThrownBy(() -> sql("SELECT * FROM %s", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "A column, variable, or function parameter with name `data` cannot be resolved"); + } + + @TestTemplate + public void readFromViewHiddenByTempView() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("viewHiddenByTempView"); + + ViewCatalog viewCatalog = viewCatalog(); + Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); + + sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", viewName, tableName); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", String.format("SELECT id FROM %s WHERE id > 5", tableName)) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema) + .create(); + + List expected = + IntStream.rangeClosed(1, 5).mapToObj(this::row).collect(Collectors.toList()); + + // returns the results from the TEMP VIEW + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(5) + .containsExactlyInAnyOrderElementsOf(expected); + } + + @TestTemplate + public void readFromViewWithGlobalTempView() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("viewWithGlobalTempView"); + String sql = String.format("SELECT id FROM %s WHERE id > 5", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + sql("CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", viewName, tableName); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + // GLOBAL TEMP VIEWS are stored in a global_temp namespace + assertThat(sql("SELECT * FROM global_temp.%s", viewName)) + .hasSize(5) + .containsExactlyInAnyOrderElementsOf( + IntStream.rangeClosed(1, 5).mapToObj(this::row).collect(Collectors.toList())); + + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(5) + .containsExactlyInAnyOrderElementsOf( + IntStream.rangeClosed(6, 10).mapToObj(this::row).collect(Collectors.toList())); + } + + @TestTemplate + public void readFromViewReferencingAnotherView() throws NoSuchTableException { + insertRows(10); + String firstView = viewName("viewBeingReferencedInAnotherView"); + String viewReferencingOtherView = viewName("viewReferencingOtherView"); + String firstSQL = String.format("SELECT id FROM %s WHERE id <= 5", tableName); + String secondSQL = String.format("SELECT id FROM %s WHERE id > 4", firstView); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, firstView)) + .withQuery("spark", firstSQL) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(firstSQL)) + .create(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewReferencingOtherView)) + .withQuery("spark", secondSQL) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(secondSQL)) + .create(); + + assertThat(sql("SELECT * FROM %s", viewReferencingOtherView)) + .hasSize(1) + .containsExactly(row(5)); + } + + @TestTemplate + public void readFromViewReferencingTempView() throws NoSuchTableException { + insertRows(10); + String tempView = viewName("tempViewBeingReferencedInAnotherView"); + String viewReferencingTempView = viewName("viewReferencingTempView"); + String sql = String.format("SELECT id FROM %s", tempView); + + ViewCatalog viewCatalog = viewCatalog(); + + sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", tempView, tableName); + + // it wouldn't be possible to reference a TEMP VIEW if the view had been created via SQL, + // but this can't be prevented when using the API directly + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewReferencingTempView)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + List expected = + IntStream.rangeClosed(1, 5).mapToObj(this::row).collect(Collectors.toList()); + + assertThat(sql("SELECT * FROM %s", tempView)) + .hasSize(5) + .containsExactlyInAnyOrderElementsOf(expected); + + // reading from a view that references a TEMP VIEW shouldn't be possible + assertThatThrownBy(() -> sql("SELECT * FROM %s", viewReferencingTempView)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The table or view") + .hasMessageContaining(tempView) + .hasMessageContaining("cannot be found"); + } + + @TestTemplate + public void readFromViewReferencingAnotherViewHiddenByTempView() throws NoSuchTableException { + insertRows(10); + String innerViewName = viewName("inner_view"); + String outerViewName = viewName("outer_view"); + String innerViewSQL = String.format("SELECT * FROM %s WHERE id > 5", tableName); + String outerViewSQL = String.format("SELECT id FROM %s", innerViewName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, innerViewName)) + .withQuery("spark", innerViewSQL) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(innerViewSQL)) + .create(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, outerViewName)) + .withQuery("spark", outerViewSQL) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(outerViewSQL)) + .create(); + + // create a temporary view that conflicts with the inner view to verify the inner name is + // resolved using the catalog and namespace defaults from the outer view + sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", innerViewName, tableName); + + // ensure that the inner view resolution uses the view namespace and catalog + sql("USE spark_catalog"); + + List tempViewRows = + IntStream.rangeClosed(1, 5).mapToObj(this::row).collect(Collectors.toList()); + + assertThat(sql("SELECT * FROM %s", innerViewName)) + .hasSize(5) + .containsExactlyInAnyOrderElementsOf(tempViewRows); + + List expectedViewRows = + IntStream.rangeClosed(6, 10).mapToObj(this::row).collect(Collectors.toList()); + + assertThat(sql("SELECT * FROM %s.%s.%s", catalogName, NAMESPACE, outerViewName)) + .hasSize(5) + .containsExactlyInAnyOrderElementsOf(expectedViewRows); + } + + @TestTemplate + public void readFromViewReferencingGlobalTempView() throws NoSuchTableException { + insertRows(10); + String globalTempView = viewName("globalTempViewBeingReferenced"); + String viewReferencingTempView = viewName("viewReferencingGlobalTempView"); + + ViewCatalog viewCatalog = viewCatalog(); + Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); + + sql( + "CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", + globalTempView, tableName); + + // it wouldn't be possible to reference a GLOBAL TEMP VIEW if the view had been created via SQL, + // but this can't be prevented when using the API directly + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewReferencingTempView)) + .withQuery("spark", String.format("SELECT id FROM global_temp.%s", globalTempView)) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema) + .create(); + + List expected = + IntStream.rangeClosed(1, 5).mapToObj(this::row).collect(Collectors.toList()); + + assertThat(sql("SELECT * FROM global_temp.%s", globalTempView)) + .hasSize(5) + .containsExactlyInAnyOrderElementsOf(expected); + + // reading from a view that references a GLOBAL TEMP VIEW shouldn't be possible + assertThatThrownBy(() -> sql("SELECT * FROM %s", viewReferencingTempView)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The table or view") + .hasMessageContaining(globalTempView) + .hasMessageContaining("cannot be found"); + } + + @TestTemplate + public void readFromViewReferencingTempFunction() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("viewReferencingTempFunction"); + String functionName = viewName("test_avg"); + String sql = String.format("SELECT %s(id) FROM %s", functionName, tableName); + sql( + "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", + functionName); + + ViewCatalog viewCatalog = viewCatalog(); + Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); + + // it wouldn't be possible to reference a TEMP FUNCTION if the view had been created via SQL, + // but this can't be prevented when using the API directly + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema) + .create(); + + assertThat(sql(sql)).hasSize(1).containsExactly(row(5.5)); + + String expectedErrorMsg = + String.format("The routine %s.%s cannot be found", NAMESPACE, functionName); + if (SPARK_CATALOG.equals(catalogName)) { + // spark session catalog tries to load a V1 function and has a different error msg + expectedErrorMsg = + String.format( + "[ROUTINE_NOT_FOUND] The routine `%s`.`%s` cannot be found", NAMESPACE, functionName); + } + + // reading from a view that references a TEMP FUNCTION shouldn't be possible + assertThatThrownBy(() -> sql("SELECT * FROM %s", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining(expectedErrorMsg); + } + + @TestTemplate + public void readFromViewWithCTE() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("viewWithCTE"); + String sql = + String.format( + "WITH max_by_data AS (SELECT max(id) as max FROM %s) " + + "SELECT max, count(1) AS count FROM max_by_data GROUP BY max", + tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThat(sql("SELECT * FROM %s", viewName)).hasSize(1).containsExactly(row(10, 1L)); + } + + @TestTemplate + public void readFromViewWithGroupByOrdinal() throws NoSuchTableException { + insertRows(3); + insertRows(2); + String viewName = viewName("viewWithGroupByOrdinal"); + String sql = String.format("SELECT id, count(1) FROM %s GROUP BY 1", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(1, 2L), row(2, 2L), row(3, 1L)); + } + + @TestTemplate + public void createViewWithGroupByOrdinal() throws NoSuchTableException { + insertRows(3); + insertRows(2); + String viewName = viewName("createViewWithGroupByOrdinal"); + sql("CREATE VIEW %s AS SELECT id, count(1) FROM %s GROUP BY 1", viewName, tableName); + + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(1, 2L), row(2, 2L), row(3, 1L)); + } + + @TestTemplate + public void rewriteFunctionIdentifier() { + assumeThat(catalogName) + .as("system namespace doesn't exist in SparkSessionCatalog") + .isNotEqualTo(SPARK_CATALOG); + String viewName = viewName("rewriteFunctionIdentifier"); + String sql = "SELECT iceberg_version() AS version"; + + assertThatThrownBy(() -> sql(sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot resolve routine") + .hasMessageContaining("iceberg_version"); + + ViewCatalog viewCatalog = viewCatalog(); + Schema schema = new Schema(Types.NestedField.required(1, "version", Types.StringType.get())); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(Namespace.of("system")) + .withDefaultCatalog(catalogName) + .withSchema(schema) + .create(); + + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(1) + .containsExactly(row(IcebergBuild.version())); + } + + @TestTemplate + public void builtinFunctionIdentifierNotRewritten() { + String viewName = viewName("builtinFunctionIdentifierNotRewritten"); + String sql = "SELECT trim(' abc ') AS result"; + + ViewCatalog viewCatalog = viewCatalog(); + Schema schema = new Schema(Types.NestedField.required(1, "result", Types.StringType.get())); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(Namespace.of("system")) + .withDefaultCatalog(catalogName) + .withSchema(schema) + .create(); + + assertThat(sql("SELECT * FROM %s", viewName)).hasSize(1).containsExactly(row("abc")); + } + + @TestTemplate + public void rewriteFunctionIdentifierWithNamespace() { + assumeThat(catalogName) + .as("system namespace doesn't exist in SparkSessionCatalog") + .isNotEqualTo(SPARK_CATALOG); + String viewName = viewName("rewriteFunctionIdentifierWithNamespace"); + String sql = "SELECT system.bucket(100, 'a') AS bucket_result, 'a' AS value"; + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(Namespace.of("system")) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + sql("USE spark_catalog"); + + assertThatThrownBy(() -> sql(sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot resolve routine `system`.`bucket`"); + + assertThat(sql("SELECT * FROM %s.%s.%s", catalogName, NAMESPACE, viewName)) + .hasSize(1) + .containsExactly(row(50, "a")); + } + + @TestTemplate + public void fullFunctionIdentifier() { + assumeThat(catalogName) + .as("system namespace doesn't exist in SparkSessionCatalog") + .isNotEqualTo(SPARK_CATALOG); + String viewName = viewName("fullFunctionIdentifier"); + String sql = + String.format( + "SELECT %s.system.bucket(100, 'a') AS bucket_result, 'a' AS value", catalogName); + + sql("USE spark_catalog"); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(Namespace.of("system")) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThat(sql("SELECT * FROM %s.%s.%s", catalogName, NAMESPACE, viewName)) + .hasSize(1) + .containsExactly(row(50, "a")); + } + + @TestTemplate + public void fullFunctionIdentifierNotRewrittenLoadFailure() { + String viewName = viewName("fullFunctionIdentifierNotRewrittenLoadFailure"); + String sql = "SELECT spark_catalog.system.bucket(100, 'a') AS bucket_result, 'a' AS value"; + + // avoid namespace failures + sql("USE spark_catalog"); + sql("CREATE NAMESPACE IF NOT EXISTS system"); + sql("USE %s", catalogName); + + Schema schema = + new Schema( + Types.NestedField.required(1, "bucket_result", Types.IntegerType.get()), + Types.NestedField.required(2, "value", Types.StringType.get())); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(Namespace.of("system")) + .withDefaultCatalog(catalogName) + .withSchema(schema) + .create(); + + // verify the v1 error message + assertThatThrownBy(() -> sql("SELECT * FROM %s", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The routine `system`.`bucket` cannot be found"); + } + + private Schema schema(String sql) { + return SparkSchemaUtil.convert(spark.sql(sql).schema()); + } + + private ViewCatalog viewCatalog() { + Catalog icebergCatalog = Spark3Util.loadIcebergCatalog(spark, catalogName); + assertThat(icebergCatalog).isInstanceOf(ViewCatalog.class); + return (ViewCatalog) icebergCatalog; + } + + private Catalog tableCatalog() { + return Spark3Util.loadIcebergCatalog(spark, catalogName); + } + + @TestTemplate + public void renameView() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("originalView"); + String renamedView = viewName("renamedView"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + sql("ALTER VIEW %s RENAME TO %s", viewName, renamedView); + + List expected = + IntStream.rangeClosed(1, 10).mapToObj(this::row).collect(Collectors.toList()); + assertThat(sql("SELECT * FROM %s", renamedView)) + .hasSize(10) + .containsExactlyInAnyOrderElementsOf(expected); + } + + @TestTemplate + public void renameViewHiddenByTempView() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("originalView"); + String renamedView = viewName("renamedView"); + String sql = String.format("SELECT id FROM %s WHERE id > 5", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", viewName, tableName); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + // renames the TEMP VIEW + sql("ALTER VIEW %s RENAME TO %s", viewName, renamedView); + assertThat(sql("SELECT * FROM %s", renamedView)) + .hasSize(5) + .containsExactlyInAnyOrderElementsOf( + IntStream.rangeClosed(1, 5).mapToObj(this::row).collect(Collectors.toList())); + + // original view still exists with its name + assertThat(viewCatalog.viewExists(TableIdentifier.of(NAMESPACE, viewName))).isTrue(); + assertThat(viewCatalog.viewExists(TableIdentifier.of(NAMESPACE, renamedView))).isFalse(); + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(5) + .containsExactlyInAnyOrderElementsOf( + IntStream.rangeClosed(6, 10).mapToObj(this::row).collect(Collectors.toList())); + + // will rename the Iceberg view + sql("ALTER VIEW %s RENAME TO %s", viewName, renamedView); + assertThat(viewCatalog.viewExists(TableIdentifier.of(NAMESPACE, renamedView))).isTrue(); + } + + @TestTemplate + public void renameViewToDifferentTargetCatalog() { + String viewName = viewName("originalView"); + String renamedView = viewName("renamedView"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + String targetCatalog = + catalogName.equals(SPARK_CATALOG) + ? SparkCatalogConfig.SPARK_WITH_VIEWS.catalogName() + : SPARK_CATALOG; + + assertThatThrownBy( + () -> + sql( + "ALTER VIEW %s RENAME TO %s.%s.%s", + viewName, targetCatalog, NAMESPACE, renamedView)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Cannot move view between catalogs: from=%s and to=%s", catalogName, targetCatalog); + } + + @TestTemplate + public void renameNonExistingView() { + assertThatThrownBy(() -> sql("ALTER VIEW non_existing RENAME TO target")) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The table or view `non_existing` cannot be found"); + } + + @TestTemplate + public void renameViewTargetAlreadyExistsAsView() { + String viewName = viewName("renameViewSource"); + String target = viewName("renameViewTarget"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, target)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThatThrownBy(() -> sql("ALTER VIEW %s RENAME TO %s", viewName, target)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot create view default.%s because it already exists", target); + } + + @TestTemplate + public void renameViewTargetAlreadyExistsAsTable() { + String viewName = viewName("renameViewSource"); + String target = viewName("renameViewTarget"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + sql( + "CREATE TABLE %s.%s.%s (id INT, data STRING)%s", + catalogName, NAMESPACE, target, catalogName.equals(SPARK_CATALOG) ? " USING iceberg" : ""); + assertThatThrownBy(() -> sql("ALTER VIEW %s RENAME TO %s", viewName, target)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot create view default.%s because it already exists", target); + } + + @TestTemplate + public void dropView() { + String viewName = viewName("viewToBeDropped"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + TableIdentifier identifier = TableIdentifier.of(NAMESPACE, viewName); + viewCatalog + .buildView(identifier) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThat(viewCatalog.viewExists(identifier)).isTrue(); + + sql("DROP VIEW %s", viewName); + assertThat(viewCatalog.viewExists(identifier)).isFalse(); + } + + @TestTemplate + public void dropNonExistingView() { + assertThatThrownBy(() -> sql("DROP VIEW non_existing")) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The view %s.%s cannot be found", NAMESPACE, "non_existing"); + } + + @TestTemplate + public void dropViewIfExists() { + String viewName = viewName("viewToBeDropped"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + TableIdentifier identifier = TableIdentifier.of(NAMESPACE, viewName); + viewCatalog + .buildView(identifier) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThat(viewCatalog.viewExists(identifier)).isTrue(); + + sql("DROP VIEW IF EXISTS %s", viewName); + assertThat(viewCatalog.viewExists(identifier)).isFalse(); + + assertThatNoException().isThrownBy(() -> sql("DROP VIEW IF EXISTS %s", viewName)); + } + + /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ + @TestTemplate + public void dropGlobalTempView() { + String globalTempView = viewName("globalViewToBeDropped"); + sql("CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id FROM %s", globalTempView, tableName); + assertThat(v1SessionCatalog().getGlobalTempView(globalTempView).isDefined()).isTrue(); + + sql("DROP VIEW global_temp.%s", globalTempView); + assertThat(v1SessionCatalog().getGlobalTempView(globalTempView).isDefined()).isFalse(); + } + + /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ + @TestTemplate + public void dropTempView() { + String tempView = viewName("tempViewToBeDropped"); + sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s", tempView, tableName); + assertThat(v1SessionCatalog().getTempView(tempView).isDefined()).isTrue(); + + sql("DROP VIEW %s", tempView); + assertThat(v1SessionCatalog().getTempView(tempView).isDefined()).isFalse(); + } + + private SessionCatalog v1SessionCatalog() { + return spark.sessionState().catalogManager().v1SessionCatalog(); + } + + private String viewName(String viewName) { + return viewName + new Random().nextInt(1000000); + } + + @TestTemplate + public void createViewIfNotExists() { + String viewName = viewName("viewThatAlreadyExists"); + sql("CREATE VIEW %s AS SELECT id FROM %s", viewName, tableName); + + assertThatThrownBy(() -> sql("CREATE VIEW %s AS SELECT id FROM %s", viewName, tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Cannot create view %s.%s because it already exists", NAMESPACE, viewName); + + // using IF NOT EXISTS should work + assertThatNoException() + .isThrownBy( + () -> sql("CREATE VIEW IF NOT EXISTS %s AS SELECT id FROM %s", viewName, tableName)); + } + + @TestTemplate + public void createOrReplaceView() throws NoSuchTableException { + insertRows(6); + String viewName = viewName("simpleView"); + + sql("CREATE OR REPLACE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); + assertThat(sql("SELECT id FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(1), row(2), row(3)); + + sql("CREATE OR REPLACE VIEW %s AS SELECT id FROM %s WHERE id > 3", viewName, tableName); + assertThat(sql("SELECT id FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(4), row(5), row(6)); + } + + @TestTemplate + public void createViewWithInvalidSQL() { + assertThatThrownBy(() -> sql("CREATE VIEW simpleViewWithInvalidSQL AS invalid SQL")) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Syntax error"); + } + + @TestTemplate + public void createViewReferencingTempView() throws NoSuchTableException { + insertRows(10); + String tempView = viewName("temporaryViewBeingReferencedInAnotherView"); + String viewReferencingTempView = viewName("viewReferencingTemporaryView"); + + sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", tempView, tableName); + + // creating a view that references a TEMP VIEW shouldn't be possible + assertThatThrownBy( + () -> sql("CREATE VIEW %s AS SELECT id FROM %s", viewReferencingTempView, tempView)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewReferencingTempView) + .hasMessageContaining("that references temporary view:") + .hasMessageContaining(tempView); + } + + @TestTemplate + public void createViewReferencingGlobalTempView() throws NoSuchTableException { + insertRows(10); + String globalTempView = viewName("globalTemporaryViewBeingReferenced"); + String viewReferencingTempView = viewName("viewReferencingGlobalTemporaryView"); + + sql( + "CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id FROM %s WHERE id <= 5", + globalTempView, tableName); + + // creating a view that references a GLOBAL TEMP VIEW shouldn't be possible + assertThatThrownBy( + () -> + sql( + "CREATE VIEW %s AS SELECT id FROM global_temp.%s", + viewReferencingTempView, globalTempView)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewReferencingTempView) + .hasMessageContaining("that references temporary view:") + .hasMessageContaining("%s.%s", "global_temp", globalTempView); + } + + @TestTemplate + public void createViewReferencingTempFunction() { + String viewName = viewName("viewReferencingTemporaryFunction"); + String functionName = viewName("test_avg_func"); + + sql( + "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", + functionName); + + // creating a view that references a TEMP FUNCTION shouldn't be possible + assertThatThrownBy( + () -> sql("CREATE VIEW %s AS SELECT %s(id) FROM %s", viewName, functionName, tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) + .hasMessageContaining("that references temporary function:") + .hasMessageContaining(functionName); + } + + @TestTemplate + public void createViewReferencingQualifiedTempFunction() { + String viewName = viewName("viewReferencingTemporaryFunction"); + String functionName = viewName("test_avg_func_qualified"); + + sql( + "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", + functionName); + + // TEMP Function can't be referenced using catalog.schema.name + assertThatThrownBy( + () -> + sql( + "CREATE VIEW %s AS SELECT %s.%s.%s(id) FROM %s", + viewName, catalogName, NAMESPACE, functionName, tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot resolve routine") + .hasMessageContaining("`%s`.`%s`.`%s`", catalogName, NAMESPACE, functionName); + + // TEMP Function can't be referenced using schema.name + assertThatThrownBy( + () -> + sql( + "CREATE VIEW %s AS SELECT %s.%s(id) FROM %s", + viewName, NAMESPACE, functionName, tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot resolve routine") + .hasMessageContaining("`%s`.`%s`", NAMESPACE, functionName); + } + + @TestTemplate + public void createViewUsingNonExistingTable() { + assertThatThrownBy( + () -> sql("CREATE VIEW viewWithNonExistingTable AS SELECT id FROM non_existing")) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The table or view `non_existing` cannot be found"); + } + + @TestTemplate + public void createViewWithMismatchedColumnCounts() { + String viewName = viewName("viewWithMismatchedColumnCounts"); + + assertThatThrownBy( + () -> sql("CREATE VIEW %s (id, data) AS SELECT id FROM %s", viewName, tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) + .hasMessageContaining("not enough data columns") + .hasMessageContaining("View columns: id, data") + .hasMessageContaining("Data columns: id"); + + assertThatThrownBy( + () -> sql("CREATE VIEW %s (id) AS SELECT id, data FROM %s", viewName, tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) + .hasMessageContaining("too many data columns") + .hasMessageContaining("View columns: id") + .hasMessageContaining("Data columns: id, data"); + } + + @TestTemplate + public void createViewWithColumnAliases() throws NoSuchTableException { + insertRows(6); + String viewName = viewName("viewWithColumnAliases"); + + sql( + "CREATE VIEW %s (new_id COMMENT 'ID', new_data COMMENT 'DATA') AS SELECT id, data FROM %s WHERE id <= 3", + viewName, tableName); + + View view = viewCatalog().loadView(TableIdentifier.of(NAMESPACE, viewName)); + assertThat(view.properties()).containsEntry("spark.query-column-names", "id,data"); + + assertThat(view.schema().columns()).hasSize(2); + Types.NestedField first = view.schema().columns().get(0); + assertThat(first.name()).isEqualTo("new_id"); + assertThat(first.doc()).isEqualTo("ID"); + + Types.NestedField second = view.schema().columns().get(1); + assertThat(second.name()).isEqualTo("new_data"); + assertThat(second.doc()).isEqualTo("DATA"); + + assertThat(sql("SELECT new_id FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(1), row(2), row(3)); + + sql("DROP VIEW %s", viewName); + + sql( + "CREATE VIEW %s (new_data, new_id) AS SELECT data, id FROM %s WHERE id <= 3", + viewName, tableName); + + assertThat(sql("SELECT new_id FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(1), row(2), row(3)); + } + + @TestTemplate + public void createViewWithDuplicateColumnNames() { + assertThatThrownBy( + () -> + sql( + "CREATE VIEW viewWithDuplicateColumnNames (new_id, new_id) AS SELECT id, id FROM %s WHERE id <= 3", + tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The column `new_id` already exists"); + } + + @TestTemplate + public void createViewWithDuplicateQueryColumnNames() throws NoSuchTableException { + insertRows(3); + String viewName = viewName("viewWithDuplicateQueryColumnNames"); + String sql = String.format("SELECT id, id FROM %s WHERE id <= 3", tableName); + + // not specifying column aliases in the view should fail + assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The column `id` already exists"); + + sql("CREATE VIEW %s (id_one, id_two) AS %s", viewName, sql); + + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(1, 1), row(2, 2), row(3, 3)); + } + + @TestTemplate + public void createViewWithCTE() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("simpleViewWithCTE"); + String sql = + String.format( + "WITH max_by_data AS (SELECT max(id) as max FROM %s) " + + "SELECT max, count(1) AS count FROM max_by_data GROUP BY max", + tableName); + + sql("CREATE VIEW %s AS %s", viewName, sql); + + assertThat(sql("SELECT * FROM %s", viewName)).hasSize(1).containsExactly(row(10, 1L)); + } + + @TestTemplate + public void createViewWithConflictingNamesForCTEAndTempView() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("viewWithConflictingNamesForCTEAndTempView"); + String cteName = viewName("cteName"); + String sql = + String.format( + "WITH %s AS (SELECT max(id) as max FROM %s) " + + "(SELECT max, count(1) AS count FROM %s GROUP BY max)", + cteName, tableName, cteName); + + // create a CTE and a TEMP VIEW with the same name + sql("CREATE TEMPORARY VIEW %s AS SELECT * from %s", cteName, tableName); + sql("CREATE VIEW %s AS %s", viewName, sql); + + // CTE should take precedence over the TEMP VIEW when data is read + assertThat(sql("SELECT * FROM %s", viewName)).hasSize(1).containsExactly(row(10, 1L)); + } + + @TestTemplate + public void createViewWithCTEReferencingTempView() { + String viewName = viewName("viewWithCTEReferencingTempView"); + String tempViewInCTE = viewName("tempViewInCTE"); + String sql = + String.format( + "WITH max_by_data AS (SELECT max(id) as max FROM %s) " + + "SELECT max, count(1) AS count FROM max_by_data GROUP BY max", + tempViewInCTE); + + sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s WHERE ID <= 5", tempViewInCTE, tableName); + + assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) + .hasMessageContaining("that references temporary view:") + .hasMessageContaining(tempViewInCTE); + } + + @TestTemplate + public void createViewWithCTEReferencingTempFunction() { + String viewName = viewName("viewWithCTEReferencingTempFunction"); + String functionName = viewName("avg_function_in_cte"); + String sql = + String.format( + "WITH avg_data AS (SELECT %s(id) as avg FROM %s) " + + "SELECT avg, count(1) AS count FROM avg_data GROUP BY max", + functionName, tableName); + + sql( + "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", + functionName); + + assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) + .hasMessageContaining("that references temporary function:") + .hasMessageContaining(functionName); + } + + @TestTemplate + public void createViewWithNonExistingQueryColumn() { + assertThatThrownBy( + () -> + sql( + "CREATE VIEW viewWithNonExistingQueryColumn AS SELECT non_existing FROM %s WHERE id <= 3", + tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "A column, variable, or function parameter with name `non_existing` cannot be resolved"); + } + + @TestTemplate + public void createViewWithSubqueryExpressionUsingTempView() { + String viewName = viewName("viewWithSubqueryExpression"); + String tempView = viewName("simpleTempView"); + String sql = + String.format("SELECT * FROM %s WHERE id = (SELECT id FROM %s)", tableName, tempView); + + sql("CREATE TEMPORARY VIEW %s AS SELECT id from %s WHERE id = 5", tempView, tableName); + + assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) + .hasMessageContaining("that references temporary view:") + .hasMessageContaining(tempView); + } + + @TestTemplate + public void createViewWithSubqueryExpressionUsingGlobalTempView() { + String viewName = viewName("simpleViewWithSubqueryExpression"); + String globalTempView = viewName("simpleGlobalTempView"); + String sql = + String.format( + "SELECT * FROM %s WHERE id = (SELECT id FROM global_temp.%s)", + tableName, globalTempView); + + sql( + "CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id from %s WHERE id = 5", + globalTempView, tableName); + + assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) + .hasMessageContaining("that references temporary view:") + .hasMessageContaining("%s.%s", "global_temp", globalTempView); + } + + @TestTemplate + public void createViewWithSubqueryExpressionUsingTempFunction() { + String viewName = viewName("viewWithSubqueryExpression"); + String functionName = viewName("avg_function_in_subquery"); + String sql = + String.format( + "SELECT * FROM %s WHERE id < (SELECT %s(id) FROM %s)", + tableName, functionName, tableName); + + sql( + "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", + functionName); + + assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) + .hasMessageContaining("that references temporary function:") + .hasMessageContaining(functionName); + } + + @TestTemplate + public void createViewWithSubqueryExpressionInFilterThatIsRewritten() + throws NoSuchTableException { + insertRows(5); + String viewName = viewName("viewWithSubqueryExpression"); + String sql = + String.format( + "SELECT id FROM %s WHERE id = (SELECT max(id) FROM %s)", tableName, tableName); + + sql("CREATE VIEW %s AS %s", viewName, sql); + + assertThat(sql("SELECT * FROM %s", viewName)).hasSize(1).containsExactly(row(5)); + + if (!catalogName.equals(SPARK_CATALOG)) { + sql("USE spark_catalog"); + + assertThatThrownBy(() -> sql(sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The table or view `%s` cannot be found", tableName); + } + + // the underlying SQL in the View should be rewritten to have catalog & namespace + assertThat(sql("SELECT * FROM %s.%s.%s", catalogName, NAMESPACE, viewName)) + .hasSize(1) + .containsExactly(row(5)); + } + + @TestTemplate + public void createViewWithSubqueryExpressionInQueryThatIsRewritten() throws NoSuchTableException { + insertRows(3); + String viewName = viewName("viewWithSubqueryExpression"); + String sql = + String.format("SELECT (SELECT max(id) FROM %s) max_id FROM %s", tableName, tableName); + + sql("CREATE VIEW %s AS %s", viewName, sql); + + assertThat(sql("SELECT * FROM %s", viewName)) + .hasSize(3) + .containsExactly(row(3), row(3), row(3)); + + if (!catalogName.equals(SPARK_CATALOG)) { + sql("USE spark_catalog"); + + assertThatThrownBy(() -> sql(sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The table or view `%s` cannot be found", tableName); + } + + // the underlying SQL in the View should be rewritten to have catalog & namespace + assertThat(sql("SELECT * FROM %s.%s.%s", catalogName, NAMESPACE, viewName)) + .hasSize(3) + .containsExactly(row(3), row(3), row(3)); + } + + @TestTemplate + public void describeView() { + String viewName = viewName("describeView"); + + sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); + assertThat(sql("DESCRIBE %s", viewName)) + .containsExactly(row("id", "int", ""), row("data", "string", "")); + } + + @TestTemplate + public void describeExtendedView() { + String viewName = viewName("describeExtendedView"); + String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); + + sql( + "CREATE VIEW %s (new_id COMMENT 'ID', new_data COMMENT 'DATA') COMMENT 'view comment' AS %s", + viewName, sql); + String location = viewCatalog().loadView(TableIdentifier.of(NAMESPACE, viewName)).location(); + assertThat(sql("DESCRIBE EXTENDED %s", viewName)) + .contains( + row("new_id", "int", "ID"), + row("new_data", "string", "DATA"), + row("", "", ""), + row("# Detailed View Information", "", ""), + row("Comment", "view comment", ""), + row("View Catalog and Namespace", String.format("%s.%s", catalogName, NAMESPACE), ""), + row("View Query Output Columns", "[id, data]", ""), + row( + "View Properties", + String.format( + "['format-version' = '1', 'location' = '%s', 'provider' = 'iceberg']", + location), + "")); + } + + @TestTemplate + public void createAndDescribeViewInDefaultNamespace() { + String viewName = viewName("createViewInDefaultNamespace"); + String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); + + sql("CREATE VIEW %s (id, data) AS %s", viewName, sql); + TableIdentifier identifier = TableIdentifier.of(NAMESPACE, viewName); + View view = viewCatalog().loadView(identifier); + assertThat(view.currentVersion().defaultCatalog()).isNull(); + assertThat(view.name()).isEqualTo(ViewUtil.fullViewName(catalogName, identifier)); + assertThat(view.currentVersion().defaultNamespace()).isEqualTo(NAMESPACE); + + String location = viewCatalog().loadView(identifier).location(); + assertThat(sql("DESCRIBE EXTENDED %s.%s", NAMESPACE, viewName)) + .contains( + row("id", "int", ""), + row("data", "string", ""), + row("", "", ""), + row("# Detailed View Information", "", ""), + row("Comment", "", ""), + row("View Catalog and Namespace", String.format("%s.%s", catalogName, NAMESPACE), ""), + row("View Query Output Columns", "[id, data]", ""), + row( + "View Properties", + String.format( + "['format-version' = '1', 'location' = '%s', 'provider' = 'iceberg']", + location), + "")); + } + + @TestTemplate + public void createAndDescribeViewWithoutCurrentNamespace() { + String viewName = viewName("createViewWithoutCurrentNamespace"); + Namespace namespace = Namespace.of("test_namespace"); + String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); + + sql("CREATE NAMESPACE IF NOT EXISTS %s", namespace); + sql("CREATE VIEW %s.%s (id, data) AS %s", namespace, viewName, sql); + TableIdentifier identifier = TableIdentifier.of(namespace, viewName); + View view = viewCatalog().loadView(identifier); + assertThat(view.currentVersion().defaultCatalog()).isNull(); + assertThat(view.name()).isEqualTo(ViewUtil.fullViewName(catalogName, identifier)); + assertThat(view.currentVersion().defaultNamespace()).isEqualTo(NAMESPACE); + + String location = viewCatalog().loadView(identifier).location(); + assertThat(sql("DESCRIBE EXTENDED %s.%s", namespace, viewName)) + .contains( + row("id", "int", ""), + row("data", "string", ""), + row("", "", ""), + row("# Detailed View Information", "", ""), + row("Comment", "", ""), + row("View Catalog and Namespace", String.format("%s.%s", catalogName, namespace), ""), + row("View Query Output Columns", "[id, data]", ""), + row( + "View Properties", + String.format( + "['format-version' = '1', 'location' = '%s', 'provider' = 'iceberg']", + location), + "")); + } + + @TestTemplate + public void showViewProperties() { + String viewName = viewName("showViewProps"); + + sql( + "CREATE VIEW %s TBLPROPERTIES ('key1'='val1', 'key2'='val2') AS SELECT id, data FROM %s WHERE id <= 3", + viewName, tableName); + assertThat(sql("SHOW TBLPROPERTIES %s", viewName)) + .contains(row("key1", "val1"), row("key2", "val2")); + } + + @TestTemplate + public void showViewPropertiesByKey() { + String viewName = viewName("showViewPropsByKey"); + + sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); + assertThat(sql("SHOW TBLPROPERTIES %s", viewName)).contains(row("provider", "iceberg")); + + assertThat(sql("SHOW TBLPROPERTIES %s (provider)", viewName)) + .contains(row("provider", "iceberg")); + + assertThat(sql("SHOW TBLPROPERTIES %s (non.existing)", viewName)) + .contains( + row( + "non.existing", + String.format( + "View %s.%s.%s does not have property: non.existing", + catalogName, NAMESPACE, viewName))); + } + + @TestTemplate + public void showViews() throws NoSuchTableException { + insertRows(6); + String sql = String.format("SELECT * from %s", tableName); + String v1 = viewName("v1"); + String prefixV2 = viewName("prefixV2"); + String prefixV3 = viewName("prefixV3"); + String globalViewForListing = viewName("globalViewForListing"); + String tempViewForListing = viewName("tempViewForListing"); + sql("CREATE VIEW %s AS %s", v1, sql); + sql("CREATE VIEW %s AS %s", prefixV2, sql); + sql("CREATE VIEW %s AS %s", prefixV3, sql); + sql("CREATE GLOBAL TEMPORARY VIEW %s AS %s", globalViewForListing, sql); + sql("CREATE TEMPORARY VIEW %s AS %s", tempViewForListing, sql); + + // spark stores temp views case-insensitive by default + Object[] tempView = row("", tempViewForListing.toLowerCase(Locale.ROOT), true); + Object[] v1Row = row(NAMESPACE.toString(), v1, false); + Object[] v2Row = row(NAMESPACE.toString(), prefixV2, false); + Object[] v3Row = row(NAMESPACE.toString(), prefixV3, false); + assertThat(sql("SHOW VIEWS")).contains(v2Row, v3Row, v1Row, tempView); + + if (!"rest".equals(catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE))) { + // REST catalog requires a namespace + assertThat(sql("SHOW VIEWS IN %s", catalogName)) + .contains(tempView) + .doesNotContain(v1Row, v2Row, v3Row); + } + + assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, NAMESPACE)) + .contains(v2Row, v3Row, v1Row, tempView); + + assertThat(sql("SHOW VIEWS LIKE 'pref*'")) + .contains(v2Row, v3Row) + .doesNotContain(v1Row, tempView); + + assertThat(sql("SHOW VIEWS LIKE 'non-existing'")).isEmpty(); + + if (!catalogName.equals(SPARK_CATALOG)) { + sql("CREATE NAMESPACE IF NOT EXISTS spark_catalog.%s", NAMESPACE); + assertThat(sql("SHOW VIEWS IN spark_catalog.%s", NAMESPACE)) + .contains(tempView) + .doesNotContain(v1Row, v2Row, v3Row); + } + + assertThat(sql("SHOW VIEWS IN global_temp")) + .contains( + // spark stores temp views case-insensitive by default + row("global_temp", globalViewForListing.toLowerCase(Locale.ROOT), true), tempView) + .doesNotContain(v1Row, v2Row, v3Row); + + sql("USE spark_catalog"); + assertThat(sql("SHOW VIEWS")).contains(tempView); + + assertThat(sql("SHOW VIEWS IN default")).contains(tempView); + } + + @TestTemplate + public void showViewsWithCurrentNamespace() { + String namespaceOne = "show_views_ns1"; + String namespaceTwo = "show_views_ns2"; + String viewOne = viewName("viewOne"); + String viewTwo = viewName("viewTwo"); + sql("CREATE NAMESPACE IF NOT EXISTS %s", namespaceOne); + sql("CREATE NAMESPACE IF NOT EXISTS %s", namespaceTwo); + + // create one view in each namespace + sql("CREATE VIEW %s.%s AS SELECT * FROM %s.%s", namespaceOne, viewOne, NAMESPACE, tableName); + sql("CREATE VIEW %s.%s AS SELECT * FROM %s.%s", namespaceTwo, viewTwo, NAMESPACE, tableName); + + Object[] v1 = row(namespaceOne, viewOne, false); + Object[] v2 = row(namespaceTwo, viewTwo, false); + + assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, namespaceOne)) + .contains(v1) + .doesNotContain(v2); + sql("USE %s", namespaceOne); + assertThat(sql("SHOW VIEWS")).contains(v1).doesNotContain(v2); + assertThat(sql("SHOW VIEWS LIKE 'viewOne*'")).contains(v1).doesNotContain(v2); + + assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, namespaceTwo)) + .contains(v2) + .doesNotContain(v1); + sql("USE %s", namespaceTwo); + assertThat(sql("SHOW VIEWS")).contains(v2).doesNotContain(v1); + assertThat(sql("SHOW VIEWS LIKE 'viewTwo*'")).contains(v2).doesNotContain(v1); + } + + @TestTemplate + public void showCreateSimpleView() { + String viewName = viewName("showCreateSimpleView"); + String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); + + sql("CREATE VIEW %s AS %s", viewName, sql); + + String location = viewCatalog().loadView(TableIdentifier.of(NAMESPACE, viewName)).location(); + String expected = + String.format( + "CREATE VIEW %s.%s.%s (\n" + + " id,\n" + + " data)\n" + + "TBLPROPERTIES (\n" + + " 'format-version' = '1',\n" + + " 'location' = '%s',\n" + + " 'provider' = 'iceberg')\n" + + "AS\n%s\n", + catalogName, NAMESPACE, viewName, location, sql); + assertThat(sql("SHOW CREATE TABLE %s", viewName)).containsExactly(row(expected)); + } + + @TestTemplate + public void showCreateComplexView() { + String viewName = viewName("showCreateComplexView"); + String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); + + sql( + "CREATE VIEW %s (new_id COMMENT 'ID', new_data COMMENT 'DATA')" + + "COMMENT 'view comment' TBLPROPERTIES ('key1'='val1', 'key2'='val2') AS %s", + viewName, sql); + + String location = viewCatalog().loadView(TableIdentifier.of(NAMESPACE, viewName)).location(); + String expected = + String.format( + "CREATE VIEW %s.%s.%s (\n" + + " new_id COMMENT 'ID',\n" + + " new_data COMMENT 'DATA')\n" + + "COMMENT 'view comment'\n" + + "TBLPROPERTIES (\n" + + " 'format-version' = '1',\n" + + " 'key1' = 'val1',\n" + + " 'key2' = 'val2',\n" + + " 'location' = '%s',\n" + + " 'provider' = 'iceberg')\n" + + "AS\n%s\n", + catalogName, NAMESPACE, viewName, location, sql); + assertThat(sql("SHOW CREATE TABLE %s", viewName)).containsExactly(row(expected)); + } + + @TestTemplate + public void alterViewSetProperties() { + String viewName = viewName("viewWithSetProperties"); + + sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); + + ViewCatalog viewCatalog = viewCatalog(); + assertThat(viewCatalog.loadView(TableIdentifier.of(NAMESPACE, viewName)).properties()) + .doesNotContainKey("key1") + .doesNotContainKey("comment"); + + sql("ALTER VIEW %s SET TBLPROPERTIES ('key1' = 'val1', 'comment' = 'view comment')", viewName); + assertThat(viewCatalog.loadView(TableIdentifier.of(NAMESPACE, viewName)).properties()) + .containsEntry("key1", "val1") + .containsEntry("comment", "view comment"); + + sql("ALTER VIEW %s SET TBLPROPERTIES ('key1' = 'new_val1')", viewName); + assertThat(viewCatalog.loadView(TableIdentifier.of(NAMESPACE, viewName)).properties()) + .containsEntry("key1", "new_val1") + .containsEntry("comment", "view comment"); + } + + @TestTemplate + public void alterViewSetReservedProperties() { + String viewName = viewName("viewWithSetReservedProperties"); + + sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); + + assertThatThrownBy(() -> sql("ALTER VIEW %s SET TBLPROPERTIES ('provider' = 'val1')", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "The feature is not supported: provider is a reserved table property"); + + assertThatThrownBy( + () -> sql("ALTER VIEW %s SET TBLPROPERTIES ('location' = 'random_location')", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "The feature is not supported: location is a reserved table property"); + + assertThatThrownBy( + () -> sql("ALTER VIEW %s SET TBLPROPERTIES ('format-version' = '99')", viewName)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Cannot set reserved property: 'format-version'"); + + assertThatThrownBy( + () -> + sql( + "ALTER VIEW %s SET TBLPROPERTIES ('spark.query-column-names' = 'a,b,c')", + viewName)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Cannot set reserved property: 'spark.query-column-names'"); + } + + @TestTemplate + public void alterViewUnsetProperties() { + String viewName = viewName("viewWithUnsetProperties"); + sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); + + ViewCatalog viewCatalog = viewCatalog(); + assertThat(viewCatalog.loadView(TableIdentifier.of(NAMESPACE, viewName)).properties()) + .doesNotContainKey("key1") + .doesNotContainKey("comment"); + + sql("ALTER VIEW %s SET TBLPROPERTIES ('key1' = 'val1', 'comment' = 'view comment')", viewName); + assertThat(viewCatalog.loadView(TableIdentifier.of(NAMESPACE, viewName)).properties()) + .containsEntry("key1", "val1") + .containsEntry("comment", "view comment"); + + sql("ALTER VIEW %s UNSET TBLPROPERTIES ('key1')", viewName); + assertThat(viewCatalog.loadView(TableIdentifier.of(NAMESPACE, viewName)).properties()) + .doesNotContainKey("key1") + .containsEntry("comment", "view comment"); + } + + @TestTemplate + public void alterViewUnsetUnknownProperty() { + String viewName = viewName("viewWithUnsetUnknownProp"); + sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); + + assertThatThrownBy(() -> sql("ALTER VIEW %s UNSET TBLPROPERTIES ('unknown-key')", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot remove property that is not set: 'unknown-key'"); + + assertThatNoException() + .isThrownBy( + () -> sql("ALTER VIEW %s UNSET TBLPROPERTIES IF EXISTS ('unknown-key')", viewName)); + } + + @TestTemplate + public void alterViewUnsetReservedProperties() { + String viewName = viewName("viewWithUnsetReservedProperties"); + + sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); + + assertThatThrownBy(() -> sql("ALTER VIEW %s UNSET TBLPROPERTIES ('provider')", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "The feature is not supported: provider is a reserved table property"); + + assertThatThrownBy(() -> sql("ALTER VIEW %s UNSET TBLPROPERTIES ('location')", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "The feature is not supported: location is a reserved table property"); + + assertThatThrownBy(() -> sql("ALTER VIEW %s UNSET TBLPROPERTIES ('format-version')", viewName)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Cannot unset reserved property: 'format-version'"); + + // spark.query-column-names is only used internally, so it technically doesn't exist on a Spark + // VIEW + assertThatThrownBy( + () -> sql("ALTER VIEW %s UNSET TBLPROPERTIES ('spark.query-column-names')", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot remove property that is not set: 'spark.query-column-names'"); + + assertThatThrownBy( + () -> + sql( + "ALTER VIEW %s UNSET TBLPROPERTIES IF EXISTS ('spark.query-column-names')", + viewName)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Cannot unset reserved property: 'spark.query-column-names'"); + } + + @TestTemplate + public void createOrReplaceViewWithColumnAliases() throws NoSuchTableException { + insertRows(6); + String viewName = viewName("viewWithColumnAliases"); + + sql( + "CREATE VIEW %s (new_id COMMENT 'ID', new_data COMMENT 'DATA') AS SELECT id, data FROM %s WHERE id <= 3", + viewName, tableName); + + View view = viewCatalog().loadView(TableIdentifier.of(NAMESPACE, viewName)); + assertThat(view.properties()).containsEntry("spark.query-column-names", "id,data"); + + assertThat(view.schema().columns()).hasSize(2); + Types.NestedField first = view.schema().columns().get(0); + assertThat(first.name()).isEqualTo("new_id"); + assertThat(first.doc()).isEqualTo("ID"); + + Types.NestedField second = view.schema().columns().get(1); + assertThat(second.name()).isEqualTo("new_data"); + assertThat(second.doc()).isEqualTo("DATA"); + + assertThat(sql("SELECT new_id FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(1), row(2), row(3)); + + sql( + "CREATE OR REPLACE VIEW %s (data2 COMMENT 'new data', id2 COMMENT 'new ID') AS SELECT data, id FROM %s WHERE id <= 3", + viewName, tableName); + + assertThat(sql("SELECT data2, id2 FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row("2", 1), row("4", 2), row("6", 3)); + + view = viewCatalog().loadView(TableIdentifier.of(NAMESPACE, viewName)); + assertThat(view.properties()).containsEntry("spark.query-column-names", "data,id"); + + assertThat(view.schema().columns()).hasSize(2); + first = view.schema().columns().get(0); + assertThat(first.name()).isEqualTo("data2"); + assertThat(first.doc()).isEqualTo("new data"); + + second = view.schema().columns().get(1); + assertThat(second.name()).isEqualTo("id2"); + assertThat(second.doc()).isEqualTo("new ID"); + } + + @TestTemplate + public void alterViewIsNotSupported() throws NoSuchTableException { + insertRows(6); + String viewName = viewName("alteredView"); + + sql("CREATE VIEW %s AS SELECT id, data FROM %s WHERE id <= 3", viewName, tableName); + + assertThat(sql("SELECT id FROM %s", viewName)) + .hasSize(3) + .containsExactlyInAnyOrder(row(1), row(2), row(3)); + + assertThatThrownBy( + () -> sql("ALTER VIEW %s AS SELECT id FROM %s WHERE id > 3", viewName, tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "ALTER VIEW AS is not supported. Use CREATE OR REPLACE VIEW instead"); + } + + @TestTemplate + public void createOrReplaceViewKeepsViewHistory() { + String viewName = viewName("viewWithHistoryAfterReplace"); + String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); + String updatedSql = String.format("SELECT id FROM %s WHERE id > 3", tableName); + + sql( + "CREATE VIEW %s (new_id COMMENT 'some ID', new_data COMMENT 'some data') AS %s", + viewName, sql); + + View view = viewCatalog().loadView(TableIdentifier.of(NAMESPACE, viewName)); + assertThat(view.history()).hasSize(1); + assertThat(view.sqlFor("spark").sql()).isEqualTo(sql); + assertThat(view.currentVersion().versionId()).isEqualTo(1); + assertThat(view.currentVersion().schemaId()).isEqualTo(0); + assertThat(view.schemas()).hasSize(1); + assertThat(view.schema().asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(0, "new_id", Types.IntegerType.get(), "some ID"), + Types.NestedField.optional(1, "new_data", Types.StringType.get(), "some data")) + .asStruct()); + + sql("CREATE OR REPLACE VIEW %s (updated_id COMMENT 'updated ID') AS %s", viewName, updatedSql); + + view = viewCatalog().loadView(TableIdentifier.of(NAMESPACE, viewName)); + assertThat(view.history()).hasSize(2); + assertThat(view.sqlFor("spark").sql()).isEqualTo(updatedSql); + assertThat(view.currentVersion().versionId()).isEqualTo(2); + assertThat(view.currentVersion().schemaId()).isEqualTo(1); + assertThat(view.schemas()).hasSize(2); + assertThat(view.schema().asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional( + 0, "updated_id", Types.IntegerType.get(), "updated ID")) + .asStruct()); + } + + @TestTemplate + public void replacingTrinoViewShouldFail() { + String viewName = viewName("trinoView"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("trino", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot replace view due to loss of view dialects (replace.drop-dialect.allowed=false):\n" + + "Previous dialects: [trino]\n" + + "New dialects: [spark]"); + } + + @TestTemplate + public void replacingTrinoAndSparkViewShouldFail() { + String viewName = viewName("trinoAndSparkView"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("trino", sql) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot replace view due to loss of view dialects (replace.drop-dialect.allowed=false):\n" + + "Previous dialects: [trino, spark]\n" + + "New dialects: [spark]"); + } + + @TestTemplate + public void replacingViewWithDialectDropAllowed() { + String viewName = viewName("trinoView"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("trino", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + // allowing to drop the trino dialect should replace the view + sql( + "CREATE OR REPLACE VIEW %s TBLPROPERTIES ('%s'='true') AS SELECT id FROM %s", + viewName, ViewProperties.REPLACE_DROP_DIALECT_ALLOWED, tableName); + + View view = viewCatalog.loadView(TableIdentifier.of(NAMESPACE, viewName)); + assertThat(view.currentVersion().representations()) + .hasSize(1) + .first() + .asInstanceOf(InstanceOfAssertFactories.type(SQLViewRepresentation.class)) + .isEqualTo(ImmutableSQLViewRepresentation.builder().dialect("spark").sql(sql).build()); + + // trino view should show up in the view versions & history + assertThat(view.history()).hasSize(2); + assertThat(view.history()).element(0).extracting(ViewHistoryEntry::versionId).isEqualTo(1); + assertThat(view.history()).element(1).extracting(ViewHistoryEntry::versionId).isEqualTo(2); + + assertThat(view.versions()).hasSize(2); + assertThat(view.versions()).element(0).extracting(ViewVersion::versionId).isEqualTo(1); + assertThat(view.versions()).element(1).extracting(ViewVersion::versionId).isEqualTo(2); + + assertThat(Lists.newArrayList(view.versions()).get(0).representations()) + .hasSize(1) + .first() + .asInstanceOf(InstanceOfAssertFactories.type(SQLViewRepresentation.class)) + .isEqualTo(ImmutableSQLViewRepresentation.builder().dialect("trino").sql(sql).build()); + + assertThat(Lists.newArrayList(view.versions()).get(1).representations()) + .hasSize(1) + .first() + .asInstanceOf(InstanceOfAssertFactories.type(SQLViewRepresentation.class)) + .isEqualTo(ImmutableSQLViewRepresentation.builder().dialect("spark").sql(sql).build()); + } + + @TestTemplate + public void createViewWithRecursiveCycle() { + String viewOne = viewName("viewOne"); + String viewTwo = viewName("viewTwo"); + + sql("CREATE VIEW %s AS SELECT * FROM %s", viewOne, tableName); + // viewTwo points to viewOne + sql("CREATE VIEW %s AS SELECT * FROM %s", viewTwo, viewOne); + + // viewOne points to viewTwo points to viewOne, creating a recursive cycle + String view1 = String.format("%s.%s.%s", catalogName, NAMESPACE, viewOne); + String view2 = String.format("%s.%s.%s", catalogName, NAMESPACE, viewTwo); + String cycle = String.format("%s -> %s -> %s", view1, view2, view1); + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS SELECT * FROM %s", viewOne, view2)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle); + } + + @TestTemplate + public void createViewWithRecursiveCycleToV1View() { + assumeThat(catalogName).isNotEqualTo(SPARK_CATALOG); + String viewOne = viewName("view_one"); + String viewTwo = viewName("view_two"); + + sql("CREATE VIEW %s AS SELECT * FROM %s", viewOne, tableName); + // viewTwo points to viewOne + sql("USE spark_catalog"); + sql("CREATE VIEW %s AS SELECT * FROM %s.%s.%s", viewTwo, catalogName, NAMESPACE, viewOne); + + sql("USE %s", catalogName); + // viewOne points to viewTwo points to viewOne, creating a recursive cycle + String view1 = String.format("%s.%s.%s", catalogName, NAMESPACE, viewOne); + String view2 = String.format("%s.%s.%s", SPARK_CATALOG, NAMESPACE, viewTwo); + String cycle = String.format("%s -> %s -> %s", view1, view2, view1); + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS SELECT * FROM %s", viewOne, view2)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle); + } + + @TestTemplate + public void createViewWithRecursiveCycleInCTE() { + String viewOne = viewName("viewOne"); + String viewTwo = viewName("viewTwo"); + + sql("CREATE VIEW %s AS SELECT * FROM %s", viewOne, tableName); + // viewTwo points to viewOne + sql("CREATE VIEW %s AS SELECT * FROM %s", viewTwo, viewOne); + + // CTE points to viewTwo + String sql = + String.format( + "WITH max_by_data AS (SELECT max(id) as max FROM %s) " + + "SELECT max, count(1) AS count FROM max_by_data GROUP BY max", + viewTwo); + + // viewOne points to CTE, creating a recursive cycle + String view1 = String.format("%s.%s.%s", catalogName, NAMESPACE, viewOne); + String cycle = String.format("%s -> %s -> %s", view1, viewTwo, view1); + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewOne, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle); + } + + @TestTemplate + public void createViewWithRecursiveCycleInSubqueryExpression() { + String viewOne = viewName("viewOne"); + String viewTwo = viewName("viewTwo"); + + sql("CREATE VIEW %s AS SELECT * FROM %s", viewOne, tableName); + // viewTwo points to viewOne + sql("CREATE VIEW %s AS SELECT * FROM %s", viewTwo, viewOne); + + // subquery expression points to viewTwo + String sql = + String.format("SELECT * FROM %s WHERE id = (SELECT id FROM %s)", tableName, viewTwo); + + // viewOne points to subquery expression, creating a recursive cycle + String view1 = String.format("%s.%s.%s", catalogName, NAMESPACE, viewOne); + String cycle = String.format("%s -> %s -> %s", view1, viewTwo, view1); + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewOne, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle); + } + + @TestTemplate + public void createViewWithCustomMetadataLocation() { + String viewName = viewName("v"); + String customMetadataLocation = + Paths.get(temp.toUri().toString(), "custom-metadata-location").toString(); + sql( + "CREATE VIEW %s TBLPROPERTIES ('%s'='%s') AS SELECT * FROM %s", + viewName, ViewProperties.WRITE_METADATA_LOCATION, customMetadataLocation, tableName); + String location = viewCatalog().loadView(TableIdentifier.of(NAMESPACE, viewName)).location(); + + assertThat(sql("DESCRIBE EXTENDED %s", viewName)) + .contains( + row( + "View Properties", + String.format( + "['format-version' = '1', 'location' = '%s', 'provider' = 'iceberg', 'write.metadata.path' = '%s']", + location, customMetadataLocation), + "")); + } + + @TestTemplate + public void createViewWithCustomMetadataLocationWithLocation() { + String viewName = viewName("v"); + String customMetadataLocation = + Paths.get(temp.toUri().toString(), "custom-metadata-location").toString(); + sql( + "CREATE VIEW %s TBLPROPERTIES ('location'='%s') AS SELECT * FROM %s", + viewName, customMetadataLocation, tableName); + + assertThat(sql("SHOW TBLPROPERTIES %s", viewName)) + .contains(row("location", customMetadataLocation)); + + String location = viewCatalog().loadView(TableIdentifier.of(NAMESPACE, viewName)).location(); + assertThat(location).isEqualTo(customMetadataLocation); + } + + private void insertRows(int numRows) throws NoSuchTableException { + List records = Lists.newArrayListWithCapacity(numRows); + for (int i = 1; i <= numRows; i++) { + records.add(new SimpleRecord(i, Integer.toString(i * 2))); + } + + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.writeTo(tableName).append(); + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java new file mode 100644 index 000000000000..1786a7f3a3c8 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.extensions; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.nio.file.Files; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.BulkDeletionFailureException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.SupportsBulkOperations; +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.SparkCatalog; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.SparkException; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestWriteAborts extends ExtensionsTestBase { + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", + "hive", + CatalogProperties.FILE_IO_IMPL, + CustomFileIO.class.getName(), + "default-namespace", + "default") + }, + { + "testhivebulk", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", + "hive", + CatalogProperties.FILE_IO_IMPL, + CustomBulkFileIO.class.getName(), + "default-namespace", + "default") + } + }; + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testBatchAppend() throws IOException { + String dataLocation = Files.createTempDirectory(temp, "junit").toFile().toString(); + + sql( + "CREATE TABLE %s (id INT, data STRING) " + + "USING iceberg " + + "PARTITIONED BY (data)" + + "TBLPROPERTIES ('%s' '%s')", + tableName, TableProperties.WRITE_DATA_LOCATION, dataLocation); + + List records = + ImmutableList.of( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + Dataset inputDF = spark.createDataFrame(records, SimpleRecord.class); + + assertThatThrownBy( + () -> + // incoming records are not ordered by partitions so the job must fail + inputDF + .coalesce(1) + .sortWithinPartitions("id") + .writeTo(tableName) + .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") + .option(SparkWriteOptions.FANOUT_ENABLED, "false") + .append()) + .isInstanceOf(SparkException.class) + .hasMessageContaining("Encountered records that belong to already closed files"); + + assertEquals("Should be no records", sql("SELECT * FROM %s", tableName), ImmutableList.of()); + + assertEquals( + "Should be no orphan data files", + ImmutableList.of(), + sql( + "CALL %s.system.remove_orphan_files(table => '%s', older_than => CAST(%dL AS TIMESTAMP), location => '%s')", + catalogName, tableName, System.currentTimeMillis() + 5000, dataLocation)); + } + + public static class CustomFileIO implements FileIO { + + private final FileIO delegate = new HadoopFileIO(new Configuration()); + + public CustomFileIO() {} + + protected FileIO delegate() { + return delegate; + } + + @Override + public InputFile newInputFile(String path) { + return delegate.newInputFile(path); + } + + @Override + public OutputFile newOutputFile(String path) { + return delegate.newOutputFile(path); + } + + @Override + public void deleteFile(String path) { + delegate.deleteFile(path); + } + + @Override + public Map properties() { + return delegate.properties(); + } + + @Override + public void initialize(Map properties) { + delegate.initialize(properties); + } + + @Override + public void close() { + delegate.close(); + } + } + + public static class CustomBulkFileIO extends CustomFileIO implements SupportsBulkOperations { + + public CustomBulkFileIO() {} + + @Override + public void deleteFile(String path) { + throw new UnsupportedOperationException("Only bulk deletes are supported"); + } + + @Override + public void deleteFiles(Iterable paths) throws BulkDeletionFailureException { + for (String path : paths) { + delegate().deleteFile(path); + } + } + } +} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/source/PlanUtils.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/source/PlanUtils.java new file mode 100644 index 000000000000..148717e14255 --- /dev/null +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/source/PlanUtils.java @@ -0,0 +1,99 @@ +/* + * 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.Collection; +import java.util.List; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation; +import scala.PartialFunction; +import scala.collection.JavaConverters; +import scala.collection.Seq; + +public class PlanUtils { + private PlanUtils() {} + + public static List collectPushDownFilters( + LogicalPlan logicalPlan) { + return JavaConverters.asJavaCollection(logicalPlan.collectLeaves()).stream() + .flatMap( + plan -> { + if (!(plan instanceof DataSourceV2ScanRelation)) { + return Stream.empty(); + } + + DataSourceV2ScanRelation scanRelation = (DataSourceV2ScanRelation) plan; + if (!(scanRelation.scan() instanceof SparkBatchQueryScan)) { + return Stream.empty(); + } + + SparkBatchQueryScan batchQueryScan = (SparkBatchQueryScan) scanRelation.scan(); + return batchQueryScan.filterExpressions().stream(); + }) + .collect(Collectors.toList()); + } + + public static List collectSparkExpressions( + LogicalPlan logicalPlan, Predicate predicate) { + Seq> list = + logicalPlan.collect( + new PartialFunction>() { + + @Override + public List apply(LogicalPlan plan) { + return JavaConverters.asJavaCollection(plan.expressions()).stream() + .flatMap(expr -> collectSparkExpressions(expr, predicate).stream()) + .collect(Collectors.toList()); + } + + @Override + public boolean isDefinedAt(LogicalPlan plan) { + return true; + } + }); + + return JavaConverters.asJavaCollection(list).stream() + .flatMap(Collection::stream) + .collect(Collectors.toList()); + } + + private static List collectSparkExpressions( + Expression expression, Predicate predicate) { + Seq list = + expression.collect( + new PartialFunction() { + @Override + public Expression apply(Expression expr) { + return expr; + } + + @Override + public boolean isDefinedAt(Expression expr) { + return predicate.test(expr); + } + }); + + return Lists.newArrayList(JavaConverters.asJavaCollection(list)); + } +} diff --git a/spark/v4.0/spark-runtime/LICENSE b/spark/v4.0/spark-runtime/LICENSE new file mode 100644 index 000000000000..e36e0b684501 --- /dev/null +++ b/spark/v4.0/spark-runtime/LICENSE @@ -0,0 +1,530 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Avro. + +Copyright: Copyright 2010-2019 The Apache Software Foundation +Home page: https://avro.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains the Jackson JSON processor. + +Copyright: 2007-2019 Tatu Saloranta and other contributors +Home page: http://jackson.codehaus.org/ +License: http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Parquet. + +Copyright: 2014-2024 The Apache Software Foundation +Home page: https://parquet.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Thrift. + +Copyright: 2006-2017 The Apache Software Foundation. +Home page: https://thrift.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains code from Daniel Lemire's JavaFastPFOR project. + +Copyright: 2013 Daniel Lemire +Home page: https://github.com/lemire/JavaFastPFOR +License: Apache License Version 2.0 http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains fastutil. + +Copyright: 2002-2014 Sebastiano Vigna +Home page: http://fastutil.di.unimi.it/ +License: http://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache ORC. + +Copyright: 2013 and onwards The Apache Software Foundation. +Home page: https://orc.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Hive's storage API via ORC. + +Copyright: 2008-2020 The Apache Software Foundation +Home page: https://hive.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Google protobuf via ORC. + +Copyright: 2008 Google Inc. +Home page: https://developers.google.com/protocol-buffers +License: https://github.com/protocolbuffers/protobuf/blob/master/LICENSE (BSD) + +License text: + +| Copyright 2008 Google Inc. All rights reserved. +| +| Redistribution and use in source and binary forms, with or without +| modification, are permitted provided that the following conditions are +| met: +| +| * Redistributions of source code must retain the above copyright +| notice, this list of conditions and the following disclaimer. +| * Redistributions in binary form must reproduce the above +| copyright notice, this list of conditions and the following disclaimer +| in the documentation and/or other materials provided with the +| distribution. +| * Neither the name of Google Inc. nor the names of its +| contributors may be used to endorse or promote products derived from +| this software without specific prior written permission. +| +| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +| +| Code generated by the Protocol Buffer compiler is owned by the owner +| of the input file used when generating it. This code is not +| standalone and requires a support library to be linked with it. This +| support library is itself covered by the above license. + +-------------------------------------------------------------------------------- + +This binary artifact contains Airlift Aircompressor. + +Copyright: 2011-2019 Aircompressor authors. +Home page: https://github.com/airlift/aircompressor +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains code from Cloudera Kite. + +Copyright: 2013-2017 Cloudera Inc. +Home page: https://kitesdk.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains code from Presto. + +Copyright: 2016 Facebook and contributors +Home page: https://prestodb.io/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Google Guava. + +Copyright: 2006-2019 The Guava Authors +Home page: https://github.com/google/guava +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Google Error Prone Annotations. + +Copyright: Copyright 2011-2019 The Error Prone Authors +Home page: https://github.com/google/error-prone +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains checkerframework checker-qual Annotations. + +Copyright: 2004-2019 the Checker Framework developers +Home page: https://github.com/typetools/checker-framework +License: https://github.com/typetools/checker-framework/blob/master/LICENSE.txt (MIT license) + +License text: +| The annotations are licensed under the MIT License. (The text of this +| license appears below.) More specifically, all the parts of the Checker +| Framework that you might want to include with your own program use the +| MIT License. This is the checker-qual.jar file and all the files that +| appear in it: every file in a qual/ directory, plus utility files such +| as NullnessUtil.java, RegexUtil.java, SignednessUtil.java, etc. +| In addition, the cleanroom implementations of third-party annotations, +| which the Checker Framework recognizes as aliases for its own +| annotations, are licensed under the MIT License. +| +| Permission is hereby granted, free of charge, to any person obtaining a copy +| of this software and associated documentation files (the "Software"), to deal +| in the Software without restriction, including without limitation the rights +| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +| copies of the Software, and to permit persons to whom the Software is +| furnished to do so, subject to the following conditions: +| +| The above copyright notice and this permission notice shall be included in +| all copies or substantial portions of the Software. +| +| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +| THE SOFTWARE. + +-------------------------------------------------------------------------------- + +This binary artifact contains Caffeine by Ben Manes. + +Copyright: 2014-2019 Ben Manes and contributors +Home page: https://github.com/ben-manes/caffeine +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Arrow. + +Copyright: 2016-2019 The Apache Software Foundation. +Home page: https://arrow.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Netty's buffer library. + +Copyright: 2014-2020 The Netty Project +Home page: https://netty.io/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Google FlatBuffers. + +Copyright: 2013-2020 Google Inc. +Home page: https://google.github.io/flatbuffers/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains ThreeTen. + +Copyright: 2007-present, Stephen Colebourne & Michael Nascimento Santos. +Home page: https://www.threeten.org/threeten-extra/ +License: https://github.com/ThreeTen/threeten-extra/blob/master/LICENSE.txt (BSD 3-clause) + +License text: + +| All rights reserved. +| +| * Redistribution and use in source and binary forms, with or without +| modification, are permitted provided that the following conditions are met: +| +| * Redistributions of source code must retain the above copyright notice, +| this list of conditions and the following disclaimer. +| +| * Redistributions in binary form must reproduce the above copyright notice, +| this list of conditions and the following disclaimer in the documentation +| and/or other materials provided with the distribution. +| +| * Neither the name of JSR-310 nor the names of its contributors +| may be used to endorse or promote products derived from this software +| without specific prior written permission. +| +| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR +| CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +| EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +| PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +| PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +| LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +| NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +| SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This binary artifact contains code from Project Nessie. + +Copyright: 2015-2025 Dremio Corporation. +Home page: https://projectnessie.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Apache Spark. + +* vectorized reading of definition levels in BaseVectorizedParquetValuesReader.java +* portions of the extensions parser +* casting logic in AssignmentAlignmentSupport +* implementation of SetAccumulator. + +Copyright: 2011-2018 The Apache Software Foundation +Home page: https://spark.apache.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Delta Lake. + +* AssignmentAlignmentSupport is an independent development but UpdateExpressionsSupport in Delta was used as a reference. + +Copyright: 2020 The Delta Lake Project Authors. +Home page: https://delta.io/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary includes code from Apache Commons. + +* Core ArrayUtil. + +Copyright: 2020 The Apache Software Foundation +Home page: https://commons.apache.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache HttpComponents Client. + +Copyright: 1999-2022 The Apache Software Foundation. +Home page: https://hc.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This product includes code from Apache HttpComponents Client. + +* retry and error handling logic in ExponentialHttpRequestRetryStrategy.java + +Copyright: 1999-2022 The Apache Software Foundation. +Home page: https://hc.apache.org/ +License: https://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains failsafe. + +Copyright: Jonathan Halterman and friends +Home page: https://failsafe.dev/ +License: https://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +This binary artifact contains Eclipse MicroProfile OpenAPI. + +Home page: https://github.com/microprofile/microprofile-open-api +License: https://www.apache.org/licenses/LICENSE-2.0 diff --git a/spark/v4.0/spark-runtime/NOTICE b/spark/v4.0/spark-runtime/NOTICE new file mode 100644 index 000000000000..b5d1c8d1be4b --- /dev/null +++ b/spark/v4.0/spark-runtime/NOTICE @@ -0,0 +1,403 @@ + +Apache Iceberg +Copyright 2017-2025 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- + +This binary artifact contains code from Kite, developed at Cloudera, Inc. with +the following copyright notice: + +| Copyright 2013 Cloudera Inc. +| +| Licensed 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. + +-------------------------------------------------------------------------------- + +This binary artifact includes Airlift Aircompressor with the following in its +NOTICE file: + +| Snappy Copyright Notices +| ========================= +| +| * Copyright 2011 Dain Sundstrom +| * Copyright 2011, Google Inc. +| +| +| Snappy License +| =============== +| Copyright 2011, Google Inc. +| All rights reserved. +| +| Redistribution and use in source and binary forms, with or without +| modification, are permitted provided that the following conditions are +| met: +| +| * Redistributions of source code must retain the above copyright +| notice, this list of conditions and the following disclaimer. +| * Redistributions in binary form must reproduce the above +| copyright notice, this list of conditions and the following disclaimer +| in the documentation and/or other materials provided with the +| distribution. +| * Neither the name of Google Inc. nor the names of its +| contributors may be used to endorse or promote products derived from +| this software without specific prior written permission. +| +| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This binary artifact includes Google Protobuf with the following copyright +notice: + +| Copyright 2008 Google Inc. All rights reserved. +| +| Redistribution and use in source and binary forms, with or without +| modification, are permitted provided that the following conditions are +| met: +| +| * Redistributions of source code must retain the above copyright +| notice, this list of conditions and the following disclaimer. +| * Redistributions in binary form must reproduce the above +| copyright notice, this list of conditions and the following disclaimer +| in the documentation and/or other materials provided with the +| distribution. +| * Neither the name of Google Inc. nor the names of its +| contributors may be used to endorse or promote products derived from +| this software without specific prior written permission. +| +| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +| +| Code generated by the Protocol Buffer compiler is owned by the owner +| of the input file used when generating it. This code is not +| standalone and requires a support library to be linked with it. This +| support library is itself covered by the above license. + +-------------------------------------------------------------------------------- + +This binary artifact includes Netty buffers with the following in its NOTICE +file: + +| The Netty Project +| ================= +| +| Please visit the Netty web site for more information: +| +| * https://netty.io/ +| +| Copyright 2014 The Netty Project +| +| The Netty Project 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. +| +| Also, please refer to each LICENSE..txt file, which is located in +| the 'license' directory of the distribution file, for the license terms of the +| components that this product depends on. +| +| ------------------------------------------------------------------------------- +| This product contains the extensions to Java Collections Framework which has +| been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: +| +| * LICENSE: +| * license/LICENSE.jsr166y.txt (Public Domain) +| * HOMEPAGE: +| * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ +| * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ +| +| This product contains a modified version of Robert Harder's Public Domain +| Base64 Encoder and Decoder, which can be obtained at: +| +| * LICENSE: +| * license/LICENSE.base64.txt (Public Domain) +| * HOMEPAGE: +| * http://iharder.sourceforge.net/current/java/base64/ +| +| This product contains a modified portion of 'Webbit', an event based +| WebSocket and HTTP server, which can be obtained at: +| +| * LICENSE: +| * license/LICENSE.webbit.txt (BSD License) +| * HOMEPAGE: +| * https://github.com/joewalnes/webbit +| +| This product contains a modified portion of 'SLF4J', a simple logging +| facade for Java, which can be obtained at: +| +| * LICENSE: +| * license/LICENSE.slf4j.txt (MIT License) +| * HOMEPAGE: +| * http://www.slf4j.org/ +| +| This product contains a modified portion of 'Apache Harmony', an open source +| Java SE, which can be obtained at: +| +| * NOTICE: +| * license/NOTICE.harmony.txt +| * LICENSE: +| * license/LICENSE.harmony.txt (Apache License 2.0) +| * HOMEPAGE: +| * http://archive.apache.org/dist/harmony/ +| +| This product contains a modified portion of 'jbzip2', a Java bzip2 compression +| and decompression library written by Matthew J. Francis. It can be obtained at: +| +| * LICENSE: +| * license/LICENSE.jbzip2.txt (MIT License) +| * HOMEPAGE: +| * https://code.google.com/p/jbzip2/ +| +| This product contains a modified portion of 'libdivsufsort', a C API library to construct +| the suffix array and the Burrows-Wheeler transformed string for any input string of +| a constant-size alphabet written by Yuta Mori. It can be obtained at: +| +| * LICENSE: +| * license/LICENSE.libdivsufsort.txt (MIT License) +| * HOMEPAGE: +| * https://github.com/y-256/libdivsufsort +| +| This product contains a modified portion of Nitsan Wakart's 'JCTools', Java Concurrency Tools for the JVM, +| which can be obtained at: +| +| * LICENSE: +| * license/LICENSE.jctools.txt (ASL2 License) +| * HOMEPAGE: +| * https://github.com/JCTools/JCTools +| +| This product optionally depends on 'JZlib', a re-implementation of zlib in +| pure Java, which can be obtained at: +| +| * LICENSE: +| * license/LICENSE.jzlib.txt (BSD style License) +| * HOMEPAGE: +| * http://www.jcraft.com/jzlib/ +| +| This product optionally depends on 'Compress-LZF', a Java library for encoding and +| decoding data in LZF format, written by Tatu Saloranta. It can be obtained at: +| +| * LICENSE: +| * license/LICENSE.compress-lzf.txt (Apache License 2.0) +| * HOMEPAGE: +| * https://github.com/ning/compress +| +| This product optionally depends on 'lz4', a LZ4 Java compression +| and decompression library written by Adrien Grand. It can be obtained at: +| +| * LICENSE: +| * license/LICENSE.lz4.txt (Apache License 2.0) +| * HOMEPAGE: +| * https://github.com/jpountz/lz4-java +| +| This product optionally depends on 'lzma-java', a LZMA Java compression +| and decompression library, which can be obtained at: +| +| * LICENSE: +| * license/LICENSE.lzma-java.txt (Apache License 2.0) +| * HOMEPAGE: +| * https://github.com/jponge/lzma-java +| +| This product contains a modified portion of 'jfastlz', a Java port of FastLZ compression +| and decompression library written by William Kinney. It can be obtained at: +| +| * LICENSE: +| * license/LICENSE.jfastlz.txt (MIT License) +| * HOMEPAGE: +| * https://code.google.com/p/jfastlz/ +| +| This product contains a modified portion of and optionally depends on 'Protocol Buffers', Google's data +| interchange format, which can be obtained at: +| +| * LICENSE: +| * license/LICENSE.protobuf.txt (New BSD License) +| * HOMEPAGE: +| * https://github.com/google/protobuf +| +| This product optionally depends on 'Bouncy Castle Crypto APIs' to generate +| a temporary self-signed X.509 certificate when the JVM does not provide the +| equivalent functionality. It can be obtained at: +| +| * LICENSE: +| * license/LICENSE.bouncycastle.txt (MIT License) +| * HOMEPAGE: +| * http://www.bouncycastle.org/ +| +| This product optionally depends on 'Snappy', a compression library produced +| by Google Inc, which can be obtained at: +| +| * LICENSE: +| * license/LICENSE.snappy.txt (New BSD License) +| * HOMEPAGE: +| * https://github.com/google/snappy +| +| This product optionally depends on 'JBoss Marshalling', an alternative Java +| serialization API, which can be obtained at: +| +| * LICENSE: +| * license/LICENSE.jboss-marshalling.txt (Apache License 2.0) +| * HOMEPAGE: +| * https://github.com/jboss-remoting/jboss-marshalling +| +| This product optionally depends on 'Caliper', Google's micro- +| benchmarking framework, which can be obtained at: +| +| * LICENSE: +| * license/LICENSE.caliper.txt (Apache License 2.0) +| * HOMEPAGE: +| * https://github.com/google/caliper +| +| This product optionally depends on 'Apache Commons Logging', a logging +| framework, which can be obtained at: +| +| * LICENSE: +| * license/LICENSE.commons-logging.txt (Apache License 2.0) +| * HOMEPAGE: +| * http://commons.apache.org/logging/ +| +| This product optionally depends on 'Apache Log4J', a logging framework, which +| can be obtained at: +| +| * LICENSE: +| * license/LICENSE.log4j.txt (Apache License 2.0) +| * HOMEPAGE: +| * http://logging.apache.org/log4j/ +| +| This product optionally depends on 'Aalto XML', an ultra-high performance +| non-blocking XML processor, which can be obtained at: +| +| * LICENSE: +| * license/LICENSE.aalto-xml.txt (Apache License 2.0) +| * HOMEPAGE: +| * http://wiki.fasterxml.com/AaltoHome +| +| This product contains a modified version of 'HPACK', a Java implementation of +| the HTTP/2 HPACK algorithm written by Twitter. It can be obtained at: +| +| * LICENSE: +| * license/LICENSE.hpack.txt (Apache License 2.0) +| * HOMEPAGE: +| * https://github.com/twitter/hpack +| +| This product contains a modified version of 'HPACK', a Java implementation of +| the HTTP/2 HPACK algorithm written by Cory Benfield. It can be obtained at: +| +| * LICENSE: +| * license/LICENSE.hyper-hpack.txt (MIT License) +| * HOMEPAGE: +| * https://github.com/python-hyper/hpack/ +| +| This product contains a modified version of 'HPACK', a Java implementation of +| the HTTP/2 HPACK algorithm written by Tatsuhiro Tsujikawa. It can be obtained at: +| +| * LICENSE: +| * license/LICENSE.nghttp2-hpack.txt (MIT License) +| * HOMEPAGE: +| * https://github.com/nghttp2/nghttp2/ +| +| This product contains a modified portion of 'Apache Commons Lang', a Java library +| provides utilities for the java.lang API, which can be obtained at: +| +| * LICENSE: +| * license/LICENSE.commons-lang.txt (Apache License 2.0) +| * HOMEPAGE: +| * https://commons.apache.org/proper/commons-lang/ +| +| +| This product contains the Maven wrapper scripts from 'Maven Wrapper', that provides an easy way to ensure a user has everything necessary to run the Maven build. +| +| * LICENSE: +| * license/LICENSE.mvn-wrapper.txt (Apache License 2.0) +| * HOMEPAGE: +| * https://github.com/takari/maven-wrapper +| +| This product contains the dnsinfo.h header file, that provides a way to retrieve the system DNS configuration on MacOS. +| This private header is also used by Apple's open source +| mDNSResponder (https://opensource.apple.com/tarballs/mDNSResponder/). +| +| * LICENSE: +| * license/LICENSE.dnsinfo.txt (Apache License 2.0) +| * HOMEPAGE: +| * http://www.opensource.apple.com/source/configd/configd-453.19/dnsinfo/dnsinfo.h + +-------------------------------------------------------------------------------- + +This binary artifact includes Project Nessie with the following in its NOTICE +file: + +| Nessie +| Copyright 2015-2025 Dremio Corporation +| +| --------------------------------------- +| This project includes code from Apache Polaris (incubating), with the following in its NOTICE file: +| +| | Apache Polaris (incubating) +| | Copyright 2024 The Apache Software Foundation +| | +| | This product includes software developed at +| | The Apache Software Foundation (http://www.apache.org/). +| | +| | The initial code for the Polaris project was donated +| | to the ASF by Snowflake Inc. (https://www.snowflake.com/) copyright 2024. + +-------------------------------------------------------------------------------- + +This binary artifact includes Eclipse MicroProfile OpenAPI with the following in its NOTICE file: + +| ========================================================================= +| == NOTICE file corresponding to section 4(d) of the Apache License, == +| == Version 2.0, in this case for MicroProfile OpenAPI == +| ========================================================================= +| +| The majority of this software were originally based on the following: +| * Swagger Core +| https://github.com/swagger-api/swagger-core +| under Apache License, v2.0 +| +| +| SPDXVersion: SPDX-2.1 +| PackageName: Eclipse MicroProfile +| PackageHomePage: http://www.eclipse.org/microprofile +| PackageLicenseDeclared: Apache-2.0 +| +| PackageCopyrightText: +| Arthur De Magalhaes arthurdm@ca.ibm.com +| diff --git a/spark/v4.0/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java b/spark/v4.0/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java new file mode 100644 index 000000000000..29f725615a21 --- /dev/null +++ b/spark/v4.0/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java @@ -0,0 +1,182 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Files; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.spark.extensions.ExtensionsTestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRoundTrip extends ExtensionsTestBase { + @AfterEach + public void dropTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + // Run through our Doc's Getting Started Example + // TODO Update doc example so that it can actually be run, modifications were required for this + // test suite to run + @TestTemplate + public void testGettingStarted() throws IOException { + // Creating a table + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + // Writing + sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); + assertThat(scalarSql("SELECT COUNT(*) FROM %s", tableName)) + .as("Should have inserted 3 rows") + .isEqualTo(3L); + + sql("DROP TABLE IF EXISTS source PURGE"); + sql( + "CREATE TABLE source (id bigint, data string) USING parquet LOCATION '%s'", + Files.createTempDirectory(temp, "junit")); + sql("INSERT INTO source VALUES (10, 'd'), (11, 'ee')"); + + sql("INSERT INTO %s SELECT id, data FROM source WHERE length(data) = 1", tableName); + assertThat(scalarSql("SELECT COUNT(*) FROM %s", tableName)) + .as("Table should now have 4 rows") + .isEqualTo(4L); + + sql("DROP TABLE IF EXISTS updates PURGE"); + sql( + "CREATE TABLE updates (id bigint, data string) USING parquet LOCATION '%s'", + Files.createTempDirectory(temp, "junit")); + sql("INSERT INTO updates VALUES (1, 'x'), (2, 'x'), (4, 'z')"); + + sql( + "MERGE INTO %s t USING (SELECT * FROM updates) u ON t.id = u.id\n" + + "WHEN MATCHED THEN UPDATE SET t.data = u.data\n" + + "WHEN NOT MATCHED THEN INSERT *", + tableName); + assertThat(scalarSql("SELECT COUNT(*) FROM %s", tableName)) + .as("Table should now have 5 rows") + .isEqualTo(5L); + assertThat(scalarSql("SELECT data FROM %s WHERE id = 1", tableName)) + .as("Record 1 should now have data x") + .isEqualTo("x"); + + // Reading + assertThat( + scalarSql( + "SELECT count(1) as count FROM %s WHERE data = 'x' GROUP BY data ", tableName)) + .as("There should be 2 records with data x") + .isEqualTo(2L); + + // Not supported because of Spark limitation + if (!catalogName.equals("spark_catalog")) { + assertThat(scalarSql("SELECT COUNT(*) FROM %s.snapshots", tableName)) + .as("There should be 3 snapshots") + .isEqualTo(3L); + } + } + + // From Spark DDL Docs section + @TestTemplate + public void testAlterTable() { + sql( + "CREATE TABLE %s (category int, id bigint, data string, ts timestamp) USING iceberg", + tableName); + Table table; + // Add examples + sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, id)", tableName); + sql("ALTER TABLE %s ADD PARTITION FIELD truncate(data, 4)", tableName); + sql("ALTER TABLE %s ADD PARTITION FIELD years(ts)", tableName); + sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, category) AS shard", tableName); + table = getTable(); + assertThat(table.spec().fields()).as("Table should have 4 partition fields").hasSize(4); + + // Drop Examples + sql("ALTER TABLE %s DROP PARTITION FIELD bucket(16, id)", tableName); + sql("ALTER TABLE %s DROP PARTITION FIELD truncate(data, 4)", tableName); + sql("ALTER TABLE %s DROP PARTITION FIELD years(ts)", tableName); + sql("ALTER TABLE %s DROP PARTITION FIELD shard", tableName); + + table = getTable(); + assertThat(table.spec().isUnpartitioned()).as("Table should be unpartitioned").isTrue(); + + // Sort order examples + sql("ALTER TABLE %s WRITE ORDERED BY category, id", tableName); + sql("ALTER TABLE %s WRITE ORDERED BY category ASC, id DESC", tableName); + sql("ALTER TABLE %s WRITE ORDERED BY category ASC NULLS LAST, id DESC NULLS FIRST", tableName); + table = getTable(); + assertThat(table.sortOrder().fields()).as("Table should be sorted on 2 fields").hasSize(2); + } + + @TestTemplate + public void testCreateTable() { + sql("DROP TABLE IF EXISTS %s", tableName("first")); + sql("DROP TABLE IF EXISTS %s", tableName("second")); + sql("DROP TABLE IF EXISTS %s", tableName("third")); + + sql( + "CREATE TABLE %s (\n" + + " id bigint COMMENT 'unique id',\n" + + " data string)\n" + + "USING iceberg", + tableName("first")); + getTable("first"); // Table should exist + + sql( + "CREATE TABLE %s (\n" + + " id bigint,\n" + + " data string,\n" + + " category string)\n" + + "USING iceberg\n" + + "PARTITIONED BY (category)", + tableName("second")); + Table second = getTable("second"); + assertThat(second.spec().fields()).as("Should be partitioned on 1 column").hasSize(1); + + sql( + "CREATE TABLE %s (\n" + + " id bigint,\n" + + " data string,\n" + + " category string,\n" + + " ts timestamp)\n" + + "USING iceberg\n" + + "PARTITIONED BY (bucket(16, id), days(ts), category)", + tableName("third")); + Table third = getTable("third"); + assertThat(third.spec().fields()).as("Should be partitioned on 3 columns").hasSize(3); + } + + @TestTemplate + public void showView() { + sql("CREATE VIEW %s AS SELECT 1 AS id", "test"); + assertThat(sql("SHOW VIEWS")).contains(row("default", "test", false)); + sql("DROP VIEW %s", "test"); + } + + private Table getTable(String name) { + return validationCatalog.loadTable(TableIdentifier.of("default", name)); + } + + private Table getTable() { + return validationCatalog.loadTable(tableIdent); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/SparkBenchmarkUtil.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/SparkBenchmarkUtil.java new file mode 100644 index 000000000000..b980c39b5bc3 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/SparkBenchmarkUtil.java @@ -0,0 +1,58 @@ +/* + * 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.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.expressions.Attribute; +import org.apache.spark.sql.catalyst.expressions.AttributeReference; +import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; +import org.apache.spark.sql.catalyst.types.DataTypeUtils; +import org.apache.spark.sql.types.StructType; +import scala.collection.JavaConverters; + +public class SparkBenchmarkUtil { + + private SparkBenchmarkUtil() {} + + public static UnsafeProjection projection(Schema expectedSchema, Schema actualSchema) { + StructType struct = SparkSchemaUtil.convert(actualSchema); + + List refs = + JavaConverters.seqAsJavaListConverter(DataTypeUtils.toAttributes(struct)).asJava(); + List attrs = Lists.newArrayListWithExpectedSize(struct.fields().length); + List exprs = Lists.newArrayListWithExpectedSize(struct.fields().length); + + for (AttributeReference ref : refs) { + attrs.add(ref.toAttribute()); + } + + for (Types.NestedField field : expectedSchema.columns()) { + int indexInIterSchema = struct.fieldIndex(field.name()); + exprs.add(refs.get(indexInIterSchema)); + } + + return UnsafeProjection.create( + JavaConverters.asScalaBufferConverter(exprs).asScala().toSeq(), + JavaConverters.asScalaBufferConverter(attrs).asScala().toSeq()); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java new file mode 100644 index 000000000000..e1d9ac18dac1 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java @@ -0,0 +1,189 @@ +/* + * 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.action; + +import static org.apache.spark.sql.functions.lit; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.sql.Timestamp; +import java.util.List; +import java.util.Locale; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * A benchmark that evaluates the performance of remove orphan files action in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=DeleteOrphanFilesBenchmark + * -PjmhOutputPath=benchmark/delete-orphan-files-benchmark-results.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 1000, timeUnit = TimeUnit.HOURS) +public class DeleteOrphanFilesBenchmark { + + private static final String TABLE_NAME = "delete_orphan_perf"; + private static final int NUM_SNAPSHOTS = 1000; + private static final int NUM_FILES = 1000; + + private SparkSession spark; + private final List validAndOrphanPaths = Lists.newArrayList(); + private Table table; + + @Setup + public void setupBench() { + setupSpark(); + initTable(); + appendData(); + addOrphans(); + } + + @TearDown + public void teardownBench() { + tearDownSpark(); + } + + @Benchmark + @Threads(1) + public void testDeleteOrphanFiles(Blackhole blackhole) { + Dataset validAndOrphanPathsDF = + spark + .createDataset(validAndOrphanPaths, Encoders.STRING()) + .withColumnRenamed("value", "file_path") + .withColumn("last_modified", lit(new Timestamp(10000))); + + DeleteOrphanFiles.Result results = + SparkActions.get(spark) + .deleteOrphanFiles(table()) + .compareToFileList(validAndOrphanPathsDF) + .execute(); + blackhole.consume(results); + } + + private void initTable() { + spark.sql( + String.format( + "CREATE TABLE %s(id INT, name STRING)" + + " USING ICEBERG" + + " TBLPROPERTIES ( 'format-version' = '2')", + TABLE_NAME)); + } + + private void appendData() { + String location = table().location(); + PartitionSpec partitionSpec = table().spec(); + + for (int i = 0; i < NUM_SNAPSHOTS; i++) { + AppendFiles appendFiles = table().newFastAppend(); + for (int j = 0; j < NUM_FILES; j++) { + String path = String.format(Locale.ROOT, "%s/path/to/data-%d-%d.parquet", location, i, j); + validAndOrphanPaths.add(path); + DataFile dataFile = + DataFiles.builder(partitionSpec) + .withPath(path) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + appendFiles.appendFile(dataFile); + } + appendFiles.commit(); + } + } + + private void addOrphans() { + String location = table.location(); + // Generate 10% orphan files + int orphanFileCount = (NUM_FILES * NUM_SNAPSHOTS) / 10; + for (int i = 0; i < orphanFileCount; i++) { + validAndOrphanPaths.add( + String.format("%s/path/to/data-%s.parquet", location, UUID.randomUUID())); + } + } + + private Table table() { + if (table == null) { + try { + table = Spark3Util.loadIcebergTable(spark, TABLE_NAME); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + return table; + } + + private String catalogWarehouse() { + try { + return Files.createTempDirectory("benchmark-").toAbsolutePath() + + "/" + + UUID.randomUUID() + + "/"; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private void setupSpark() { + SparkSession.Builder builder = + SparkSession.builder() + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", catalogWarehouse()) + .master("local"); + spark = builder.getOrCreate(); + } + + private void tearDownSpark() { + spark.stop(); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java new file mode 100644 index 000000000000..77b79384a6d8 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/IcebergSortCompactionBenchmark.java @@ -0,0 +1,404 @@ +/* + * 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.action; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.current_date; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +import java.io.IOException; +import java.nio.file.Files; +import java.util.Collections; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.SizeBasedFileRewritePlanner; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.types.DataTypes; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; + +@Fork(1) +@State(Scope.Benchmark) +@Measurement(iterations = 10) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 1000, timeUnit = TimeUnit.HOURS) +public class IcebergSortCompactionBenchmark { + + private static final String[] NAMESPACE = new String[] {"default"}; + private static final String NAME = "sortbench"; + private static final Identifier IDENT = Identifier.of(NAMESPACE, NAME); + private static final int NUM_FILES = 8; + private static final long NUM_ROWS = 7500000L; + private static final long UNIQUE_VALUES = NUM_ROWS / 4; + + private final Configuration hadoopConf = initHadoopConf(); + private SparkSession spark; + + @Setup + public void setupBench() { + setupSpark(); + } + + @TearDown + public void teardownBench() { + tearDownSpark(); + } + + @Setup(Level.Iteration) + public void setupIteration() { + initTable(); + appendData(); + } + + @TearDown(Level.Iteration) + public void cleanUpIteration() throws IOException { + cleanupFiles(); + } + + @Benchmark + @Threads(1) + public void sortInt() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .sort( + SortOrder.builderFor(table().schema()) + .sortBy("intCol", SortDirection.ASC, NullOrder.NULLS_FIRST) + .build()) + .execute(); + } + + @Benchmark + @Threads(1) + public void sortInt2() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .sort( + SortOrder.builderFor(table().schema()) + .sortBy("intCol", SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy("intCol2", SortDirection.ASC, NullOrder.NULLS_FIRST) + .build()) + .execute(); + } + + @Benchmark + @Threads(1) + public void sortInt3() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .sort( + SortOrder.builderFor(table().schema()) + .sortBy("intCol", SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy("intCol2", SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy("intCol3", SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy("intCol4", SortDirection.ASC, NullOrder.NULLS_FIRST) + .build()) + .execute(); + } + + @Benchmark + @Threads(1) + public void sortInt4() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .sort( + SortOrder.builderFor(table().schema()) + .sortBy("intCol", SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy("intCol2", SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy("intCol3", SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy("intCol4", SortDirection.ASC, NullOrder.NULLS_FIRST) + .build()) + .execute(); + } + + @Benchmark + @Threads(1) + public void sortString() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .sort( + SortOrder.builderFor(table().schema()) + .sortBy("stringCol", SortDirection.ASC, NullOrder.NULLS_FIRST) + .build()) + .execute(); + } + + @Benchmark + @Threads(1) + public void sortFourColumns() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .sort( + SortOrder.builderFor(table().schema()) + .sortBy("stringCol", SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy("intCol", SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy("dateCol", SortDirection.DESC, NullOrder.NULLS_FIRST) + .sortBy("doubleCol", SortDirection.DESC, NullOrder.NULLS_FIRST) + .build()) + .execute(); + } + + @Benchmark + @Threads(1) + public void sortSixColumns() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .sort( + SortOrder.builderFor(table().schema()) + .sortBy("stringCol", SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy("intCol", SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy("dateCol", SortDirection.DESC, NullOrder.NULLS_FIRST) + .sortBy("timestampCol", SortDirection.DESC, NullOrder.NULLS_FIRST) + .sortBy("doubleCol", SortDirection.DESC, NullOrder.NULLS_FIRST) + .sortBy("longCol", SortDirection.DESC, NullOrder.NULLS_FIRST) + .build()) + .execute(); + } + + @Benchmark + @Threads(1) + public void zSortInt() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .zOrder("intCol") + .execute(); + } + + @Benchmark + @Threads(1) + public void zSortInt2() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .zOrder("intCol", "intCol2") + .execute(); + } + + @Benchmark + @Threads(1) + public void zSortInt3() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .zOrder("intCol", "intCol2", "intCol3") + .execute(); + } + + @Benchmark + @Threads(1) + public void zSortInt4() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .zOrder("intCol", "intCol2", "intCol3", "intCol4") + .execute(); + } + + @Benchmark + @Threads(1) + public void zSortString() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .zOrder("stringCol") + .execute(); + } + + @Benchmark + @Threads(1) + public void zSortFourColumns() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .zOrder("stringCol", "intCol", "dateCol", "doubleCol") + .execute(); + } + + @Benchmark + @Threads(1) + public void zSortSixColumns() { + SparkActions.get() + .rewriteDataFiles(table()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .zOrder("stringCol", "intCol", "dateCol", "timestampCol", "doubleCol", "longCol") + .execute(); + } + + protected Configuration initHadoopConf() { + return new Configuration(); + } + + protected final void initTable() { + Schema schema = + new Schema( + required(1, "longCol", Types.LongType.get()), + required(2, "intCol", Types.IntegerType.get()), + required(3, "intCol2", Types.IntegerType.get()), + required(4, "intCol3", Types.IntegerType.get()), + required(5, "intCol4", Types.IntegerType.get()), + required(6, "floatCol", Types.FloatType.get()), + optional(7, "doubleCol", Types.DoubleType.get()), + optional(8, "dateCol", Types.DateType.get()), + optional(9, "timestampCol", Types.TimestampType.withZone()), + optional(10, "stringCol", Types.StringType.get())); + + SparkSessionCatalog catalog; + try { + catalog = + (SparkSessionCatalog) + Spark3Util.catalogAndIdentifier(spark(), "spark_catalog").catalog(); + catalog.dropTable(IDENT); + catalog.createTable( + IDENT, SparkSchemaUtil.convert(schema), new Transform[0], Collections.emptyMap()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private void appendData() { + Dataset df = + spark() + .range(0, NUM_ROWS * NUM_FILES, 1, NUM_FILES) + .drop("id") + .withColumn("longCol", new RandomGeneratingUDF(UNIQUE_VALUES).randomLongUDF().apply()) + .withColumn( + "intCol", + new RandomGeneratingUDF(UNIQUE_VALUES) + .randomLongUDF() + .apply() + .cast(DataTypes.IntegerType)) + .withColumn( + "intCol2", + new RandomGeneratingUDF(UNIQUE_VALUES) + .randomLongUDF() + .apply() + .cast(DataTypes.IntegerType)) + .withColumn( + "intCol3", + new RandomGeneratingUDF(UNIQUE_VALUES) + .randomLongUDF() + .apply() + .cast(DataTypes.IntegerType)) + .withColumn( + "intCol4", + new RandomGeneratingUDF(UNIQUE_VALUES) + .randomLongUDF() + .apply() + .cast(DataTypes.IntegerType)) + .withColumn( + "floatCol", + new RandomGeneratingUDF(UNIQUE_VALUES) + .randomLongUDF() + .apply() + .cast(DataTypes.FloatType)) + .withColumn( + "doubleCol", + new RandomGeneratingUDF(UNIQUE_VALUES) + .randomLongUDF() + .apply() + .cast(DataTypes.DoubleType)) + .withColumn("dateCol", date_add(current_date(), col("intCol").mod(NUM_FILES))) + .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)")) + .withColumn("stringCol", new RandomGeneratingUDF(UNIQUE_VALUES).randomString().apply()); + writeData(df); + } + + private void writeData(Dataset df) { + df.write().format("iceberg").mode(SaveMode.Append).save(NAME); + } + + protected final Table table() { + try { + return Spark3Util.loadIcebergTable(spark(), NAME); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + protected final SparkSession spark() { + return spark; + } + + protected String getCatalogWarehouse() { + try { + String location = + Files.createTempDirectory("benchmark-").toAbsolutePath() + "/" + UUID.randomUUID() + "/"; + return location; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + protected void cleanupFiles() throws IOException { + spark.sql("DROP TABLE IF EXISTS " + NAME); + } + + protected void setupSpark() { + SparkSession.Builder builder = + SparkSession.builder() + .config( + "spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog") + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", getCatalogWarehouse()) + .master("local[*]"); + spark = builder.getOrCreate(); + Configuration sparkHadoopConf = spark.sessionState().newHadoopConf(); + hadoopConf.forEach(entry -> sparkHadoopConf.set(entry.getKey(), entry.getValue())); + } + + protected void tearDownSpark() { + spark.stop(); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java new file mode 100644 index 000000000000..d8f9301a7d82 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java @@ -0,0 +1,50 @@ +/* + * 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.action; + +import static org.apache.spark.sql.functions.udf; + +import java.io.Serializable; +import java.util.Random; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.RandomUtil; +import org.apache.spark.sql.expressions.UserDefinedFunction; +import org.apache.spark.sql.types.DataTypes; + +class RandomGeneratingUDF implements Serializable { + private final long uniqueValues; + private final Random rand = new Random(); + + RandomGeneratingUDF(long uniqueValues) { + this.uniqueValues = uniqueValues; + } + + UserDefinedFunction randomLongUDF() { + return udf(() -> rand.nextLong() % (uniqueValues / 2), DataTypes.LongType) + .asNondeterministic() + .asNonNullable(); + } + + UserDefinedFunction randomString() { + return udf( + () -> RandomUtil.generatePrimitive(Types.StringType.get(), rand), DataTypes.StringType) + .asNondeterministic() + .asNonNullable(); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java new file mode 100644 index 000000000000..3dbee5dfd0f5 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java @@ -0,0 +1,229 @@ +/* + * 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.data.parquet; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import org.apache.avro.generic.GenericData; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.spark.SparkBenchmarkUtil; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.SparkParquetReaders; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; +import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport; +import org.apache.spark.sql.types.StructType; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * A benchmark that evaluates the performance of reading Parquet data with a flat schema using + * Iceberg and Spark Parquet readers. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=SparkParquetReadersFlatDataBenchmark + * -PjmhOutputPath=benchmark/spark-parquet-readers-flat-data-benchmark-result.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class SparkParquetReadersFlatDataBenchmark { + + private static final DynMethods.UnboundMethod APPLY_PROJECTION = + DynMethods.builder("apply").impl(UnsafeProjection.class, InternalRow.class).build(); + private static final Schema SCHEMA = + new Schema( + required(1, "longCol", Types.LongType.get()), + required(2, "intCol", Types.IntegerType.get()), + required(3, "floatCol", Types.FloatType.get()), + optional(4, "doubleCol", Types.DoubleType.get()), + optional(5, "decimalCol", Types.DecimalType.of(20, 5)), + optional(6, "dateCol", Types.DateType.get()), + optional(7, "timestampCol", Types.TimestampType.withZone()), + optional(8, "stringCol", Types.StringType.get())); + private static final Schema PROJECTED_SCHEMA = + new Schema( + required(1, "longCol", Types.LongType.get()), + optional(5, "decimalCol", Types.DecimalType.of(20, 5)), + optional(8, "stringCol", Types.StringType.get())); + private static final int NUM_RECORDS = 1000000; + private File dataFile; + + @Setup + public void setupBenchmark() throws IOException { + dataFile = File.createTempFile("parquet-flat-data-benchmark", ".parquet"); + dataFile.delete(); + List records = RandomData.generateList(SCHEMA, NUM_RECORDS, 0L); + try (FileAppender writer = + Parquet.write(Files.localOutput(dataFile)).schema(SCHEMA).named("benchmark").build()) { + writer.addAll(records); + } + } + + @TearDown + public void tearDownBenchmark() { + if (dataFile != null) { + dataFile.delete(); + } + } + + @Benchmark + @Threads(1) + public void readUsingIcebergReader(Blackhole blackHole) throws IOException { + try (CloseableIterable rows = + Parquet.read(Files.localInput(dataFile)) + .project(SCHEMA) + .createReaderFunc(type -> SparkParquetReaders.buildReader(SCHEMA, type)) + .build()) { + + for (InternalRow row : rows) { + blackHole.consume(row); + } + } + } + + @Benchmark + @Threads(1) + public void readUsingIcebergReaderUnsafe(Blackhole blackhole) throws IOException { + try (CloseableIterable rows = + Parquet.read(Files.localInput(dataFile)) + .project(SCHEMA) + .createReaderFunc(type -> SparkParquetReaders.buildReader(SCHEMA, type)) + .build()) { + + Iterable unsafeRows = + Iterables.transform( + rows, APPLY_PROJECTION.bind(SparkBenchmarkUtil.projection(SCHEMA, SCHEMA))::invoke); + + for (InternalRow row : unsafeRows) { + blackhole.consume(row); + } + } + } + + @Benchmark + @Threads(1) + public void readUsingSparkReader(Blackhole blackhole) throws IOException { + StructType sparkSchema = SparkSchemaUtil.convert(SCHEMA); + try (CloseableIterable rows = + Parquet.read(Files.localInput(dataFile)) + .project(SCHEMA) + .readSupport(new ParquetReadSupport()) + .set("org.apache.spark.sql.parquet.row.requested_schema", sparkSchema.json()) + .set("spark.sql.parquet.binaryAsString", "false") + .set("spark.sql.parquet.int96AsTimestamp", "false") + .set("spark.sql.caseSensitive", "false") + .set("spark.sql.parquet.fieldId.write.enabled", "false") + .set("spark.sql.parquet.inferTimestampNTZ.enabled", "false") + .set("spark.sql.legacy.parquet.nanosAsLong", "false") + .callInit() + .build()) { + + for (InternalRow row : rows) { + blackhole.consume(row); + } + } + } + + @Benchmark + @Threads(1) + public void readWithProjectionUsingIcebergReader(Blackhole blackhole) throws IOException { + try (CloseableIterable rows = + Parquet.read(Files.localInput(dataFile)) + .project(PROJECTED_SCHEMA) + .createReaderFunc(type -> SparkParquetReaders.buildReader(PROJECTED_SCHEMA, type)) + .build()) { + + for (InternalRow row : rows) { + blackhole.consume(row); + } + } + } + + @Benchmark + @Threads(1) + public void readWithProjectionUsingIcebergReaderUnsafe(Blackhole blackhole) throws IOException { + try (CloseableIterable rows = + Parquet.read(Files.localInput(dataFile)) + .project(PROJECTED_SCHEMA) + .createReaderFunc(type -> SparkParquetReaders.buildReader(PROJECTED_SCHEMA, type)) + .build()) { + + Iterable unsafeRows = + Iterables.transform( + rows, + APPLY_PROJECTION.bind( + SparkBenchmarkUtil.projection(PROJECTED_SCHEMA, PROJECTED_SCHEMA)) + ::invoke); + + for (InternalRow row : unsafeRows) { + blackhole.consume(row); + } + } + } + + @Benchmark + @Threads(1) + public void readWithProjectionUsingSparkReader(Blackhole blackhole) throws IOException { + StructType sparkSchema = SparkSchemaUtil.convert(PROJECTED_SCHEMA); + try (CloseableIterable rows = + Parquet.read(Files.localInput(dataFile)) + .project(PROJECTED_SCHEMA) + .readSupport(new ParquetReadSupport()) + .set("org.apache.spark.sql.parquet.row.requested_schema", sparkSchema.json()) + .set("spark.sql.parquet.binaryAsString", "false") + .set("spark.sql.parquet.int96AsTimestamp", "false") + .set("spark.sql.caseSensitive", "false") + .set("spark.sql.parquet.inferTimestampNTZ.enabled", "false") + .set("spark.sql.legacy.parquet.nanosAsLong", "false") + .callInit() + .build()) { + + for (InternalRow row : rows) { + blackhole.consume(row); + } + } + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java new file mode 100644 index 000000000000..8487988d9e5b --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java @@ -0,0 +1,227 @@ +/* + * 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.data.parquet; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import org.apache.avro.generic.GenericData; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.spark.SparkBenchmarkUtil; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.SparkParquetReaders; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.UnsafeProjection; +import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport; +import org.apache.spark.sql.types.StructType; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * A benchmark that evaluates the performance of reading nested Parquet data using Iceberg and Spark + * Parquet readers. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=SparkParquetReadersNestedDataBenchmark + * -PjmhOutputPath=benchmark/spark-parquet-readers-nested-data-benchmark-result.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class SparkParquetReadersNestedDataBenchmark { + + private static final DynMethods.UnboundMethod APPLY_PROJECTION = + DynMethods.builder("apply").impl(UnsafeProjection.class, InternalRow.class).build(); + private static final Schema SCHEMA = + new Schema( + required(0, "id", Types.LongType.get()), + optional( + 4, + "nested", + Types.StructType.of( + required(1, "col1", Types.StringType.get()), + required(2, "col2", Types.DoubleType.get()), + required(3, "col3", Types.LongType.get())))); + private static final Schema PROJECTED_SCHEMA = + new Schema( + optional(4, "nested", Types.StructType.of(required(1, "col1", Types.StringType.get())))); + private static final int NUM_RECORDS = 1000000; + private File dataFile; + + @Setup + public void setupBenchmark() throws IOException { + dataFile = File.createTempFile("parquet-nested-data-benchmark", ".parquet"); + dataFile.delete(); + List records = RandomData.generateList(SCHEMA, NUM_RECORDS, 0L); + try (FileAppender writer = + Parquet.write(Files.localOutput(dataFile)).schema(SCHEMA).named("benchmark").build()) { + writer.addAll(records); + } + } + + @TearDown + public void tearDownBenchmark() { + if (dataFile != null) { + dataFile.delete(); + } + } + + @Benchmark + @Threads(1) + public void readUsingIcebergReader(Blackhole blackhole) throws IOException { + try (CloseableIterable rows = + Parquet.read(Files.localInput(dataFile)) + .project(SCHEMA) + .createReaderFunc(type -> SparkParquetReaders.buildReader(SCHEMA, type)) + .build()) { + + for (InternalRow row : rows) { + blackhole.consume(row); + } + } + } + + @Benchmark + @Threads(1) + public void readUsingIcebergReaderUnsafe(Blackhole blackhole) throws IOException { + try (CloseableIterable rows = + Parquet.read(Files.localInput(dataFile)) + .project(SCHEMA) + .createReaderFunc(type -> SparkParquetReaders.buildReader(SCHEMA, type)) + .build()) { + + Iterable unsafeRows = + Iterables.transform( + rows, APPLY_PROJECTION.bind(SparkBenchmarkUtil.projection(SCHEMA, SCHEMA))::invoke); + + for (InternalRow row : unsafeRows) { + blackhole.consume(row); + } + } + } + + @Benchmark + @Threads(1) + public void readUsingSparkReader(Blackhole blackhole) throws IOException { + StructType sparkSchema = SparkSchemaUtil.convert(SCHEMA); + try (CloseableIterable rows = + Parquet.read(Files.localInput(dataFile)) + .project(SCHEMA) + .readSupport(new ParquetReadSupport()) + .set("org.apache.spark.sql.parquet.row.requested_schema", sparkSchema.json()) + .set("spark.sql.parquet.binaryAsString", "false") + .set("spark.sql.parquet.int96AsTimestamp", "false") + .set("spark.sql.caseSensitive", "false") + .set("spark.sql.parquet.fieldId.write.enabled", "false") + .set("spark.sql.parquet.inferTimestampNTZ.enabled", "false") + .set("spark.sql.legacy.parquet.nanosAsLong", "false") + .callInit() + .build()) { + + for (InternalRow row : rows) { + blackhole.consume(row); + } + } + } + + @Benchmark + @Threads(1) + public void readWithProjectionUsingIcebergReader(Blackhole blackhole) throws IOException { + try (CloseableIterable rows = + Parquet.read(Files.localInput(dataFile)) + .project(PROJECTED_SCHEMA) + .createReaderFunc(type -> SparkParquetReaders.buildReader(PROJECTED_SCHEMA, type)) + .build()) { + + for (InternalRow row : rows) { + blackhole.consume(row); + } + } + } + + @Benchmark + @Threads(1) + public void readWithProjectionUsingIcebergReaderUnsafe(Blackhole blackhole) throws IOException { + try (CloseableIterable rows = + Parquet.read(Files.localInput(dataFile)) + .project(PROJECTED_SCHEMA) + .createReaderFunc(type -> SparkParquetReaders.buildReader(PROJECTED_SCHEMA, type)) + .build()) { + + Iterable unsafeRows = + Iterables.transform( + rows, + APPLY_PROJECTION.bind( + SparkBenchmarkUtil.projection(PROJECTED_SCHEMA, PROJECTED_SCHEMA)) + ::invoke); + + for (InternalRow row : unsafeRows) { + blackhole.consume(row); + } + } + } + + @Benchmark + @Threads(1) + public void readWithProjectionUsingSparkReader(Blackhole blackhole) throws IOException { + StructType sparkSchema = SparkSchemaUtil.convert(PROJECTED_SCHEMA); + try (CloseableIterable rows = + Parquet.read(Files.localInput(dataFile)) + .project(PROJECTED_SCHEMA) + .readSupport(new ParquetReadSupport()) + .set("org.apache.spark.sql.parquet.row.requested_schema", sparkSchema.json()) + .set("spark.sql.parquet.binaryAsString", "false") + .set("spark.sql.parquet.int96AsTimestamp", "false") + .set("spark.sql.caseSensitive", "false") + .set("spark.sql.parquet.inferTimestampNTZ.enabled", "false") + .set("spark.sql.legacy.parquet.nanosAsLong", "false") + .callInit() + .build()) { + + for (InternalRow row : rows) { + blackhole.consume(row); + } + } + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java new file mode 100644 index 000000000000..47f0b72088f5 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java @@ -0,0 +1,130 @@ +/* + * 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.data.parquet; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.File; +import java.io.IOException; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.SparkParquetWriters; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport; +import org.apache.spark.sql.types.StructType; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; + +/** + * A benchmark that evaluates the performance of writing Parquet data with a flat schema using + * Iceberg and Spark Parquet writers. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=SparkParquetWritersFlatDataBenchmark + * -PjmhOutputPath=benchmark/spark-parquet-writers-flat-data-benchmark-result.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class SparkParquetWritersFlatDataBenchmark { + + private static final Schema SCHEMA = + new Schema( + required(1, "longCol", Types.LongType.get()), + required(2, "intCol", Types.IntegerType.get()), + required(3, "floatCol", Types.FloatType.get()), + optional(4, "doubleCol", Types.DoubleType.get()), + optional(5, "decimalCol", Types.DecimalType.of(20, 5)), + optional(6, "dateCol", Types.DateType.get()), + optional(7, "timestampCol", Types.TimestampType.withZone()), + optional(8, "stringCol", Types.StringType.get())); + private static final int NUM_RECORDS = 1000000; + private Iterable rows; + private File dataFile; + + @Setup + public void setupBenchmark() throws IOException { + rows = RandomData.generateSpark(SCHEMA, NUM_RECORDS, 0L); + dataFile = File.createTempFile("parquet-flat-data-benchmark", ".parquet"); + dataFile.delete(); + } + + @TearDown(Level.Iteration) + public void tearDownBenchmark() { + if (dataFile != null) { + dataFile.delete(); + } + } + + @Benchmark + @Threads(1) + public void writeUsingIcebergWriter() throws IOException { + try (FileAppender writer = + Parquet.write(Files.localOutput(dataFile)) + .createWriterFunc( + msgType -> + SparkParquetWriters.buildWriter(SparkSchemaUtil.convert(SCHEMA), msgType)) + .schema(SCHEMA) + .build()) { + + writer.addAll(rows); + } + } + + @Benchmark + @Threads(1) + public void writeUsingSparkWriter() throws IOException { + StructType sparkSchema = SparkSchemaUtil.convert(SCHEMA); + try (FileAppender writer = + Parquet.write(Files.localOutput(dataFile)) + .writeSupport(new ParquetWriteSupport()) + .set("org.apache.spark.sql.parquet.row.attributes", sparkSchema.json()) + .set("spark.sql.parquet.writeLegacyFormat", "false") + .set("spark.sql.parquet.binaryAsString", "false") + .set("spark.sql.parquet.int96AsTimestamp", "false") + .set("spark.sql.parquet.outputTimestampType", "TIMESTAMP_MICROS") + .set("spark.sql.caseSensitive", "false") + .set("spark.sql.parquet.fieldId.write.enabled", "false") + .schema(SCHEMA) + .build()) { + + writer.addAll(rows); + } + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java new file mode 100644 index 000000000000..4df890d86164 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java @@ -0,0 +1,130 @@ +/* + * 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.data.parquet; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.File; +import java.io.IOException; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.SparkParquetWriters; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport; +import org.apache.spark.sql.types.StructType; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; + +/** + * A benchmark that evaluates the performance of writing nested Parquet data using Iceberg and Spark + * Parquet writers. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=SparkParquetWritersNestedDataBenchmark + * -PjmhOutputPath=benchmark/spark-parquet-writers-nested-data-benchmark-result.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class SparkParquetWritersNestedDataBenchmark { + + private static final Schema SCHEMA = + new Schema( + required(0, "id", Types.LongType.get()), + optional( + 4, + "nested", + Types.StructType.of( + required(1, "col1", Types.StringType.get()), + required(2, "col2", Types.DoubleType.get()), + required(3, "col3", Types.LongType.get())))); + private static final int NUM_RECORDS = 1000000; + private Iterable rows; + private File dataFile; + + @Setup + public void setupBenchmark() throws IOException { + rows = RandomData.generateSpark(SCHEMA, NUM_RECORDS, 0L); + dataFile = File.createTempFile("parquet-nested-data-benchmark", ".parquet"); + dataFile.delete(); + } + + @TearDown(Level.Iteration) + public void tearDownBenchmark() { + if (dataFile != null) { + dataFile.delete(); + } + } + + @Benchmark + @Threads(1) + public void writeUsingIcebergWriter() throws IOException { + try (FileAppender writer = + Parquet.write(Files.localOutput(dataFile)) + .createWriterFunc( + msgType -> + SparkParquetWriters.buildWriter(SparkSchemaUtil.convert(SCHEMA), msgType)) + .schema(SCHEMA) + .build()) { + + writer.addAll(rows); + } + } + + @Benchmark + @Threads(1) + public void writeUsingSparkWriter() throws IOException { + StructType sparkSchema = SparkSchemaUtil.convert(SCHEMA); + try (FileAppender writer = + Parquet.write(Files.localOutput(dataFile)) + .writeSupport(new ParquetWriteSupport()) + .set("org.apache.spark.sql.parquet.row.attributes", sparkSchema.json()) + .set("spark.sql.parquet.writeLegacyFormat", "false") + .set("spark.sql.parquet.binaryAsString", "false") + .set("spark.sql.parquet.int96AsTimestamp", "false") + .set("spark.sql.parquet.outputTimestampType", "TIMESTAMP_MICROS") + .set("spark.sql.caseSensitive", "false") + .set("spark.sql.parquet.fieldId.write.enabled", "false") + .schema(SCHEMA) + .build()) { + + writer.addAll(rows); + } + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/Action.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/Action.java new file mode 100644 index 000000000000..0dbf07285060 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/Action.java @@ -0,0 +1,24 @@ +/* + * 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; + +@FunctionalInterface +public interface Action { + void invoke(); +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java new file mode 100644 index 000000000000..c6794e43c636 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/DVReaderBenchmark.java @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import com.google.errorprone.annotations.FormatMethod; +import com.google.errorprone.annotations.FormatString; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileGenerationUtil; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.BaseDeleteLoader; +import org.apache.iceberg.data.DeleteLoader; +import org.apache.iceberg.deletes.BaseDVFileWriter; +import org.apache.iceberg.deletes.DVFileWriter; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.iceberg.io.DeleteWriteResult; +import org.apache.iceberg.io.FanoutPositionOnlyDeleteWriter; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +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.Sets; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.util.ContentFileUtil; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.unsafe.types.UTF8String; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * A benchmark that compares the performance of DV and position delete readers. + * + *

    To run this benchmark for spark-3.5: + * ./gradlew -DsparkVersions=3.5 :iceberg-spark:iceberg-spark-3.5_2.12:jmh + * -PjmhIncludeRegex=DVReaderBenchmark + * -PjmhOutputPath=benchmark/iceberg-dv-reader-benchmark-result.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 15) +@Timeout(time = 20, timeUnit = TimeUnit.MINUTES) +@BenchmarkMode(Mode.SingleShotTime) +public class DVReaderBenchmark { + + private static final String TABLE_NAME = "test_table"; + private static final int DATA_FILE_RECORD_COUNT = 2_000_000; + private static final long TARGET_FILE_SIZE = Long.MAX_VALUE; + + @Param({"5", "10"}) + private int referencedDataFileCount; + + @Param({"0.01", "0.03", "0.05", "0.10", "0.2"}) + private double deletedRowsRatio; + + private final Configuration hadoopConf = new Configuration(); + private final Random random = ThreadLocalRandom.current(); + private SparkSession spark; + private Table table; + private DeleteWriteResult dvsResult; + private DeleteWriteResult fileDeletesResult; + private DeleteWriteResult partitionDeletesResult; + + @Setup + public void setupBenchmark() throws NoSuchTableException, ParseException, IOException { + setupSpark(); + initTable(); + List deletes = generatePositionDeletes(); + this.dvsResult = writeDVs(deletes); + this.fileDeletesResult = writePositionDeletes(deletes, DeleteGranularity.FILE); + this.partitionDeletesResult = writePositionDeletes(deletes, DeleteGranularity.PARTITION); + } + + @TearDown + public void tearDownBenchmark() { + dropTable(); + tearDownSpark(); + } + + @Benchmark + @Threads(1) + public void dv(Blackhole blackhole) { + DeleteLoader loader = new BaseDeleteLoader(file -> table.io().newInputFile(file), null); + DeleteFile dv = dvsResult.deleteFiles().get(0); + CharSequence dataFile = dv.referencedDataFile(); + PositionDeleteIndex index = loader.loadPositionDeletes(ImmutableList.of(dv), dataFile); + blackhole.consume(index); + } + + @Benchmark + @Threads(1) + public void fileScopedParquetDeletes(Blackhole blackhole) { + DeleteLoader loader = new BaseDeleteLoader(file -> table.io().newInputFile(file), null); + DeleteFile deleteFile = fileDeletesResult.deleteFiles().get(0); + CharSequence dataFile = ContentFileUtil.referencedDataFile(deleteFile); + PositionDeleteIndex index = loader.loadPositionDeletes(ImmutableList.of(deleteFile), dataFile); + blackhole.consume(index); + } + + @Benchmark + @Threads(1) + public void partitionScopedParquetDeletes(Blackhole blackhole) { + DeleteLoader loader = new BaseDeleteLoader(file -> table.io().newInputFile(file), null); + DeleteFile deleteFile = Iterables.getOnlyElement(partitionDeletesResult.deleteFiles()); + CharSequence dataFile = Iterables.getLast(partitionDeletesResult.referencedDataFiles()); + PositionDeleteIndex index = loader.loadPositionDeletes(ImmutableList.of(deleteFile), dataFile); + blackhole.consume(index); + } + + private FanoutPositionOnlyDeleteWriter newWriter(DeleteGranularity granularity) { + return new FanoutPositionOnlyDeleteWriter<>( + newWriterFactory(), + newFileFactory(FileFormat.PARQUET), + table.io(), + TARGET_FILE_SIZE, + granularity); + } + + private SparkFileWriterFactory newWriterFactory() { + return SparkFileWriterFactory.builderFor(table).dataFileFormat(FileFormat.PARQUET).build(); + } + + private OutputFileFactory newFileFactory(FileFormat format) { + return OutputFileFactory.builderFor(table, 1, 1).format(format).build(); + } + + private List generatePositionDeletes() { + int numDeletesPerFile = (int) (DATA_FILE_RECORD_COUNT * deletedRowsRatio); + int numDeletes = referencedDataFileCount * numDeletesPerFile; + List deletes = Lists.newArrayListWithExpectedSize(numDeletes); + + for (int pathIndex = 0; pathIndex < referencedDataFileCount; pathIndex++) { + UTF8String dataFilePath = UTF8String.fromString(generateDataFilePath()); + Set positions = generatePositions(numDeletesPerFile); + for (long pos : positions) { + deletes.add(new GenericInternalRow(new Object[] {dataFilePath, pos})); + } + } + + Collections.shuffle(deletes); + + return deletes; + } + + private DeleteWriteResult writeDVs(Iterable rows) throws IOException { + OutputFileFactory fileFactory = newFileFactory(FileFormat.PUFFIN); + DVFileWriter writer = new BaseDVFileWriter(fileFactory, path -> null); + try (DVFileWriter closableWriter = writer) { + for (InternalRow row : rows) { + String path = row.getString(0); + long pos = row.getLong(1); + closableWriter.delete(path, pos, table.spec(), null); + } + } + return writer.result(); + } + + private DeleteWriteResult writePositionDeletes( + Iterable rows, DeleteGranularity granularity) throws IOException { + FanoutPositionOnlyDeleteWriter writer = newWriter(granularity); + try (FanoutPositionOnlyDeleteWriter closableWriter = writer) { + PositionDelete positionDelete = PositionDelete.create(); + for (InternalRow row : rows) { + String path = row.getString(0); + long pos = row.getLong(1); + positionDelete.set(path, pos, null /* no row */); + closableWriter.write(positionDelete, table.spec(), null); + } + } + return writer.result(); + } + + public Set generatePositions(int numPositions) { + Set positions = Sets.newHashSet(); + + while (positions.size() < numPositions) { + long pos = random.nextInt(DATA_FILE_RECORD_COUNT); + positions.add(pos); + } + + return positions; + } + + private String generateDataFilePath() { + String fileName = FileGenerationUtil.generateFileName(); + return table.locationProvider().newDataLocation(table.spec(), null, fileName); + } + + private void setupSpark() { + this.spark = + SparkSession.builder() + .config("spark.ui.enabled", false) + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) + .master("local[*]") + .getOrCreate(); + } + + private void tearDownSpark() { + spark.stop(); + } + + private void initTable() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (c1 INT, c2 INT, c3 STRING) USING iceberg", TABLE_NAME); + this.table = Spark3Util.loadIcebergTable(spark, TABLE_NAME); + } + + private void dropTable() { + sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME); + } + + private String newWarehouseDir() { + return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID(); + } + + @FormatMethod + private void sql(@FormatString String query, Object... args) { + spark.sql(String.format(query, args)); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java new file mode 100644 index 000000000000..ac74fb5a109c --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/DVWriterBenchmark.java @@ -0,0 +1,251 @@ +/* + * 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 com.google.errorprone.annotations.FormatMethod; +import com.google.errorprone.annotations.FormatString; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileGenerationUtil; +import org.apache.iceberg.Table; +import org.apache.iceberg.deletes.BaseDVFileWriter; +import org.apache.iceberg.deletes.DVFileWriter; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.io.DeleteWriteResult; +import org.apache.iceberg.io.FanoutPositionOnlyDeleteWriter; +import org.apache.iceberg.io.OutputFileFactory; +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.SparkSessionCatalog; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.unsafe.types.UTF8String; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +/** + * A benchmark that compares the performance of DV and position delete writers. + * + *

    To run this benchmark for spark-3.5: + * ./gradlew -DsparkVersions=3.5 :iceberg-spark:iceberg-spark-3.5_2.12:jmh + * -PjmhIncludeRegex=DVWriterBenchmark + * -PjmhOutputPath=benchmark/iceberg-dv-writer-benchmark-result.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 10) +@Timeout(time = 20, timeUnit = TimeUnit.MINUTES) +@BenchmarkMode(Mode.SingleShotTime) +public class DVWriterBenchmark { + + private static final String TABLE_NAME = "test_table"; + private static final int DATA_FILE_RECORD_COUNT = 2_000_000; + private static final long TARGET_FILE_SIZE = Long.MAX_VALUE; + + @Param({"5", "10"}) + private int referencedDataFileCount; + + @Param({"0.01", "0.03", "0.05", "0.10", "0.2"}) + private double deletedRowsRatio; + + private final Configuration hadoopConf = new Configuration(); + private final Random random = ThreadLocalRandom.current(); + private SparkSession spark; + private Table table; + private Iterable positionDeletes; + + @Setup + public void setupBenchmark() throws NoSuchTableException, ParseException { + setupSpark(); + initTable(); + generatePositionDeletes(); + } + + @TearDown + public void tearDownBenchmark() { + dropTable(); + tearDownSpark(); + } + + @Benchmark + @Threads(1) + public void dv(Blackhole blackhole) throws IOException { + OutputFileFactory fileFactory = newFileFactory(FileFormat.PUFFIN); + DVFileWriter writer = new BaseDVFileWriter(fileFactory, path -> null); + + try (DVFileWriter closableWriter = writer) { + for (InternalRow row : positionDeletes) { + String path = row.getString(0); + long pos = row.getLong(1); + closableWriter.delete(path, pos, table.spec(), null); + } + } + + DeleteWriteResult result = writer.result(); + blackhole.consume(result); + } + + @Benchmark + @Threads(1) + public void fileScopedParquetDeletes(Blackhole blackhole) throws IOException { + FanoutPositionOnlyDeleteWriter writer = newWriter(DeleteGranularity.FILE); + write(writer, positionDeletes); + DeleteWriteResult result = writer.result(); + blackhole.consume(result); + } + + @Benchmark + @Threads(1) + public void partitionScopedParquetDeletes(Blackhole blackhole) throws IOException { + FanoutPositionOnlyDeleteWriter writer = newWriter(DeleteGranularity.PARTITION); + write(writer, positionDeletes); + DeleteWriteResult result = writer.result(); + blackhole.consume(result); + } + + private FanoutPositionOnlyDeleteWriter newWriter(DeleteGranularity granularity) { + return new FanoutPositionOnlyDeleteWriter<>( + newWriterFactory(), + newFileFactory(FileFormat.PARQUET), + table.io(), + TARGET_FILE_SIZE, + granularity); + } + + private DeleteWriteResult write( + FanoutPositionOnlyDeleteWriter writer, Iterable rows) + throws IOException { + + try (FanoutPositionOnlyDeleteWriter closableWriter = writer) { + PositionDelete positionDelete = PositionDelete.create(); + + for (InternalRow row : rows) { + String path = row.getString(0); + long pos = row.getLong(1); + positionDelete.set(path, pos, null /* no row */); + closableWriter.write(positionDelete, table.spec(), null); + } + } + + return writer.result(); + } + + private SparkFileWriterFactory newWriterFactory() { + return SparkFileWriterFactory.builderFor(table).dataFileFormat(FileFormat.PARQUET).build(); + } + + private OutputFileFactory newFileFactory(FileFormat format) { + return OutputFileFactory.builderFor(table, 1, 1).format(format).build(); + } + + private void generatePositionDeletes() { + int numDeletesPerFile = (int) (DATA_FILE_RECORD_COUNT * deletedRowsRatio); + int numDeletes = referencedDataFileCount * numDeletesPerFile; + List deletes = Lists.newArrayListWithExpectedSize(numDeletes); + + for (int pathIndex = 0; pathIndex < referencedDataFileCount; pathIndex++) { + UTF8String dataFilePath = UTF8String.fromString(generateDataFilePath()); + Set positions = generatePositions(numDeletesPerFile); + for (long pos : positions) { + deletes.add(new GenericInternalRow(new Object[] {dataFilePath, pos})); + } + } + + Collections.shuffle(deletes); + + this.positionDeletes = deletes; + } + + public Set generatePositions(int numPositions) { + Set positions = Sets.newHashSet(); + + while (positions.size() < numPositions) { + long pos = random.nextInt(DATA_FILE_RECORD_COUNT); + positions.add(pos); + } + + return positions; + } + + private String generateDataFilePath() { + String fileName = FileGenerationUtil.generateFileName(); + return table.locationProvider().newDataLocation(table.spec(), null, fileName); + } + + private void setupSpark() { + this.spark = + SparkSession.builder() + .config("spark.ui.enabled", false) + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config("spark.sql.catalog.spark_catalog", SparkSessionCatalog.class.getName()) + .config("spark.sql.catalog.spark_catalog.type", "hadoop") + .config("spark.sql.catalog.spark_catalog.warehouse", newWarehouseDir()) + .master("local[*]") + .getOrCreate(); + } + + private void tearDownSpark() { + spark.stop(); + } + + private void initTable() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (c1 INT, c2 INT, c3 STRING) USING iceberg", TABLE_NAME); + this.table = Spark3Util.loadIcebergTable(spark, TABLE_NAME); + } + + private void dropTable() { + sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME); + } + + private String newWarehouseDir() { + return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID(); + } + + @FormatMethod + private void sql(@FormatString String query, Object... args) { + spark.sql(String.format(query, args)); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java new file mode 100644 index 000000000000..68c537e34a4a --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceBenchmark.java @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.io.IOException; +import java.util.Map; +import java.util.UUID; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.UpdateProperties; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.StructType; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public abstract class IcebergSourceBenchmark { + + private final Configuration hadoopConf = initHadoopConf(); + private final Table table = initTable(); + private SparkSession spark; + + protected abstract Configuration initHadoopConf(); + + protected final Configuration hadoopConf() { + return hadoopConf; + } + + protected abstract Table initTable(); + + protected final Table table() { + return table; + } + + protected final SparkSession spark() { + return spark; + } + + protected String newTableLocation() { + String tmpDir = hadoopConf.get("hadoop.tmp.dir"); + Path tablePath = new Path(tmpDir, "spark-iceberg-table-" + UUID.randomUUID()); + return tablePath.toString(); + } + + protected String dataLocation() { + Map properties = table.properties(); + return properties.getOrDefault( + TableProperties.WRITE_DATA_LOCATION, String.format("%s/data", table.location())); + } + + protected void cleanupFiles() throws IOException { + try (FileSystem fileSystem = FileSystem.get(hadoopConf)) { + Path dataPath = new Path(dataLocation()); + fileSystem.delete(dataPath, true); + Path tablePath = new Path(table.location()); + fileSystem.delete(tablePath, true); + } + } + + protected void setupSpark(boolean enableDictionaryEncoding) { + SparkSession.Builder builder = SparkSession.builder().config("spark.ui.enabled", false); + if (!enableDictionaryEncoding) { + builder + .config("parquet.dictionary.page.size", "1") + .config("parquet.enable.dictionary", false) + .config(TableProperties.PARQUET_DICT_SIZE_BYTES, "1"); + } + builder.master("local"); + spark = builder.getOrCreate(); + Configuration sparkHadoopConf = spark.sessionState().newHadoopConf(); + hadoopConf.forEach(entry -> sparkHadoopConf.set(entry.getKey(), entry.getValue())); + } + + protected void setupSpark() { + setupSpark(false); + } + + protected void tearDownSpark() { + spark.stop(); + } + + protected void materialize(Dataset ds) { + ds.queryExecution().toRdd().toJavaRDD().foreach(record -> {}); + } + + protected void materialize(Dataset ds, Blackhole blackhole) { + blackhole.consume(ds.queryExecution().toRdd().toJavaRDD().count()); + } + + protected void appendAsFile(Dataset ds) { + // ensure the schema is precise (including nullability) + StructType sparkSchema = SparkSchemaUtil.convert(table.schema()); + spark + .createDataFrame(ds.rdd(), sparkSchema) + .coalesce(1) + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(table.location()); + } + + protected void withSQLConf(Map conf, Action action) { + SQLConf sqlConf = SQLConf.get(); + + Map currentConfValues = Maps.newHashMap(); + conf.keySet() + .forEach( + confKey -> { + if (sqlConf.contains(confKey)) { + String currentConfValue = sqlConf.getConfString(confKey); + currentConfValues.put(confKey, currentConfValue); + } + }); + + conf.forEach( + (confKey, confValue) -> { + if (SQLConf.isStaticConfigKey(confKey)) { + throw new RuntimeException("Cannot modify the value of a static config: " + confKey); + } + sqlConf.setConfString(confKey, confValue); + }); + + try { + action.invoke(); + } finally { + conf.forEach( + (confKey, confValue) -> { + if (currentConfValues.containsKey(confKey)) { + sqlConf.setConfString(confKey, currentConfValues.get(confKey)); + } else { + sqlConf.unsetConf(confKey); + } + }); + } + } + + protected void withTableProperties(Map props, Action action) { + Map tableProps = table.properties(); + Map currentPropValues = Maps.newHashMap(); + props + .keySet() + .forEach( + propKey -> { + if (tableProps.containsKey(propKey)) { + String currentPropValue = tableProps.get(propKey); + currentPropValues.put(propKey, currentPropValue); + } + }); + + UpdateProperties updateProperties = table.updateProperties(); + props.forEach(updateProperties::set); + updateProperties.commit(); + + try { + action.invoke(); + } finally { + UpdateProperties restoreProperties = table.updateProperties(); + props.forEach( + (propKey, propValue) -> { + if (currentPropValues.containsKey(propKey)) { + restoreProperties.set(propKey, currentPropValues.get(propKey)); + } else { + restoreProperties.remove(propKey); + } + }); + restoreProperties.commit(); + } + } + + protected FileFormat fileFormat() { + throw new UnsupportedOperationException("Unsupported file format"); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceDeleteBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceDeleteBenchmark.java new file mode 100644 index 000000000000..e42707bf102b --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceDeleteBenchmark.java @@ -0,0 +1,340 @@ +/* + * 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.PARQUET_VECTORIZATION_ENABLED; +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.apache.spark.sql.functions.current_date; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.ClusteredEqualityDeleteWriter; +import org.apache.iceberg.io.ClusteredPositionDeleteWriter; +import org.apache.iceberg.io.OutputFileFactory; +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.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.infra.Blackhole; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class IcebergSourceDeleteBenchmark extends IcebergSourceBenchmark { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSourceDeleteBenchmark.class); + private static final long TARGET_FILE_SIZE_IN_BYTES = 512L * 1024 * 1024; + + protected static final int NUM_FILES = 1; + protected static final int NUM_ROWS = 10 * 1000 * 1000; + + @Setup + public void setupBenchmark() throws IOException { + setupSpark(); + appendData(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Benchmark + @Threads(1) + public void readIceberg(Blackhole blackhole) { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + tableProperties.put(PARQUET_VECTORIZATION_ENABLED, "false"); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation); + materialize(df, blackhole); + }); + } + + @Benchmark + @Threads(1) + public void readIcebergWithIsDeletedColumn(Blackhole blackhole) { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + tableProperties.put(PARQUET_VECTORIZATION_ENABLED, "false"); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).filter("_deleted = false"); + materialize(df, blackhole); + }); + } + + @Benchmark + @Threads(1) + public void readDeletedRows(Blackhole blackhole) { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + tableProperties.put(PARQUET_VECTORIZATION_ENABLED, "false"); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).filter("_deleted = true"); + materialize(df, blackhole); + }); + } + + @Benchmark + @Threads(1) + public void readIcebergVectorized(Blackhole blackhole) { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + tableProperties.put(PARQUET_VECTORIZATION_ENABLED, "true"); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation); + materialize(df, blackhole); + }); + } + + @Benchmark + @Threads(1) + public void readIcebergWithIsDeletedColumnVectorized(Blackhole blackhole) { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + tableProperties.put(PARQUET_VECTORIZATION_ENABLED, "true"); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).filter("_deleted = false"); + materialize(df, blackhole); + }); + } + + @Benchmark + @Threads(1) + public void readDeletedRowsVectorized(Blackhole blackhole) { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + tableProperties.put(PARQUET_VECTORIZATION_ENABLED, "true"); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).filter("_deleted = true"); + materialize(df, blackhole); + }); + } + + protected abstract void appendData() throws IOException; + + protected void writeData(int fileNum) { + Dataset df = + spark() + .range(NUM_ROWS) + .withColumnRenamed("id", "longCol") + .withColumn("intCol", expr("CAST(MOD(longCol, 2147483647) AS INT)")) + .withColumn("floatCol", expr("CAST(longCol AS FLOAT)")) + .withColumn("doubleCol", expr("CAST(longCol AS DOUBLE)")) + .withColumn("dateCol", date_add(current_date(), fileNum)) + .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)")) + .withColumn("stringCol", expr("CAST(dateCol AS STRING)")); + appendAsFile(df); + } + + @Override + protected Table initTable() { + Schema schema = + new Schema( + required(1, "longCol", Types.LongType.get()), + required(2, "intCol", Types.IntegerType.get()), + required(3, "floatCol", Types.FloatType.get()), + optional(4, "doubleCol", Types.DoubleType.get()), + optional(6, "dateCol", Types.DateType.get()), + optional(7, "timestampCol", Types.TimestampType.withZone()), + optional(8, "stringCol", Types.StringType.get())); + PartitionSpec partitionSpec = PartitionSpec.unpartitioned(); + HadoopTables tables = new HadoopTables(hadoopConf()); + Map properties = Maps.newHashMap(); + properties.put(TableProperties.METADATA_COMPRESSION, "gzip"); + properties.put(TableProperties.FORMAT_VERSION, "2"); + return tables.create(schema, partitionSpec, properties, newTableLocation()); + } + + @Override + protected Configuration initHadoopConf() { + return new Configuration(); + } + + protected void writePosDeletes(CharSequence path, long numRows, double percentage) + throws IOException { + writePosDeletes(path, numRows, percentage, 1); + } + + protected void writePosDeletes( + CharSequence path, long numRows, double percentage, int numDeleteFile) throws IOException { + writePosDeletesWithNoise(path, numRows, percentage, 0, numDeleteFile); + } + + protected void writePosDeletesWithNoise( + CharSequence path, long numRows, double percentage, int numNoise, int numDeleteFile) + throws IOException { + Set deletedPos = Sets.newHashSet(); + while (deletedPos.size() < numRows * percentage) { + deletedPos.add(ThreadLocalRandom.current().nextLong(numRows)); + } + LOG.info("pos delete row count: {}, num of delete files: {}", deletedPos.size(), numDeleteFile); + + int partitionSize = (int) (numRows * percentage) / numDeleteFile; + Iterable> sets = Iterables.partition(deletedPos, partitionSize); + for (List item : sets) { + writePosDeletes(path, item, numNoise); + } + } + + protected void writePosDeletes(CharSequence path, List deletedPos, int numNoise) + throws IOException { + OutputFileFactory fileFactory = newFileFactory(); + SparkFileWriterFactory writerFactory = + SparkFileWriterFactory.builderFor(table()).dataFileFormat(fileFormat()).build(); + + ClusteredPositionDeleteWriter writer = + new ClusteredPositionDeleteWriter<>( + writerFactory, fileFactory, table().io(), TARGET_FILE_SIZE_IN_BYTES); + + PartitionSpec unpartitionedSpec = table().specs().get(0); + + PositionDelete positionDelete = PositionDelete.create(); + try (ClusteredPositionDeleteWriter closeableWriter = writer) { + for (Long pos : deletedPos) { + positionDelete.set(path, pos, null); + closeableWriter.write(positionDelete, unpartitionedSpec, null); + for (int i = 0; i < numNoise; i++) { + positionDelete.set(noisePath(path), pos, null); + closeableWriter.write(positionDelete, unpartitionedSpec, null); + } + } + } + + RowDelta rowDelta = table().newRowDelta(); + writer.result().deleteFiles().forEach(rowDelta::addDeletes); + rowDelta.validateDeletedFiles().commit(); + } + + protected void writeEqDeletes(long numRows, double percentage) throws IOException { + Set deletedValues = Sets.newHashSet(); + while (deletedValues.size() < numRows * percentage) { + deletedValues.add(ThreadLocalRandom.current().nextLong(numRows)); + } + + List rows = Lists.newArrayList(); + for (Long value : deletedValues) { + GenericInternalRow genericInternalRow = new GenericInternalRow(7); + genericInternalRow.setLong(0, value); + genericInternalRow.setInt(1, (int) (value % Integer.MAX_VALUE)); + genericInternalRow.setFloat(2, (float) value); + genericInternalRow.setNullAt(3); + genericInternalRow.setNullAt(4); + genericInternalRow.setNullAt(5); + genericInternalRow.setNullAt(6); + rows.add(genericInternalRow); + } + LOG.info("Num of equality deleted rows: {}", rows.size()); + + writeEqDeletes(rows); + } + + private void writeEqDeletes(List rows) throws IOException { + int equalityFieldId = table().schema().findField("longCol").fieldId(); + + OutputFileFactory fileFactory = newFileFactory(); + SparkFileWriterFactory writerFactory = + SparkFileWriterFactory.builderFor(table()) + .dataFileFormat(fileFormat()) + .equalityDeleteRowSchema(table().schema()) + .equalityFieldIds(new int[] {equalityFieldId}) + .build(); + + ClusteredEqualityDeleteWriter writer = + new ClusteredEqualityDeleteWriter<>( + writerFactory, fileFactory, table().io(), TARGET_FILE_SIZE_IN_BYTES); + + PartitionSpec unpartitionedSpec = table().specs().get(0); + try (ClusteredEqualityDeleteWriter closeableWriter = writer) { + for (InternalRow row : rows) { + closeableWriter.write(row, unpartitionedSpec, null); + } + } + + RowDelta rowDelta = table().newRowDelta(); + LOG.info("Num of Delete File: {}", writer.result().deleteFiles().size()); + writer.result().deleteFiles().forEach(rowDelta::addDeletes); + rowDelta.validateDeletedFiles().commit(); + } + + private OutputFileFactory newFileFactory() { + return OutputFileFactory.builderFor(table(), 1, 1).format(fileFormat()).build(); + } + + private CharSequence noisePath(CharSequence path) { + // assume the data file name would be something like + // "00000-0-30da64e0-56b5-4743-a11b-3188a1695bf7-00001.parquet" + // so the dataFileSuffixLen is the UUID string length + length of "-00001.parquet", which is 36 + // + 14 = 60. It's OK + // to be not accurate here. + int dataFileSuffixLen = 60; + UUID uuid = UUID.randomUUID(); + if (path.length() > dataFileSuffixLen) { + return path.subSequence(0, dataFileSuffixLen) + uuid.toString(); + } else { + return uuid.toString(); + } + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceFlatDataBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceFlatDataBenchmark.java new file mode 100644 index 000000000000..59e6230350d9 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceFlatDataBenchmark.java @@ -0,0 +1,59 @@ +/* + * 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.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +public abstract class IcebergSourceFlatDataBenchmark extends IcebergSourceBenchmark { + + @Override + protected Configuration initHadoopConf() { + return new Configuration(); + } + + @Override + protected final Table initTable() { + Schema schema = + new Schema( + required(1, "longCol", Types.LongType.get()), + required(2, "intCol", Types.IntegerType.get()), + required(3, "floatCol", Types.FloatType.get()), + optional(4, "doubleCol", Types.DoubleType.get()), + optional(5, "decimalCol", Types.DecimalType.of(20, 5)), + optional(6, "dateCol", Types.DateType.get()), + optional(7, "timestampCol", Types.TimestampType.withZone()), + optional(8, "stringCol", Types.StringType.get())); + PartitionSpec partitionSpec = PartitionSpec.unpartitioned(); + HadoopTables tables = new HadoopTables(hadoopConf()); + Map properties = Maps.newHashMap(); + properties.put(TableProperties.METADATA_COMPRESSION, "gzip"); + return tables.create(schema, partitionSpec, properties, newTableLocation()); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedDataBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedDataBenchmark.java new file mode 100644 index 000000000000..a1c61b9b4de0 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedDataBenchmark.java @@ -0,0 +1,59 @@ +/* + * 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.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +public abstract class IcebergSourceNestedDataBenchmark extends IcebergSourceBenchmark { + + @Override + protected Configuration initHadoopConf() { + return new Configuration(); + } + + @Override + protected final Table initTable() { + Schema schema = + new Schema( + required(0, "id", Types.LongType.get()), + optional( + 4, + "nested", + Types.StructType.of( + required(1, "col1", Types.StringType.get()), + required(2, "col2", Types.DoubleType.get()), + required(3, "col3", Types.LongType.get())))); + PartitionSpec partitionSpec = PartitionSpec.unpartitioned(); + HadoopTables tables = new HadoopTables(hadoopConf()); + Map properties = Maps.newHashMap(); + properties.put(TableProperties.METADATA_COMPRESSION, "gzip"); + return tables.create(schema, partitionSpec, properties, newTableLocation()); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedListDataBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedListDataBenchmark.java new file mode 100644 index 000000000000..f68b587735dd --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/IcebergSourceNestedListDataBenchmark.java @@ -0,0 +1,62 @@ +/* + * 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.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +public abstract class IcebergSourceNestedListDataBenchmark extends IcebergSourceBenchmark { + + @Override + protected Configuration initHadoopConf() { + return new Configuration(); + } + + @Override + protected final Table initTable() { + Schema schema = + new Schema( + required(0, "id", Types.LongType.get()), + optional( + 1, + "outerlist", + Types.ListType.ofOptional( + 2, + Types.StructType.of( + required( + 3, + "innerlist", + Types.ListType.ofRequired(4, Types.StringType.get())))))); + PartitionSpec partitionSpec = PartitionSpec.unpartitioned(); + HadoopTables tables = new HadoopTables(hadoopConf()); + Map properties = Maps.newHashMap(); + properties.put(TableProperties.METADATA_COMPRESSION, "gzip"); + return tables.create(schema, partitionSpec, properties, newTableLocation()); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java new file mode 100644 index 000000000000..06efb6ba20e4 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/WritersBenchmark.java @@ -0,0 +1,393 @@ +/* + * 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.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.ClusteredDataWriter; +import org.apache.iceberg.io.ClusteredEqualityDeleteWriter; +import org.apache.iceberg.io.ClusteredPositionDeleteWriter; +import org.apache.iceberg.io.FanoutDataWriter; +import org.apache.iceberg.io.FanoutPositionOnlyDeleteWriter; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +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.Maps; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.transforms.Transform; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.infra.Blackhole; + +public abstract class WritersBenchmark extends IcebergSourceBenchmark { + + private static final int NUM_ROWS = 2500000; + private static final int NUM_DATA_FILES_PER_POSITION_DELETE_FILE = 100; + private static final int NUM_DELETED_POSITIONS_PER_DATA_FILE = 50_000; + private static final int DELETE_POSITION_STEP = 10; + private static final long TARGET_FILE_SIZE_IN_BYTES = 50L * 1024 * 1024; + + private static final Schema SCHEMA = + new Schema( + required(1, "longCol", Types.LongType.get()), + required(2, "intCol", Types.IntegerType.get()), + required(3, "floatCol", Types.FloatType.get()), + optional(4, "doubleCol", Types.DoubleType.get()), + optional(5, "decimalCol", Types.DecimalType.of(20, 5)), + optional(6, "timestampCol", Types.TimestampType.withZone()), + optional(7, "stringCol", Types.StringType.get())); + + private Iterable rows; + private Iterable positionDeleteRows; + private Iterable shuffledPositionDeleteRows; + private PartitionSpec unpartitionedSpec; + private PartitionSpec partitionedSpec; + + @Override + protected abstract FileFormat fileFormat(); + + @Setup + public void setupBenchmark() { + setupSpark(); + + List data = Lists.newArrayList(RandomData.generateSpark(SCHEMA, NUM_ROWS, 0L)); + Transform transform = Transforms.bucket(32); + data.sort( + Comparator.comparingInt( + row -> transform.bind(Types.IntegerType.get()).apply(row.getInt(1)))); + this.rows = data; + + this.positionDeleteRows = generatePositionDeletes(false /* no shuffle */); + this.shuffledPositionDeleteRows = generatePositionDeletes(true /* shuffle */); + + this.unpartitionedSpec = table().specs().get(0); + Preconditions.checkArgument(unpartitionedSpec.isUnpartitioned()); + this.partitionedSpec = table().specs().get(1); + } + + private Iterable generatePositionDeletes(boolean shuffle) { + int numDeletes = NUM_DATA_FILES_PER_POSITION_DELETE_FILE * NUM_DELETED_POSITIONS_PER_DATA_FILE; + List deletes = Lists.newArrayListWithExpectedSize(numDeletes); + + for (int pathIndex = 0; pathIndex < NUM_DATA_FILES_PER_POSITION_DELETE_FILE; pathIndex++) { + UTF8String path = UTF8String.fromString("path/to/position/delete/file/" + UUID.randomUUID()); + for (long pos = 0; pos < NUM_DELETED_POSITIONS_PER_DATA_FILE; pos++) { + deletes.add(new GenericInternalRow(new Object[] {path, pos * DELETE_POSITION_STEP})); + } + } + + if (shuffle) { + Collections.shuffle(deletes); + } + + return deletes; + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Override + protected Configuration initHadoopConf() { + return new Configuration(); + } + + @Override + protected final Table initTable() { + HadoopTables tables = new HadoopTables(hadoopConf()); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map properties = Maps.newHashMap(); + Table table = tables.create(SCHEMA, spec, properties, newTableLocation()); + + // add a partitioned spec to the table + table.updateSpec().addField(Expressions.bucket("intCol", 32)).commit(); + + return table; + } + + @Benchmark + @Threads(1) + public void writeUnpartitionedClusteredDataWriter(Blackhole blackhole) throws IOException { + FileIO io = table().io(); + + OutputFileFactory fileFactory = newFileFactory(); + SparkFileWriterFactory writerFactory = + SparkFileWriterFactory.builderFor(table()) + .dataFileFormat(fileFormat()) + .dataSchema(table().schema()) + .build(); + + ClusteredDataWriter writer = + new ClusteredDataWriter<>(writerFactory, fileFactory, io, TARGET_FILE_SIZE_IN_BYTES); + + try (ClusteredDataWriter closeableWriter = writer) { + for (InternalRow row : rows) { + closeableWriter.write(row, unpartitionedSpec, null); + } + } + + blackhole.consume(writer); + } + + @Benchmark + @Threads(1) + public void writePartitionedClusteredDataWriter(Blackhole blackhole) throws IOException { + FileIO io = table().io(); + + OutputFileFactory fileFactory = newFileFactory(); + SparkFileWriterFactory writerFactory = + SparkFileWriterFactory.builderFor(table()) + .dataFileFormat(fileFormat()) + .dataSchema(table().schema()) + .build(); + + ClusteredDataWriter writer = + new ClusteredDataWriter<>(writerFactory, fileFactory, io, TARGET_FILE_SIZE_IN_BYTES); + + PartitionKey partitionKey = new PartitionKey(partitionedSpec, table().schema()); + StructType dataSparkType = SparkSchemaUtil.convert(table().schema()); + InternalRowWrapper internalRowWrapper = + new InternalRowWrapper(dataSparkType, table().schema().asStruct()); + + try (ClusteredDataWriter closeableWriter = writer) { + for (InternalRow row : rows) { + partitionKey.partition(internalRowWrapper.wrap(row)); + closeableWriter.write(row, partitionedSpec, partitionKey); + } + } + + blackhole.consume(writer); + } + + @Benchmark + @Threads(1) + public void writePartitionedFanoutDataWriter(Blackhole blackhole) throws IOException { + FileIO io = table().io(); + + OutputFileFactory fileFactory = newFileFactory(); + SparkFileWriterFactory writerFactory = + SparkFileWriterFactory.builderFor(table()) + .dataFileFormat(fileFormat()) + .dataSchema(table().schema()) + .build(); + + FanoutDataWriter writer = + new FanoutDataWriter<>(writerFactory, fileFactory, io, TARGET_FILE_SIZE_IN_BYTES); + + PartitionKey partitionKey = new PartitionKey(partitionedSpec, table().schema()); + StructType dataSparkType = SparkSchemaUtil.convert(table().schema()); + InternalRowWrapper internalRowWrapper = + new InternalRowWrapper(dataSparkType, table().schema().asStruct()); + + try (FanoutDataWriter closeableWriter = writer) { + for (InternalRow row : rows) { + partitionKey.partition(internalRowWrapper.wrap(row)); + closeableWriter.write(row, partitionedSpec, partitionKey); + } + } + + blackhole.consume(writer); + } + + @Benchmark + @Threads(1) + public void writePartitionedClusteredEqualityDeleteWriter(Blackhole blackhole) + throws IOException { + FileIO io = table().io(); + + int equalityFieldId = table().schema().findField("longCol").fieldId(); + + OutputFileFactory fileFactory = newFileFactory(); + SparkFileWriterFactory writerFactory = + SparkFileWriterFactory.builderFor(table()) + .dataFileFormat(fileFormat()) + .equalityDeleteRowSchema(table().schema()) + .equalityFieldIds(new int[] {equalityFieldId}) + .build(); + + ClusteredEqualityDeleteWriter writer = + new ClusteredEqualityDeleteWriter<>( + writerFactory, fileFactory, io, TARGET_FILE_SIZE_IN_BYTES); + + PartitionKey partitionKey = new PartitionKey(partitionedSpec, table().schema()); + StructType deleteSparkType = SparkSchemaUtil.convert(table().schema()); + InternalRowWrapper internalRowWrapper = + new InternalRowWrapper(deleteSparkType, table().schema().asStruct()); + + try (ClusteredEqualityDeleteWriter closeableWriter = writer) { + for (InternalRow row : rows) { + partitionKey.partition(internalRowWrapper.wrap(row)); + closeableWriter.write(row, partitionedSpec, partitionKey); + } + } + + blackhole.consume(writer); + } + + @Benchmark + @Threads(1) + public void writeUnpartitionedClusteredPositionDeleteWriterPartitionGranularity( + Blackhole blackhole) throws IOException { + writeUnpartitionedClusteredPositionDeleteWriter(blackhole, DeleteGranularity.PARTITION); + } + + @Benchmark + @Threads(1) + public void writeUnpartitionedClusteredPositionDeleteWriterFileGranularity(Blackhole blackhole) + throws IOException { + writeUnpartitionedClusteredPositionDeleteWriter(blackhole, DeleteGranularity.FILE); + } + + private void writeUnpartitionedClusteredPositionDeleteWriter( + Blackhole blackhole, DeleteGranularity deleteGranularity) throws IOException { + FileIO io = table().io(); + + OutputFileFactory fileFactory = newFileFactory(); + SparkFileWriterFactory writerFactory = + SparkFileWriterFactory.builderFor(table()).dataFileFormat(fileFormat()).build(); + + ClusteredPositionDeleteWriter writer = + new ClusteredPositionDeleteWriter<>( + writerFactory, fileFactory, io, TARGET_FILE_SIZE_IN_BYTES, deleteGranularity); + + PositionDelete positionDelete = PositionDelete.create(); + try (ClusteredPositionDeleteWriter closeableWriter = writer) { + for (InternalRow row : positionDeleteRows) { + String path = row.getString(0); + long pos = row.getLong(1); + positionDelete.set(path, pos, null); + closeableWriter.write(positionDelete, unpartitionedSpec, null); + } + } + + blackhole.consume(writer); + } + + @Benchmark + @Threads(1) + public void writeUnpartitionedFanoutPositionDeleteWriterPartitionGranularity(Blackhole blackhole) + throws IOException { + writeUnpartitionedFanoutPositionDeleteWriterPartition(blackhole, DeleteGranularity.PARTITION); + } + + @Benchmark + @Threads(1) + public void writeUnpartitionedFanoutPositionDeleteWriterFileGranularity(Blackhole blackhole) + throws IOException { + writeUnpartitionedFanoutPositionDeleteWriterPartition(blackhole, DeleteGranularity.FILE); + } + + private void writeUnpartitionedFanoutPositionDeleteWriterPartition( + Blackhole blackhole, DeleteGranularity deleteGranularity) throws IOException { + FileIO io = table().io(); + + OutputFileFactory fileFactory = newFileFactory(); + SparkFileWriterFactory writerFactory = + SparkFileWriterFactory.builderFor(table()).dataFileFormat(fileFormat()).build(); + + FanoutPositionOnlyDeleteWriter writer = + new FanoutPositionOnlyDeleteWriter<>( + writerFactory, fileFactory, io, TARGET_FILE_SIZE_IN_BYTES, deleteGranularity); + + PositionDelete positionDelete = PositionDelete.create(); + try (FanoutPositionOnlyDeleteWriter closeableWriter = writer) { + for (InternalRow row : positionDeleteRows) { + String path = row.getString(0); + long pos = row.getLong(1); + positionDelete.set(path, pos, null); + closeableWriter.write(positionDelete, unpartitionedSpec, null); + } + } + + blackhole.consume(writer); + } + + @Benchmark + @Threads(1) + public void writeUnpartitionedFanoutPositionDeleteWriterShuffledPartitionGranularity( + Blackhole blackhole) throws IOException { + writeUnpartitionedFanoutPositionDeleteWriterShuffled(blackhole, DeleteGranularity.PARTITION); + } + + @Benchmark + @Threads(1) + public void writeUnpartitionedFanoutPositionDeleteWriterShuffledFileGranularity( + Blackhole blackhole) throws IOException { + writeUnpartitionedFanoutPositionDeleteWriterShuffled(blackhole, DeleteGranularity.FILE); + } + + private void writeUnpartitionedFanoutPositionDeleteWriterShuffled( + Blackhole blackhole, DeleteGranularity deleteGranularity) throws IOException { + + FileIO io = table().io(); + + OutputFileFactory fileFactory = newFileFactory(); + SparkFileWriterFactory writerFactory = + SparkFileWriterFactory.builderFor(table()).dataFileFormat(fileFormat()).build(); + + FanoutPositionOnlyDeleteWriter writer = + new FanoutPositionOnlyDeleteWriter<>( + writerFactory, fileFactory, io, TARGET_FILE_SIZE_IN_BYTES, deleteGranularity); + + PositionDelete positionDelete = PositionDelete.create(); + try (FanoutPositionOnlyDeleteWriter closeableWriter = writer) { + for (InternalRow row : shuffledPositionDeleteRows) { + String path = row.getString(0); + long pos = row.getLong(1); + positionDelete.set(path, pos, null); + closeableWriter.write(positionDelete, unpartitionedSpec, null); + } + } + + blackhole.consume(writer); + } + + private OutputFileFactory newFileFactory() { + return OutputFileFactory.builderFor(table(), 1, 1).format(fileFormat()).build(); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/AvroWritersBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/AvroWritersBenchmark.java new file mode 100644 index 000000000000..4dcd58c0c4d0 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/AvroWritersBenchmark.java @@ -0,0 +1,39 @@ +/* + * 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.avro; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.spark.source.WritersBenchmark; + +/** + * A benchmark that evaluates the performance of various Iceberg writers for Avro data. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=AvroWritersBenchmark + * -PjmhOutputPath=benchmark/avro-writers-benchmark-result.txt + * + */ +public class AvroWritersBenchmark extends WritersBenchmark { + + @Override + protected FileFormat fileFormat() { + return FileFormat.AVRO; + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java new file mode 100644 index 000000000000..f0297f644a52 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java @@ -0,0 +1,142 @@ +/* + * 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.avro; + +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.spark.sql.functions.current_date; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.IcebergSourceFlatDataBenchmark; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * A benchmark that evaluates the performance of reading Avro data with a flat schema using Iceberg + * and the built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=IcebergSourceFlatAvroDataReadBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-flat-avro-data-read-benchmark-result.txt + * + */ +public class IcebergSourceFlatAvroDataReadBenchmark extends IcebergSourceFlatDataBenchmark { + + private static final int NUM_FILES = 10; + private static final int NUM_ROWS = 1000000; + + @Setup + public void setupBenchmark() { + setupSpark(); + appendData(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Benchmark + @Threads(1) + public void readIceberg() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readFileSource() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().format("avro").load(dataLocation()); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionIceberg() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation).select("longCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionFileSource() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().format("avro").load(dataLocation()).select("longCol"); + materialize(df); + }); + } + + private void appendData() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(DEFAULT_FILE_FORMAT, "avro"); + withTableProperties( + tableProperties, + () -> { + for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) { + Dataset df = + spark() + .range(NUM_ROWS) + .withColumnRenamed("id", "longCol") + .withColumn("intCol", expr("CAST(longCol AS INT)")) + .withColumn("floatCol", expr("CAST(longCol AS FLOAT)")) + .withColumn("doubleCol", expr("CAST(longCol AS DOUBLE)")) + .withColumn("decimalCol", expr("CAST(longCol AS DECIMAL(20, 5))")) + .withColumn("dateCol", date_add(current_date(), fileNum)) + .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)")) + .withColumn("stringCol", expr("CAST(dateCol AS STRING)")); + appendAsFile(df); + } + }); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java new file mode 100644 index 000000000000..00d06566fbaa --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java @@ -0,0 +1,142 @@ +/* + * 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.avro; + +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.lit; +import static org.apache.spark.sql.functions.struct; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.IcebergSourceNestedDataBenchmark; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * A benchmark that evaluates the performance of reading Avro data with a flat schema using Iceberg + * and the built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=IcebergSourceNestedAvroDataReadBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-nested-avro-data-read-benchmark-result.txt + * + */ +public class IcebergSourceNestedAvroDataReadBenchmark extends IcebergSourceNestedDataBenchmark { + + private static final int NUM_FILES = 10; + private static final int NUM_ROWS = 1000000; + + @Setup + public void setupBenchmark() { + setupSpark(); + appendData(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Benchmark + @Threads(1) + public void readIceberg() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readFileSource() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().format("avro").load(dataLocation()); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionIceberg() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).select("nested.col3"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionFileSource() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = + spark().read().format("avro").load(dataLocation()).select("nested.col3"); + materialize(df); + }); + } + + private void appendData() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(DEFAULT_FILE_FORMAT, "avro"); + withTableProperties( + tableProperties, + () -> { + for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) { + Dataset df = + spark() + .range(NUM_ROWS) + .withColumn( + "nested", + struct( + expr("CAST(id AS string) AS col1"), + expr("CAST(id AS double) AS col2"), + lit(fileNum).cast("long").as("col3"))); + appendAsFile(df); + } + }); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataBenchmark.java new file mode 100644 index 000000000000..d0fdd8915780 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataBenchmark.java @@ -0,0 +1,68 @@ +/* + * 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.orc; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.IcebergSourceBenchmark; +import org.apache.iceberg.types.Types; + +/** + * Same as {@link org.apache.iceberg.spark.source.IcebergSourceFlatDataBenchmark} but we disable the + * Timestamp with zone type for ORC performance tests as Spark native reader does not support ORC's + * TIMESTAMP_INSTANT type + */ +public abstract class IcebergSourceFlatORCDataBenchmark extends IcebergSourceBenchmark { + + @Override + protected Configuration initHadoopConf() { + return new Configuration(); + } + + @Override + protected final Table initTable() { + Schema schema = + new Schema( + required(1, "longCol", Types.LongType.get()), + required(2, "intCol", Types.IntegerType.get()), + required(3, "floatCol", Types.FloatType.get()), + optional(4, "doubleCol", Types.DoubleType.get()), + optional(5, "decimalCol", Types.DecimalType.of(20, 5)), + optional(6, "dateCol", Types.DateType.get()), + // Disable timestamp column for ORC performance tests as Spark native reader does not + // support ORC's + // TIMESTAMP_INSTANT type + // optional(7, "timestampCol", Types.TimestampType.withZone()), + optional(8, "stringCol", Types.StringType.get())); + PartitionSpec partitionSpec = PartitionSpec.unpartitioned(); + HadoopTables tables = new HadoopTables(hadoopConf()); + Map properties = Maps.newHashMap(); + properties.put(TableProperties.METADATA_COMPRESSION, "gzip"); + return tables.create(schema, partitionSpec, properties, newTableLocation()); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java new file mode 100644 index 000000000000..593fbc955703 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java @@ -0,0 +1,210 @@ +/* + * 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.orc; + +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.spark.sql.functions.current_date; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * A benchmark that evaluates the performance of reading ORC data with a flat schema using Iceberg + * and the built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=IcebergSourceFlatORCDataReadBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-flat-orc-data-read-benchmark-result.txt + * + */ +public class IcebergSourceFlatORCDataReadBenchmark extends IcebergSourceFlatORCDataBenchmark { + + private static final int NUM_FILES = 10; + private static final int NUM_ROWS = 1000000; + + @Setup + public void setupBenchmark() { + setupSpark(); + appendData(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Benchmark + @Threads(1) + public void readIcebergNonVectorized() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readIcebergVectorized() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = + spark() + .read() + .option(SparkReadOptions.VECTORIZATION_ENABLED, "true") + .format("iceberg") + .load(tableLocation); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readFileSourceVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.ORC_VECTORIZED_READER_ENABLED().key(), "true"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().orc(dataLocation()); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readFileSourceNonVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.ORC_VECTORIZED_READER_ENABLED().key(), "false"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().orc(dataLocation()); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionIcebergNonVectorized() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation).select("longCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionIcebergVectorized() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = + spark() + .read() + .option(SparkReadOptions.VECTORIZATION_ENABLED, "true") + .format("iceberg") + .load(tableLocation) + .select("longCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionFileSourceVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.ORC_VECTORIZED_READER_ENABLED().key(), "true"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().orc(dataLocation()).select("longCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionFileSourceNonVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.ORC_VECTORIZED_READER_ENABLED().key(), "false"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().orc(dataLocation()).select("longCol"); + materialize(df); + }); + } + + private void appendData() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(DEFAULT_FILE_FORMAT, "orc"); + withTableProperties( + tableProperties, + () -> { + for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) { + Dataset df = + spark() + .range(NUM_ROWS) + .withColumnRenamed("id", "longCol") + .withColumn("intCol", expr("CAST(longCol AS INT)")) + .withColumn("floatCol", expr("CAST(longCol AS FLOAT)")) + .withColumn("doubleCol", expr("CAST(longCol AS DOUBLE)")) + .withColumn("decimalCol", expr("CAST(longCol AS DECIMAL(20, 5))")) + .withColumn("dateCol", date_add(current_date(), fileNum)) + .withColumn("stringCol", expr("CAST(dateCol AS STRING)")); + appendAsFile(df); + } + }); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java new file mode 100644 index 000000000000..0442ed02eb49 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java @@ -0,0 +1,109 @@ +/* + * 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.orc; + +import static org.apache.spark.sql.functions.array_repeat; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.struct; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.IcebergSourceNestedListDataBenchmark; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * A benchmark that evaluates the performance of writing nested Parquet data using Iceberg and the + * built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=IcebergSourceNestedListORCDataWriteBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-nested-list-orc-data-write-benchmark-result.txt + * + */ +public class IcebergSourceNestedListORCDataWriteBenchmark + extends IcebergSourceNestedListDataBenchmark { + + @Setup + public void setupBenchmark() { + setupSpark(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Param({"2000", "20000"}) + private int numRows; + + @Benchmark + @Threads(1) + public void writeIceberg() { + String tableLocation = table().location(); + benchmarkData() + .write() + .format("iceberg") + .option("write-format", "orc") + .mode(SaveMode.Append) + .save(tableLocation); + } + + @Benchmark + @Threads(1) + public void writeIcebergDictionaryOff() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put("orc.dictionary.key.threshold", "0"); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + benchmarkData() + .write() + .format("iceberg") + .option("write-format", "orc") + .mode(SaveMode.Append) + .save(tableLocation); + }); + } + + @Benchmark + @Threads(1) + public void writeFileSource() { + benchmarkData().write().mode(SaveMode.Append).orc(dataLocation()); + } + + private Dataset benchmarkData() { + return spark() + .range(numRows) + .withColumn( + "outerlist", + array_repeat(struct(expr("array_repeat(CAST(id AS string), 1000) AS innerlist")), 10)) + .coalesce(1); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java new file mode 100644 index 000000000000..a64a23774eec --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.orc; + +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.lit; +import static org.apache.spark.sql.functions.struct; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.source.IcebergSourceNestedDataBenchmark; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * A benchmark that evaluates the performance of reading ORC data with a flat schema using Iceberg + * and the built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=IcebergSourceNestedORCDataReadBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-nested-orc-data-read-benchmark-result.txt + * + */ +public class IcebergSourceNestedORCDataReadBenchmark extends IcebergSourceNestedDataBenchmark { + + private static final int NUM_FILES = 10; + private static final int NUM_ROWS = 1000000; + + @Setup + public void setupBenchmark() { + setupSpark(); + appendData(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Benchmark + @Threads(1) + public void readIcebergNonVectorized() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readIcebergVectorized() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = + spark() + .read() + .option(SparkReadOptions.VECTORIZATION_ENABLED, "true") + .format("iceberg") + .load(tableLocation); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readFileSourceNonVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.ORC_VECTORIZED_READER_ENABLED().key(), "false"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().orc(dataLocation()); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionIcebergNonVectorized() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).selectExpr("nested.col3"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionIcebergVectorized() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = + spark() + .read() + .option(SparkReadOptions.VECTORIZATION_ENABLED, "true") + .format("iceberg") + .load(tableLocation) + .selectExpr("nested.col3"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionFileSourceNonVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.ORC_VECTORIZED_READER_ENABLED().key(), "false"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().orc(dataLocation()).selectExpr("nested.col3"); + materialize(df); + }); + } + + private void appendData() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(DEFAULT_FILE_FORMAT, "orc"); + withTableProperties( + tableProperties, + () -> { + for (int fileNum = 0; fileNum < NUM_FILES; fileNum++) { + Dataset df = + spark() + .range(NUM_ROWS) + .withColumn( + "nested", + struct( + expr("CAST(id AS string) AS col1"), + expr("CAST(id AS double) AS col2"), + lit(fileNum).cast("long").as("col3"))); + appendAsFile(df); + } + }); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java new file mode 100644 index 000000000000..5b7b22f5ead7 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java @@ -0,0 +1,129 @@ +/* + * 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.parquet; + +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.spark.sql.functions.current_date; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.IcebergSourceFlatDataBenchmark; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * A benchmark that evaluates the file skipping capabilities in the Spark data source for Iceberg. + * + *

    This class uses a dataset with a flat schema, where the records are clustered according to the + * column used in the filter predicate. + * + *

    The performance is compared to the built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=IcebergSourceFlatParquetDataFilterBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-flat-parquet-data-filter-benchmark-result.txt + * + */ +public class IcebergSourceFlatParquetDataFilterBenchmark extends IcebergSourceFlatDataBenchmark { + + private static final String FILTER_COND = "dateCol == date_add(current_date(), 1)"; + private static final int NUM_FILES = 500; + private static final int NUM_ROWS = 10000; + + @Setup + public void setupBenchmark() { + setupSpark(); + appendData(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Benchmark + @Threads(1) + public void readWithFilterIceberg() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).filter(FILTER_COND); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithFilterFileSourceVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "true"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().parquet(dataLocation()).filter(FILTER_COND); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithFilterFileSourceNonVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "false"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().parquet(dataLocation()).filter(FILTER_COND); + materialize(df); + }); + } + + private void appendData() { + for (int fileNum = 1; fileNum < NUM_FILES; fileNum++) { + Dataset df = + spark() + .range(NUM_ROWS) + .withColumnRenamed("id", "longCol") + .withColumn("intCol", expr("CAST(longCol AS INT)")) + .withColumn("floatCol", expr("CAST(longCol AS FLOAT)")) + .withColumn("doubleCol", expr("CAST(longCol AS DOUBLE)")) + .withColumn("decimalCol", expr("CAST(longCol AS DECIMAL(20, 5))")) + .withColumn("dateCol", date_add(current_date(), fileNum)) + .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)")) + .withColumn("stringCol", expr("CAST(dateCol AS STRING)")); + appendAsFile(df); + } + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java new file mode 100644 index 000000000000..ec1514fe4297 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java @@ -0,0 +1,165 @@ +/* + * 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.parquet; + +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.spark.sql.functions.current_date; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.IcebergSourceFlatDataBenchmark; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * A benchmark that evaluates the performance of reading Parquet data with a flat schema using + * Iceberg and the built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=IcebergSourceFlatParquetDataReadBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-flat-parquet-data-read-benchmark-result.txt + * + */ +public class IcebergSourceFlatParquetDataReadBenchmark extends IcebergSourceFlatDataBenchmark { + + private static final int NUM_FILES = 10; + private static final int NUM_ROWS = 1000000; + + @Setup + public void setupBenchmark() { + setupSpark(); + appendData(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Benchmark + @Threads(1) + public void readIceberg() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readFileSourceVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "true"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().parquet(dataLocation()); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readFileSourceNonVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "false"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().parquet(dataLocation()); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionIceberg() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation).select("longCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionFileSourceVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "true"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("longCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionFileSourceNonVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "false"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("longCol"); + materialize(df); + }); + } + + private void appendData() { + for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) { + Dataset df = + spark() + .range(NUM_ROWS) + .withColumnRenamed("id", "longCol") + .withColumn("intCol", expr("CAST(longCol AS INT)")) + .withColumn("floatCol", expr("CAST(longCol AS FLOAT)")) + .withColumn("doubleCol", expr("CAST(longCol AS DOUBLE)")) + .withColumn("decimalCol", expr("CAST(longCol AS DECIMAL(20, 5))")) + .withColumn("dateCol", date_add(current_date(), fileNum)) + .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)")) + .withColumn("stringCol", expr("CAST(dateCol AS STRING)")); + appendAsFile(df); + } + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java new file mode 100644 index 000000000000..787ae389ca6b --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java @@ -0,0 +1,89 @@ +/* + * 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.parquet; + +import static org.apache.spark.sql.functions.expr; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.IcebergSourceFlatDataBenchmark; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.internal.SQLConf; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * A benchmark that evaluates the performance of writing Parquet data with a flat schema using + * Iceberg and the built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=IcebergSourceFlatParquetDataWriteBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-flat-parquet-data-write-benchmark-result.txt + * + */ +public class IcebergSourceFlatParquetDataWriteBenchmark extends IcebergSourceFlatDataBenchmark { + + private static final int NUM_ROWS = 5000000; + + @Setup + public void setupBenchmark() { + setupSpark(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Benchmark + @Threads(1) + public void writeIceberg() { + String tableLocation = table().location(); + benchmarkData().write().format("iceberg").mode(SaveMode.Append).save(tableLocation); + } + + @Benchmark + @Threads(1) + public void writeFileSource() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_COMPRESSION().key(), "gzip"); + withSQLConf(conf, () -> benchmarkData().write().mode(SaveMode.Append).parquet(dataLocation())); + } + + private Dataset benchmarkData() { + return spark() + .range(NUM_ROWS) + .withColumnRenamed("id", "longCol") + .withColumn("intCol", expr("CAST(longCol AS INT)")) + .withColumn("floatCol", expr("CAST(longCol AS FLOAT)")) + .withColumn("doubleCol", expr("CAST(longCol AS DOUBLE)")) + .withColumn("decimalCol", expr("CAST(longCol AS DECIMAL(20, 5))")) + .withColumn("dateCol", expr("DATE_ADD(CURRENT_DATE(), (intCol % 20))")) + .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)")) + .withColumn("stringCol", expr("CAST(dateCol AS STRING)")) + .coalesce(1); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java new file mode 100644 index 000000000000..0d17bd3e5653 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java @@ -0,0 +1,89 @@ +/* + * 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.parquet; + +import static org.apache.spark.sql.functions.array_repeat; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.struct; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.IcebergSourceNestedListDataBenchmark; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.internal.SQLConf; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * A benchmark that evaluates the performance of writing nested Parquet data using Iceberg and the + * built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=IcebergSourceNestedListParquetDataWriteBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-nested-list-parquet-data-write-benchmark-result.txt + * + */ +public class IcebergSourceNestedListParquetDataWriteBenchmark + extends IcebergSourceNestedListDataBenchmark { + + @Setup + public void setupBenchmark() { + setupSpark(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Param({"2000", "20000"}) + private int numRows; + + @Benchmark + @Threads(1) + public void writeIceberg() { + String tableLocation = table().location(); + benchmarkData().write().format("iceberg").mode(SaveMode.Append).save(tableLocation); + } + + @Benchmark + @Threads(1) + public void writeFileSource() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_COMPRESSION().key(), "gzip"); + withSQLConf(conf, () -> benchmarkData().write().mode(SaveMode.Append).parquet(dataLocation())); + } + + private Dataset benchmarkData() { + return spark() + .range(numRows) + .withColumn( + "outerlist", + array_repeat(struct(expr("array_repeat(CAST(id AS string), 1000) AS innerlist")), 10)) + .coalesce(1); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java new file mode 100644 index 000000000000..a5ddd060422f --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java @@ -0,0 +1,128 @@ +/* + * 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.parquet; + +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.lit; +import static org.apache.spark.sql.functions.struct; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.IcebergSourceNestedDataBenchmark; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * A benchmark that evaluates the file skipping capabilities in the Spark data source for Iceberg. + * + *

    This class uses a dataset with nested data, where the records are clustered according to the + * column used in the filter predicate. + * + *

    The performance is compared to the built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=IcebergSourceNestedParquetDataFilterBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-nested-parquet-data-filter-benchmark-result.txt + * + */ +public class IcebergSourceNestedParquetDataFilterBenchmark + extends IcebergSourceNestedDataBenchmark { + + private static final String FILTER_COND = "nested.col3 == 0"; + private static final int NUM_FILES = 500; + private static final int NUM_ROWS = 10000; + + @Setup + public void setupBenchmark() { + setupSpark(); + appendData(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Benchmark + @Threads(1) + public void readWithFilterIceberg() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).filter(FILTER_COND); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithFilterFileSourceVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "true"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().parquet(dataLocation()).filter(FILTER_COND); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithFilterFileSourceNonVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "false"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().parquet(dataLocation()).filter(FILTER_COND); + materialize(df); + }); + } + + private void appendData() { + for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) { + Dataset df = + spark() + .range(NUM_ROWS) + .withColumn( + "nested", + struct( + expr("CAST(id AS string) AS col1"), + expr("CAST(id AS double) AS col2"), + lit(fileNum).cast("long").as("col3"))); + appendAsFile(df); + } + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java new file mode 100644 index 000000000000..24e2d99902b4 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java @@ -0,0 +1,166 @@ +/* + * 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.parquet; + +import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.lit; +import static org.apache.spark.sql.functions.struct; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.IcebergSourceNestedDataBenchmark; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * A benchmark that evaluates the performance of reading nested Parquet data using Iceberg and the + * built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=IcebergSourceNestedParquetDataReadBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-nested-parquet-data-read-benchmark-result.txt + * + */ +public class IcebergSourceNestedParquetDataReadBenchmark extends IcebergSourceNestedDataBenchmark { + + private static final int NUM_FILES = 10; + private static final int NUM_ROWS = 1000000; + + @Setup + public void setupBenchmark() { + setupSpark(); + appendData(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Benchmark + @Threads(1) + public void readIceberg() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readFileSourceVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "true"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().parquet(dataLocation()); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readFileSourceNonVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "false"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().parquet(dataLocation()); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionIceberg() { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(SPLIT_OPEN_FILE_COST, Integer.toString(128 * 1024 * 1024)); + withTableProperties( + tableProperties, + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).selectExpr("nested.col3"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionFileSourceVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "true"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + conf.put(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED().key(), "true"); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().parquet(dataLocation()).selectExpr("nested.col3"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readWithProjectionFileSourceNonVectorized() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "false"); + conf.put(SQLConf.FILES_OPEN_COST_IN_BYTES().key(), Integer.toString(128 * 1024 * 1024)); + conf.put(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED().key(), "true"); + withSQLConf( + conf, + () -> { + Dataset df = spark().read().parquet(dataLocation()).selectExpr("nested.col3"); + materialize(df); + }); + } + + private void appendData() { + for (int fileNum = 0; fileNum < NUM_FILES; fileNum++) { + Dataset df = + spark() + .range(NUM_ROWS) + .withColumn( + "nested", + struct( + expr("CAST(id AS string) AS col1"), + expr("CAST(id AS double) AS col2"), + lit(fileNum).cast("long").as("col3"))); + appendAsFile(df); + } + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java new file mode 100644 index 000000000000..eef14854c4d6 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java @@ -0,0 +1,88 @@ +/* + * 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.parquet; + +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.struct; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.IcebergSourceNestedDataBenchmark; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.internal.SQLConf; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * A benchmark that evaluates the performance of writing nested Parquet data using Iceberg and the + * built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + * -PjmhIncludeRegex=IcebergSourceNestedParquetDataWriteBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-nested-parquet-data-write-benchmark-result.txt + * + */ +public class IcebergSourceNestedParquetDataWriteBenchmark extends IcebergSourceNestedDataBenchmark { + + private static final int NUM_ROWS = 5000000; + + @Setup + public void setupBenchmark() { + setupSpark(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Benchmark + @Threads(1) + public void writeIceberg() { + String tableLocation = table().location(); + benchmarkData().write().format("iceberg").mode(SaveMode.Append).save(tableLocation); + } + + @Benchmark + @Threads(1) + public void writeFileSource() { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_COMPRESSION().key(), "gzip"); + withSQLConf(conf, () -> benchmarkData().write().mode(SaveMode.Append).parquet(dataLocation())); + } + + private Dataset benchmarkData() { + return spark() + .range(NUM_ROWS) + .withColumn( + "nested", + struct( + expr("CAST(id AS string) AS col1"), + expr("CAST(id AS double) AS col2"), + expr("id AS col3"))) + .coalesce(1); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetEqDeleteBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetEqDeleteBenchmark.java new file mode 100644 index 000000000000..3b54b448b8b5 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetEqDeleteBenchmark.java @@ -0,0 +1,57 @@ +/* + * 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.parquet; + +import java.io.IOException; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; +import org.openjdk.jmh.annotations.Param; + +/** + * A benchmark that evaluates the non-vectorized read and vectorized read with equality delete in + * the Spark data source for Iceberg. + * + *

    This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh + * -PjmhIncludeRegex=IcebergSourceParquetEqDeleteBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-parquet-eq-delete-benchmark-result.txt + * + */ +public class IcebergSourceParquetEqDeleteBenchmark extends IcebergSourceDeleteBenchmark { + @Param({"0", "0.000001", "0.05", "0.25", "0.5", "1"}) + private double percentDeleteRow; + + @Override + protected void appendData() throws IOException { + for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) { + writeData(fileNum); + + if (percentDeleteRow > 0) { + // add equality deletes + table().refresh(); + writeEqDeletes(NUM_ROWS, percentDeleteRow); + } + } + } + + @Override + protected FileFormat fileFormat() { + return FileFormat.PARQUET; + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java new file mode 100644 index 000000000000..7891890dff4a --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java @@ -0,0 +1,57 @@ +/* + * 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.parquet; + +import java.io.IOException; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; +import org.openjdk.jmh.annotations.Param; + +/** + * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the + * Spark data source for Iceberg. + * + *

    This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh \ + * -PjmhIncludeRegex=IcebergSourceParquetMultiDeleteFileBenchmark \ + * -PjmhOutputPath=benchmark/iceberg-source-parquet-multi-delete-file-benchmark-result.txt + * + */ +public class IcebergSourceParquetMultiDeleteFileBenchmark extends IcebergSourceDeleteBenchmark { + @Param({"1", "2", "5", "10"}) + private int numDeleteFile; + + @Override + protected void appendData() throws IOException { + for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) { + writeData(fileNum); + + table().refresh(); + for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + writePosDeletes(file.location(), NUM_ROWS, 0.25, numDeleteFile); + } + } + } + + @Override + protected FileFormat fileFormat() { + return FileFormat.PARQUET; + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java new file mode 100644 index 000000000000..3c6dfa6bd94f --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java @@ -0,0 +1,60 @@ +/* + * 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.parquet; + +import java.io.IOException; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; +import org.openjdk.jmh.annotations.Param; + +/** + * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the + * Spark data source for Iceberg. + * + *

    This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh + * -PjmhIncludeRegex=IcebergSourceParquetPosDeleteBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-parquet-pos-delete-benchmark-result.txt + * + */ +public class IcebergSourceParquetPosDeleteBenchmark extends IcebergSourceDeleteBenchmark { + @Param({"0", "0.000001", "0.05", "0.25", "0.5", "1"}) + private double percentDeleteRow; + + @Override + protected void appendData() throws IOException { + for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) { + writeData(fileNum); + + if (percentDeleteRow > 0) { + // add pos-deletes + table().refresh(); + for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + writePosDeletes(file.location(), NUM_ROWS, percentDeleteRow); + } + } + } + } + + @Override + protected FileFormat fileFormat() { + return FileFormat.PARQUET; + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java new file mode 100644 index 000000000000..01096ac79649 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.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.source.parquet; + +import java.io.IOException; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.spark.source.IcebergSourceDeleteBenchmark; +import org.openjdk.jmh.annotations.Param; + +/** + * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the + * Spark data source for Iceberg. + * + *

    This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh + * -PjmhIncludeRegex=IcebergSourceParquetWithUnrelatedDeleteBenchmark + * -PjmhOutputPath=benchmark/iceberg-source-parquet-with-unrelated-delete-benchmark-result.txt + * + */ +public class IcebergSourceParquetWithUnrelatedDeleteBenchmark extends IcebergSourceDeleteBenchmark { + private static final double PERCENT_DELETE_ROW = 0.05; + + @Param({"0", "0.05", "0.25", "0.5"}) + private double percentUnrelatedDeletes; + + @Override + protected void appendData() throws IOException { + for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) { + writeData(fileNum); + + table().refresh(); + for (DataFile file : table().currentSnapshot().addedDataFiles(table().io())) { + writePosDeletesWithNoise( + file.location(), + NUM_ROWS, + PERCENT_DELETE_ROW, + (int) (percentUnrelatedDeletes / PERCENT_DELETE_ROW), + 1); + } + } + } + + @Override + protected FileFormat fileFormat() { + return FileFormat.PARQUET; + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/ParquetWritersBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/ParquetWritersBenchmark.java new file mode 100644 index 000000000000..8bcd871a07da --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/ParquetWritersBenchmark.java @@ -0,0 +1,39 @@ +/* + * 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.parquet; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.spark.source.WritersBenchmark; + +/** + * A benchmark that evaluates the performance of various Iceberg writers for Parquet data. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh \ + * -PjmhIncludeRegex=ParquetWritersBenchmark \ + * -PjmhOutputPath=benchmark/parquet-writers-benchmark-result.txt + * + */ +public class ParquetWritersBenchmark extends WritersBenchmark { + + @Override + protected FileFormat fileFormat() { + return FileFormat.PARQUET; + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java new file mode 100644 index 000000000000..73d4f6211803 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java @@ -0,0 +1,127 @@ +/* + * 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.parquet.vectorized; + +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.lit; +import static org.apache.spark.sql.functions.pmod; +import static org.apache.spark.sql.functions.to_date; +import static org.apache.spark.sql.functions.to_timestamp; + +import java.util.Map; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.types.DataTypes; +import org.openjdk.jmh.annotations.Setup; + +/** + * Benchmark to compare performance of reading Parquet dictionary encoded data with a flat schema + * using vectorized Iceberg read path and the built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh \ + * -PjmhIncludeRegex=VectorizedReadDictionaryEncodedFlatParquetDataBenchmark \ + * -PjmhOutputPath=benchmark/results.txt + * + */ +public class VectorizedReadDictionaryEncodedFlatParquetDataBenchmark + extends VectorizedReadFlatParquetDataBenchmark { + + @Setup + @Override + public void setupBenchmark() { + setupSpark(true); + appendData(); + } + + @Override + Map parquetWriteProps() { + Map properties = Maps.newHashMap(); + properties.put(TableProperties.METADATA_COMPRESSION, "gzip"); + return properties; + } + + @Override + void appendData() { + Dataset df = idDF(); + df = withLongColumnDictEncoded(df); + df = withIntColumnDictEncoded(df); + df = withFloatColumnDictEncoded(df); + df = withDoubleColumnDictEncoded(df); + df = withBigDecimalColumnNotDictEncoded(df); // no dictionary for fixed len binary in Parquet v1 + df = withDecimalColumnDictEncoded(df); + df = withDateColumnDictEncoded(df); + df = withTimestampColumnDictEncoded(df); + df = withStringColumnDictEncoded(df); + df = df.drop("id"); + df.write().format("iceberg").mode(SaveMode.Append).save(table().location()); + } + + private static Column modColumn() { + return pmod(col("id"), lit(9)); + } + + private Dataset idDF() { + return spark().range(0, NUM_ROWS_PER_FILE * NUM_FILES, 1, NUM_FILES).toDF(); + } + + private static Dataset withLongColumnDictEncoded(Dataset df) { + return df.withColumn("longCol", modColumn().cast(DataTypes.LongType)); + } + + private static Dataset withIntColumnDictEncoded(Dataset df) { + return df.withColumn("intCol", modColumn().cast(DataTypes.IntegerType)); + } + + private static Dataset withFloatColumnDictEncoded(Dataset df) { + return df.withColumn("floatCol", modColumn().cast(DataTypes.FloatType)); + } + + private static Dataset withDoubleColumnDictEncoded(Dataset df) { + return df.withColumn("doubleCol", modColumn().cast(DataTypes.DoubleType)); + } + + private static Dataset withBigDecimalColumnNotDictEncoded(Dataset df) { + return df.withColumn("bigDecimalCol", modColumn().cast("decimal(20,5)")); + } + + private static Dataset withDecimalColumnDictEncoded(Dataset df) { + return df.withColumn("decimalCol", modColumn().cast("decimal(18,5)")); + } + + private static Dataset withDateColumnDictEncoded(Dataset df) { + Column days = modColumn().cast(DataTypes.ShortType); + return df.withColumn("dateCol", date_add(to_date(lit("04/12/2019"), "MM/dd/yyyy"), days)); + } + + private static Dataset withTimestampColumnDictEncoded(Dataset df) { + Column days = modColumn().cast(DataTypes.ShortType); + return df.withColumn( + "timestampCol", to_timestamp(date_add(to_date(lit("04/12/2019"), "MM/dd/yyyy"), days))); + } + + private static Dataset withStringColumnDictEncoded(Dataset df) { + return df.withColumn("stringCol", modColumn().cast(DataTypes.StringType)); + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java new file mode 100644 index 000000000000..6cf327c1cf81 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java @@ -0,0 +1,355 @@ +/* + * 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.parquet.vectorized; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.current_date; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.lit; +import static org.apache.spark.sql.functions.pmod; +import static org.apache.spark.sql.functions.when; + +import java.io.IOException; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.IcebergSourceBenchmark; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * Benchmark to compare performance of reading Parquet data with a flat schema using vectorized + * Iceberg read path and the built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh \ + * -PjmhIncludeRegex=VectorizedReadFlatParquetDataBenchmark \ + * -PjmhOutputPath=benchmark/results.txt + * + */ +public class VectorizedReadFlatParquetDataBenchmark extends IcebergSourceBenchmark { + + static final int NUM_FILES = 5; + static final int NUM_ROWS_PER_FILE = 10_000_000; + + @Setup + public void setupBenchmark() { + setupSpark(); + appendData(); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Override + protected Configuration initHadoopConf() { + return new Configuration(); + } + + @Override + protected Table initTable() { + // bigDecimalCol is big enough to be encoded as fix len binary (9 bytes), + // decimalCol is small enough to be encoded as a 64-bit int + Schema schema = + new Schema( + optional(1, "longCol", Types.LongType.get()), + optional(2, "intCol", Types.IntegerType.get()), + optional(3, "floatCol", Types.FloatType.get()), + optional(4, "doubleCol", Types.DoubleType.get()), + optional(5, "bigDecimalCol", Types.DecimalType.of(20, 5)), + optional(6, "decimalCol", Types.DecimalType.of(18, 5)), + optional(7, "dateCol", Types.DateType.get()), + optional(8, "timestampCol", Types.TimestampType.withZone()), + optional(9, "stringCol", Types.StringType.get())); + PartitionSpec partitionSpec = PartitionSpec.unpartitioned(); + HadoopTables tables = new HadoopTables(hadoopConf()); + Map properties = parquetWriteProps(); + return tables.create(schema, partitionSpec, properties, newTableLocation()); + } + + Map parquetWriteProps() { + Map properties = Maps.newHashMap(); + properties.put(TableProperties.METADATA_COMPRESSION, "gzip"); + properties.put(TableProperties.PARQUET_DICT_SIZE_BYTES, "1"); + return properties; + } + + void appendData() { + for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) { + Dataset df = + spark() + .range(NUM_ROWS_PER_FILE) + .withColumn( + "longCol", + when(pmod(col("id"), lit(10)).equalTo(lit(0)), lit(null)).otherwise(col("id"))) + .drop("id") + .withColumn("intCol", expr("CAST(longCol AS INT)")) + .withColumn("floatCol", expr("CAST(longCol AS FLOAT)")) + .withColumn("doubleCol", expr("CAST(longCol AS DOUBLE)")) + .withColumn("bigDecimalCol", expr("CAST(longCol AS DECIMAL(20, 5))")) + .withColumn("decimalCol", expr("CAST(longCol AS DECIMAL(18, 5))")) + .withColumn("dateCol", date_add(current_date(), fileNum)) + .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)")) + .withColumn("stringCol", expr("CAST(longCol AS STRING)")); + appendAsFile(df); + } + } + + @Benchmark + @Threads(1) + public void readIntegersIcebergVectorized5k() { + withTableProperties( + tablePropsWithVectorizationEnabled(5000), + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation).select("intCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readIntegersSparkVectorized5k() { + withSQLConf( + sparkConfWithVectorizationEnabled(5000), + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("intCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readLongsIcebergVectorized5k() { + withTableProperties( + tablePropsWithVectorizationEnabled(5000), + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation).select("longCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readLongsSparkVectorized5k() { + withSQLConf( + sparkConfWithVectorizationEnabled(5000), + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("longCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readFloatsIcebergVectorized5k() { + withTableProperties( + tablePropsWithVectorizationEnabled(5000), + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation).select("floatCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readFloatsSparkVectorized5k() { + withSQLConf( + sparkConfWithVectorizationEnabled(5000), + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("floatCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readDoublesIcebergVectorized5k() { + withTableProperties( + tablePropsWithVectorizationEnabled(5000), + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).select("doubleCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readDoublesSparkVectorized5k() { + withSQLConf( + sparkConfWithVectorizationEnabled(5000), + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("doubleCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readDecimalsIcebergVectorized5k() { + withTableProperties( + tablePropsWithVectorizationEnabled(5000), + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).select("decimalCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readDecimalsSparkVectorized5k() { + withSQLConf( + sparkConfWithVectorizationEnabled(5000), + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("decimalCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readBigDecimalsIcebergVectorized5k() { + withTableProperties( + tablePropsWithVectorizationEnabled(5000), + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).select("bigDecimalCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readBigDecimalsSparkVectorized5k() { + withSQLConf( + sparkConfWithVectorizationEnabled(5000), + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("bigDecimalCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readDatesIcebergVectorized5k() { + withTableProperties( + tablePropsWithVectorizationEnabled(5000), + () -> { + String tableLocation = table().location(); + Dataset df = spark().read().format("iceberg").load(tableLocation).select("dateCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readDatesSparkVectorized5k() { + withSQLConf( + sparkConfWithVectorizationEnabled(5000), + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("dateCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readTimestampsIcebergVectorized5k() { + withTableProperties( + tablePropsWithVectorizationEnabled(5000), + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).select("timestampCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readTimestampsSparkVectorized5k() { + withSQLConf( + sparkConfWithVectorizationEnabled(5000), + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("timestampCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readStringsIcebergVectorized5k() { + withTableProperties( + tablePropsWithVectorizationEnabled(5000), + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).select("stringCol"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readStringsSparkVectorized5k() { + withSQLConf( + sparkConfWithVectorizationEnabled(5000), + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("stringCol"); + materialize(df); + }); + } + + private static Map tablePropsWithVectorizationEnabled(int batchSize) { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(TableProperties.PARQUET_VECTORIZATION_ENABLED, "true"); + tableProperties.put(TableProperties.PARQUET_BATCH_SIZE, String.valueOf(batchSize)); + return tableProperties; + } + + private static Map sparkConfWithVectorizationEnabled(int batchSize) { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "true"); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_BATCH_SIZE().key(), String.valueOf(batchSize)); + return conf; + } +} diff --git a/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadParquetDecimalBenchmark.java b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadParquetDecimalBenchmark.java new file mode 100644 index 000000000000..ccf28e3fdc77 --- /dev/null +++ b/spark/v4.0/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadParquetDecimalBenchmark.java @@ -0,0 +1,207 @@ +/* + * 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.parquet.vectorized; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.lit; +import static org.apache.spark.sql.functions.pmod; +import static org.apache.spark.sql.functions.when; + +import java.io.IOException; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.IcebergSourceBenchmark; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; + +/** + * Benchmark to compare performance of reading Parquet decimal data using vectorized Iceberg read + * path and the built-in file source in Spark. + * + *

    To run this benchmark for spark-4.0: + * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh \ + * -PjmhIncludeRegex=VectorizedReadParquetDecimalBenchmark \ + * -PjmhOutputPath=benchmark/results.txt + * + */ +public class VectorizedReadParquetDecimalBenchmark extends IcebergSourceBenchmark { + + static final int NUM_FILES = 5; + static final int NUM_ROWS_PER_FILE = 10_000_000; + + @Setup + public void setupBenchmark() { + setupSpark(); + appendData(); + // Allow unsafe memory access to avoid the costly check arrow does to check if index is within + // bounds + System.setProperty("arrow.enable_unsafe_memory_access", "true"); + // Disable expensive null check for every get(index) call. + // Iceberg manages nullability checks itself instead of relying on arrow. + System.setProperty("arrow.enable_null_check_for_get", "false"); + } + + @TearDown + public void tearDownBenchmark() throws IOException { + tearDownSpark(); + cleanupFiles(); + } + + @Override + protected Configuration initHadoopConf() { + return new Configuration(); + } + + @Override + protected Table initTable() { + Schema schema = + new Schema( + optional(1, "decimalCol1", Types.DecimalType.of(7, 2)), + optional(2, "decimalCol2", Types.DecimalType.of(15, 2)), + optional(3, "decimalCol3", Types.DecimalType.of(20, 2))); + PartitionSpec partitionSpec = PartitionSpec.unpartitioned(); + HadoopTables tables = new HadoopTables(hadoopConf()); + Map properties = parquetWriteProps(); + return tables.create(schema, partitionSpec, properties, newTableLocation()); + } + + Map parquetWriteProps() { + Map properties = Maps.newHashMap(); + properties.put(TableProperties.METADATA_COMPRESSION, "gzip"); + properties.put(TableProperties.PARQUET_DICT_SIZE_BYTES, "1"); + return properties; + } + + void appendData() { + for (int fileNum = 1; fileNum <= NUM_FILES; fileNum++) { + Dataset df = + spark() + .range(NUM_ROWS_PER_FILE) + .withColumn( + "longCol", + when(pmod(col("id"), lit(10)).equalTo(lit(0)), lit(null)).otherwise(col("id"))) + .drop("id") + .withColumn("decimalCol1", expr("CAST(longCol AS DECIMAL(7, 2))")) + .withColumn("decimalCol2", expr("CAST(longCol AS DECIMAL(15, 2))")) + .withColumn("decimalCol3", expr("CAST(longCol AS DECIMAL(20, 2))")) + .drop("longCol"); + appendAsFile(df); + } + } + + @Benchmark + @Threads(1) + public void readIntBackedDecimalsIcebergVectorized5k() { + withTableProperties( + tablePropsWithVectorizationEnabled(5000), + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).select("decimalCol1"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readIntBackedDecimalsSparkVectorized5k() { + withSQLConf( + sparkConfWithVectorizationEnabled(5000), + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("decimalCol1"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readLongBackedDecimalsIcebergVectorized5k() { + withTableProperties( + tablePropsWithVectorizationEnabled(5000), + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).select("decimalCol2"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readLongBackedDecimalsSparkVectorized5k() { + withSQLConf( + sparkConfWithVectorizationEnabled(5000), + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("decimalCol2"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readDecimalsIcebergVectorized5k() { + withTableProperties( + tablePropsWithVectorizationEnabled(5000), + () -> { + String tableLocation = table().location(); + Dataset df = + spark().read().format("iceberg").load(tableLocation).select("decimalCol3"); + materialize(df); + }); + } + + @Benchmark + @Threads(1) + public void readDecimalsSparkVectorized5k() { + withSQLConf( + sparkConfWithVectorizationEnabled(5000), + () -> { + Dataset df = spark().read().parquet(dataLocation()).select("decimalCol3"); + materialize(df); + }); + } + + private static Map tablePropsWithVectorizationEnabled(int batchSize) { + Map tableProperties = Maps.newHashMap(); + tableProperties.put(TableProperties.PARQUET_VECTORIZATION_ENABLED, "true"); + tableProperties.put(TableProperties.PARQUET_BATCH_SIZE, String.valueOf(batchSize)); + return tableProperties; + } + + private static Map sparkConfWithVectorizationEnabled(int batchSize) { + Map conf = Maps.newHashMap(); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key(), "true"); + conf.put(SQLConf.PARQUET_VECTORIZED_READER_BATCH_SIZE().key(), String.valueOf(batchSize)); + return conf; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/SparkDistributedDataScan.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/SparkDistributedDataScan.java new file mode 100644 index 000000000000..43ce2a303e2b --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/SparkDistributedDataScan.java @@ -0,0 +1,272 @@ +/* + * 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; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.ClosingIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.metrics.MetricsReporter; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.spark.JobGroupUtils; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.actions.ManifestFileBean; +import org.apache.iceberg.spark.source.SerializableTableWithSize; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.SparkSession; + +/** + * A batch data scan that can utilize Spark cluster resources for planning. + * + *

    This scan remotely filters manifests, fetching only the relevant data and delete files to the + * driver. The delete file assignment is done locally after the remote filtering step. Such approach + * is beneficial if the remote parallelism is much higher than the number of driver cores. + * + *

    This scan is best suited for queries with selective filters on lower/upper bounds across all + * partitions, or against poorly clustered metadata. This allows job planning to benefit from highly + * concurrent remote filtering while not incurring high serialization and data transfer costs. This + * class is also useful for full table scans over large tables but the cost of bringing data and + * delete file details to the driver may become noticeable. Make sure to follow the performance tips + * below in such cases. + * + *

    Ensure the filtered metadata size doesn't exceed the driver's max result size. For large table + * scans, consider increasing `spark.driver.maxResultSize` to avoid job failures. + * + *

    Performance tips: + * + *

      + *
    • Enable Kryo serialization (`spark.serializer`) + *
    • Increase the number of driver cores (`spark.driver.cores`) + *
    • Tune the number of threads used to fetch task results (`spark.resultGetter.threads`) + *
    + */ +public class SparkDistributedDataScan extends BaseDistributedDataScan { + + private static final Joiner COMMA = Joiner.on(','); + private static final String DELETE_PLANNING_JOB_GROUP_ID = "DELETE-PLANNING"; + private static final String DATA_PLANNING_JOB_GROUP_ID = "DATA-PLANNING"; + + private final SparkSession spark; + private final JavaSparkContext sparkContext; + private final SparkReadConf readConf; + + private Broadcast

  • tableBroadcast = null; + + public SparkDistributedDataScan(SparkSession spark, Table table, SparkReadConf readConf) { + this(spark, table, readConf, table.schema(), newTableScanContext(table)); + } + + private SparkDistributedDataScan( + SparkSession spark, + Table table, + SparkReadConf readConf, + Schema schema, + TableScanContext context) { + super(table, schema, context); + this.spark = spark; + this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + this.readConf = readConf; + } + + @Override + protected BatchScan newRefinedScan( + Table newTable, Schema newSchema, TableScanContext newContext) { + return new SparkDistributedDataScan(spark, newTable, readConf, newSchema, newContext); + } + + @Override + protected int remoteParallelism() { + return readConf.parallelism(); + } + + @Override + protected PlanningMode dataPlanningMode() { + return readConf.dataPlanningMode(); + } + + @Override + protected boolean shouldCopyRemotelyPlannedDataFiles() { + return false; + } + + @Override + protected Iterable> planDataRemotely( + List dataManifests, boolean withColumnStats) { + JobGroupInfo info = new JobGroupInfo(DATA_PLANNING_JOB_GROUP_ID, jobDesc("data")); + return withJobGroupInfo(info, () -> doPlanDataRemotely(dataManifests, withColumnStats)); + } + + private Iterable> doPlanDataRemotely( + List dataManifests, boolean withColumnStats) { + scanMetrics().scannedDataManifests().increment(dataManifests.size()); + + JavaRDD dataFileRDD = + sparkContext + .parallelize(toBeans(dataManifests), dataManifests.size()) + .flatMap(new ReadDataManifest(tableBroadcast(), context(), withColumnStats)); + List> dataFileGroups = collectPartitions(dataFileRDD); + + int matchingFilesCount = dataFileGroups.stream().mapToInt(List::size).sum(); + int skippedFilesCount = liveFilesCount(dataManifests) - matchingFilesCount; + scanMetrics().skippedDataFiles().increment(skippedFilesCount); + + return Iterables.transform(dataFileGroups, CloseableIterable::withNoopClose); + } + + @Override + protected PlanningMode deletePlanningMode() { + return readConf.deletePlanningMode(); + } + + @Override + protected DeleteFileIndex planDeletesRemotely(List deleteManifests) { + JobGroupInfo info = new JobGroupInfo(DELETE_PLANNING_JOB_GROUP_ID, jobDesc("deletes")); + return withJobGroupInfo(info, () -> doPlanDeletesRemotely(deleteManifests)); + } + + private DeleteFileIndex doPlanDeletesRemotely(List deleteManifests) { + scanMetrics().scannedDeleteManifests().increment(deleteManifests.size()); + + List deleteFiles = + sparkContext + .parallelize(toBeans(deleteManifests), deleteManifests.size()) + .flatMap(new ReadDeleteManifest(tableBroadcast(), context())) + .collect(); + + int skippedFilesCount = liveFilesCount(deleteManifests) - deleteFiles.size(); + scanMetrics().skippedDeleteFiles().increment(skippedFilesCount); + + return DeleteFileIndex.builderFor(deleteFiles) + .specsById(table().specs()) + .caseSensitive(isCaseSensitive()) + .scanMetrics(scanMetrics()) + .build(); + } + + private T withJobGroupInfo(JobGroupInfo info, Supplier supplier) { + return JobGroupUtils.withJobGroupInfo(sparkContext, info, supplier); + } + + private String jobDesc(String type) { + List options = Lists.newArrayList(); + options.add("snapshot_id=" + snapshot().snapshotId()); + String optionsAsString = COMMA.join(options); + return String.format("Planning %s (%s) for %s", type, optionsAsString, table().name()); + } + + private List toBeans(List manifests) { + return manifests.stream().map(ManifestFileBean::fromManifest).collect(Collectors.toList()); + } + + private Broadcast
    tableBroadcast() { + if (tableBroadcast == null) { + Table serializableTable = SerializableTableWithSize.copyOf(table()); + this.tableBroadcast = sparkContext.broadcast(serializableTable); + } + + return tableBroadcast; + } + + private List> collectPartitions(JavaRDD rdd) { + int[] partitionIds = IntStream.range(0, rdd.getNumPartitions()).toArray(); + return Arrays.asList(rdd.collectPartitions(partitionIds)); + } + + private int liveFilesCount(List manifests) { + return manifests.stream().mapToInt(this::liveFilesCount).sum(); + } + + private int liveFilesCount(ManifestFile manifest) { + return manifest.existingFilesCount() + manifest.addedFilesCount(); + } + + private static TableScanContext newTableScanContext(Table table) { + if (table instanceof BaseTable) { + MetricsReporter reporter = ((BaseTable) table).reporter(); + return ImmutableTableScanContext.builder().metricsReporter(reporter).build(); + } else { + return TableScanContext.empty(); + } + } + + private static class ReadDataManifest implements FlatMapFunction { + + private final Broadcast
    table; + private final Expression filter; + private final boolean withStats; + private final boolean isCaseSensitive; + + ReadDataManifest(Broadcast
    table, TableScanContext context, boolean withStats) { + this.table = table; + this.filter = context.rowFilter(); + this.withStats = withStats; + this.isCaseSensitive = context.caseSensitive(); + } + + @Override + public Iterator call(ManifestFileBean manifest) throws Exception { + FileIO io = table.value().io(); + Map specs = table.value().specs(); + return new ClosingIterator<>( + ManifestFiles.read(manifest, io, specs) + .select(withStats ? SCAN_WITH_STATS_COLUMNS : SCAN_COLUMNS) + .filterRows(filter) + .caseSensitive(isCaseSensitive) + .iterator()); + } + } + + private static class ReadDeleteManifest implements FlatMapFunction { + + private final Broadcast
    table; + private final Expression filter; + private final boolean isCaseSensitive; + + ReadDeleteManifest(Broadcast
    table, TableScanContext context) { + this.table = table; + this.filter = context.rowFilter(); + this.isCaseSensitive = context.caseSensitive(); + } + + @Override + public Iterator call(ManifestFileBean manifest) throws Exception { + FileIO io = table.value().io(); + Map specs = table.value().specs(); + return new ClosingIterator<>( + ManifestFiles.readDeleteManifest(manifest, io, specs) + .select(DELETE_SCAN_WITH_STATS_COLUMNS) + .filterRows(filter) + .caseSensitive(isCaseSensitive) + .iterator()); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java new file mode 100644 index 000000000000..c6784f1041be --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java @@ -0,0 +1,94 @@ +/* + * 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 org.apache.iceberg.spark.procedures.SparkProcedures; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.iceberg.spark.source.HasIcebergCatalog; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.ProcedureCatalog; +import org.apache.spark.sql.connector.catalog.StagingTableCatalog; +import org.apache.spark.sql.connector.catalog.SupportsNamespaces; +import org.apache.spark.sql.connector.catalog.ViewCatalog; +import org.apache.spark.sql.connector.catalog.procedures.UnboundProcedure; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +abstract class BaseCatalog + implements StagingTableCatalog, + ProcedureCatalog, + SupportsNamespaces, + HasIcebergCatalog, + SupportsFunctions, + ViewCatalog, + SupportsReplaceView { + private static final String USE_NULLABLE_QUERY_SCHEMA_CTAS_RTAS = "use-nullable-query-schema"; + private static final boolean USE_NULLABLE_QUERY_SCHEMA_CTAS_RTAS_DEFAULT = true; + + private boolean useNullableQuerySchema = USE_NULLABLE_QUERY_SCHEMA_CTAS_RTAS_DEFAULT; + + @Override + public UnboundProcedure loadProcedure(Identifier ident) { + String[] namespace = ident.namespace(); + String name = ident.name(); + + // namespace resolution is case insensitive until we have a way to configure case sensitivity in + // catalogs + if (isSystemNamespace(namespace)) { + ProcedureBuilder builder = SparkProcedures.newBuilder(name); + if (builder != null) { + return builder.withTableCatalog(this).build(); + } + } + + throw new RuntimeException("Procedure " + ident + " not found"); + } + + @Override + public boolean isFunctionNamespace(String[] namespace) { + // Allow for empty namespace, as Spark's storage partitioned joins look up + // the corresponding functions to generate transforms for partitioning + // with an empty namespace, such as `bucket`. + // Otherwise, use `system` namespace. + return namespace.length == 0 || isSystemNamespace(namespace); + } + + @Override + public boolean isExistingNamespace(String[] namespace) { + return namespaceExists(namespace); + } + + @Override + public void initialize(String name, CaseInsensitiveStringMap options) { + this.useNullableQuerySchema = + PropertyUtil.propertyAsBoolean( + options, + USE_NULLABLE_QUERY_SCHEMA_CTAS_RTAS, + USE_NULLABLE_QUERY_SCHEMA_CTAS_RTAS_DEFAULT); + } + + @Override + public boolean useNullableQuerySchema() { + return useNullableQuerySchema; + } + + private static boolean isSystemNamespace(String[] namespace) { + return namespace.length == 1 && namespace[0].equalsIgnoreCase("system"); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/BaseFileRewriteCoordinator.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/BaseFileRewriteCoordinator.java new file mode 100644 index 000000000000..5c95475d3302 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/BaseFileRewriteCoordinator.java @@ -0,0 +1,81 @@ +/* + * 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.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +abstract class BaseFileRewriteCoordinator> { + + private static final Logger LOG = LoggerFactory.getLogger(BaseFileRewriteCoordinator.class); + + private final Map, Set> resultMap = Maps.newConcurrentMap(); + + /** + * Called to persist the output of a rewrite action for a specific group. Since the write is done + * via a Spark Datasource, we have to propagate the result through this side-effect call. + * + * @param table table where the rewrite is occurring + * @param fileSetId the id used to identify the source set of files being rewritten + * @param newFiles the new files which have been written + */ + public void stageRewrite(Table table, String fileSetId, Set newFiles) { + LOG.debug( + "Staging the output for {} - fileset {} with {} files", + table.name(), + fileSetId, + newFiles.size()); + Pair id = toId(table, fileSetId); + resultMap.put(id, newFiles); + } + + public Set fetchNewFiles(Table table, String fileSetId) { + Pair id = toId(table, fileSetId); + Set result = resultMap.get(id); + ValidationException.check( + result != null, "No results for rewrite of file set %s in table %s", fileSetId, table); + + return result; + } + + public void clearRewrite(Table table, String fileSetId) { + LOG.debug("Removing entry for {} - id {}", table.name(), fileSetId); + Pair id = toId(table, fileSetId); + resultMap.remove(id); + } + + public Set fetchSetIds(Table table) { + return resultMap.keySet().stream() + .filter(e -> e.first().equals(Spark3Util.baseTableUUID(table))) + .map(Pair::second) + .collect(Collectors.toSet()); + } + + private Pair toId(Table table, String setId) { + return Pair.of(Spark3Util.baseTableUUID(table), setId); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java new file mode 100644 index 000000000000..cc44b1f3992c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ChangelogIterator.java @@ -0,0 +1,127 @@ +/* + * 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.Iterator; +import java.util.Objects; +import java.util.Set; +import org.apache.iceberg.ChangelogOperation; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterators; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; + +/** An iterator that transforms rows from changelog tables within a single Spark task. */ +public abstract class ChangelogIterator implements Iterator { + protected static final String DELETE = ChangelogOperation.DELETE.name(); + protected static final String INSERT = ChangelogOperation.INSERT.name(); + protected static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name(); + protected static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name(); + + private final Iterator rowIterator; + private final int changeTypeIndex; + private final StructType rowType; + + protected ChangelogIterator(Iterator rowIterator, StructType rowType) { + this.rowIterator = rowIterator; + this.rowType = rowType; + this.changeTypeIndex = rowType.fieldIndex(MetadataColumns.CHANGE_TYPE.name()); + } + + protected int changeTypeIndex() { + return changeTypeIndex; + } + + protected StructType rowType() { + return rowType; + } + + protected String changeType(Row row) { + String changeType = row.getString(changeTypeIndex()); + Preconditions.checkNotNull(changeType, "Change type should not be null"); + return changeType; + } + + protected Iterator rowIterator() { + return rowIterator; + } + + /** + * Creates an iterator composing {@link RemoveCarryoverIterator} and {@link ComputeUpdateIterator} + * to remove carry-over rows and compute update rows + * + * @param rowIterator the iterator of rows from a changelog table + * @param rowType the schema of the rows + * @param identifierFields the names of the identifier columns, which determine if rows are the + * same + * @return a new iterator instance + */ + public static Iterator computeUpdates( + Iterator rowIterator, StructType rowType, String[] identifierFields) { + Iterator carryoverRemoveIterator = removeCarryovers(rowIterator, rowType); + ChangelogIterator changelogIterator = + new ComputeUpdateIterator(carryoverRemoveIterator, rowType, identifierFields); + return Iterators.filter(changelogIterator, Objects::nonNull); + } + + /** + * Creates an iterator that removes carry-over rows from a changelog table. + * + * @param rowIterator the iterator of rows from a changelog table + * @param rowType the schema of the rows + * @return a new iterator instance + */ + public static Iterator removeCarryovers(Iterator rowIterator, StructType rowType) { + RemoveCarryoverIterator changelogIterator = new RemoveCarryoverIterator(rowIterator, rowType); + return Iterators.filter(changelogIterator, Objects::nonNull); + } + + public static Iterator removeNetCarryovers(Iterator rowIterator, StructType rowType) { + ChangelogIterator changelogIterator = new RemoveNetCarryoverIterator(rowIterator, rowType); + return Iterators.filter(changelogIterator, Objects::nonNull); + } + + protected boolean isSameRecord(Row currentRow, Row nextRow, int[] indicesToIdentifySameRow) { + for (int idx : indicesToIdentifySameRow) { + if (isDifferentValue(currentRow, nextRow, idx)) { + return false; + } + } + + return true; + } + + protected boolean isDifferentValue(Row currentRow, Row nextRow, int idx) { + return !Objects.equals(nextRow.get(idx), currentRow.get(idx)); + } + + protected static int[] generateIndicesToIdentifySameRow( + int totalColumnCount, Set metadataColumnIndices) { + int[] indices = new int[totalColumnCount - metadataColumnIndices.size()]; + + for (int i = 0, j = 0; i < indices.length; i++) { + if (!metadataColumnIndices.contains(i)) { + indices[j] = i; + j++; + } + } + return indices; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java new file mode 100644 index 000000000000..ea400a779235 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/CommitMetadata.java @@ -0,0 +1,66 @@ +/* + * 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.Map; +import java.util.concurrent.Callable; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.ExceptionUtil; + +/** utility class to accept thread local commit properties */ +public class CommitMetadata { + + private CommitMetadata() {} + + private static final ThreadLocal> COMMIT_PROPERTIES = + ThreadLocal.withInitial(ImmutableMap::of); + + /** + * running the code wrapped as a caller, and any snapshot committed within the callable object + * will be attached with the metadata defined in properties + * + * @param properties extra commit metadata to attach to the snapshot committed within callable. + * The prefix will be removed for properties starting with {@link + * SnapshotSummary#EXTRA_METADATA_PREFIX} + * @param callable the code to be executed + * @param exClass the expected type of exception which would be thrown from callable + */ + public static R withCommitProperties( + Map properties, Callable callable, Class exClass) throws E { + Map props = Maps.newHashMap(); + properties.forEach( + (k, v) -> props.put(k.replace(SnapshotSummary.EXTRA_METADATA_PREFIX, ""), v)); + + COMMIT_PROPERTIES.set(props); + try { + return callable.call(); + } catch (Throwable e) { + ExceptionUtil.castAndThrow(e, exClass); + return null; + } finally { + COMMIT_PROPERTIES.set(ImmutableMap.of()); + } + } + + public static Map commitProperties() { + return COMMIT_PROPERTIES.get(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java new file mode 100644 index 000000000000..6951c33e51aa --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ComputeUpdateIterator.java @@ -0,0 +1,140 @@ +/* + * 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.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.catalyst.expressions.GenericRow; +import org.apache.spark.sql.types.StructType; + +/** + * An iterator that finds delete/insert rows which represent an update, and converts them into + * update records from changelog tables within a single Spark task. It assumes that rows are sorted + * by identifier columns and change type. + * + *

    For example, these two rows + * + *

      + *
    • (id=1, data='a', op='DELETE') + *
    • (id=1, data='b', op='INSERT') + *
    + * + *

    will be marked as update-rows: + * + *

      + *
    • (id=1, data='a', op='UPDATE_BEFORE') + *
    • (id=1, data='b', op='UPDATE_AFTER') + *
    + */ +public class ComputeUpdateIterator extends ChangelogIterator { + + private final String[] identifierFields; + private final List identifierFieldIdx; + + private Row cachedRow = null; + + ComputeUpdateIterator(Iterator rowIterator, StructType rowType, String[] identifierFields) { + super(rowIterator, rowType); + this.identifierFieldIdx = + Arrays.stream(identifierFields).map(rowType::fieldIndex).collect(Collectors.toList()); + this.identifierFields = identifierFields; + } + + @Override + public boolean hasNext() { + if (cachedRow != null) { + return true; + } + return rowIterator().hasNext(); + } + + @Override + public Row next() { + // if there is an updated cached row, return it directly + if (cachedUpdateRecord()) { + Row row = cachedRow; + cachedRow = null; + return row; + } + + // either a cached record which is not an UPDATE or the next record in the iterator. + Row currentRow = currentRow(); + + if (changeType(currentRow).equals(DELETE) && rowIterator().hasNext()) { + Row nextRow = rowIterator().next(); + cachedRow = nextRow; + + if (sameLogicalRow(currentRow, nextRow)) { + Preconditions.checkState( + changeType(nextRow).equals(INSERT), + "Cannot compute updates because there are multiple rows with the same identifier" + + " fields([%s]). Please make sure the rows are unique.", + String.join(",", identifierFields)); + + currentRow = modify(currentRow, changeTypeIndex(), UPDATE_BEFORE); + cachedRow = modify(nextRow, changeTypeIndex(), UPDATE_AFTER); + } + } + + return currentRow; + } + + private Row modify(Row row, int valueIndex, Object value) { + if (row instanceof GenericRow) { + GenericRow genericRow = (GenericRow) row; + genericRow.values()[valueIndex] = value; + return genericRow; + } else { + Object[] values = new Object[row.size()]; + for (int index = 0; index < row.size(); index++) { + values[index] = row.get(index); + } + values[valueIndex] = value; + return RowFactory.create(values); + } + } + + private boolean cachedUpdateRecord() { + return cachedRow != null && changeType(cachedRow).equals(UPDATE_AFTER); + } + + private Row currentRow() { + if (cachedRow != null) { + Row row = cachedRow; + cachedRow = null; + return row; + } else { + return rowIterator().next(); + } + } + + private boolean sameLogicalRow(Row currentRow, Row nextRow) { + for (int idx : identifierFieldIdx) { + if (isDifferentValue(currentRow, nextRow, idx)) { + return false; + } + } + return true; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ExtendedParser.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ExtendedParser.java new file mode 100644 index 000000000000..5bfefbb97409 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ExtendedParser.java @@ -0,0 +1,110 @@ +/* + * 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.lang.reflect.Field; +import java.util.List; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.expressions.Term; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.parser.ParserInterface; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public interface ExtendedParser extends ParserInterface { + class RawOrderField { + private final Term term; + private final SortDirection direction; + private final NullOrder nullOrder; + + public RawOrderField(Term term, SortDirection direction, NullOrder nullOrder) { + this.term = term; + this.direction = direction; + this.nullOrder = nullOrder; + } + + public Term term() { + return term; + } + + public SortDirection direction() { + return direction; + } + + public NullOrder nullOrder() { + return nullOrder; + } + } + + static List parseSortOrder(SparkSession spark, String orderString) { + ExtendedParser extParser = findParser(spark.sessionState().sqlParser(), ExtendedParser.class); + if (extParser != null) { + try { + return extParser.parseSortOrder(orderString); + } catch (AnalysisException e) { + throw new IllegalArgumentException( + String.format("Unable to parse sortOrder: %s", orderString), e); + } + } else { + throw new IllegalStateException( + "Cannot parse order: parser is not an Iceberg ExtendedParser"); + } + } + + private static T findParser(ParserInterface parser, Class clazz) { + ParserInterface current = parser; + while (current != null) { + if (clazz.isInstance(current)) { + return clazz.cast(current); + } + + current = getNextDelegateParser(current); + } + + return null; + } + + private static ParserInterface getNextDelegateParser(ParserInterface parser) { + try { + Class clazz = parser.getClass(); + while (clazz != null) { + for (Field field : clazz.getDeclaredFields()) { + field.setAccessible(true); + Object value = field.get(parser); + if (value instanceof ParserInterface && value != parser) { + return (ParserInterface) value; + } + } + clazz = clazz.getSuperclass(); + } + } catch (Exception e) { + log().warn("Failed to scan delegate parser in {}: ", parser.getClass().getName(), e); + } + + return null; + } + + private static Logger log() { + return LoggerFactory.getLogger(ExtendedParser.class); + } + + List parseSortOrder(String orderString) throws AnalysisException; +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java new file mode 100644 index 000000000000..432f7737d623 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/FileRewriteCoordinator.java @@ -0,0 +1,32 @@ +/* + * 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 org.apache.iceberg.DataFile; + +public class FileRewriteCoordinator extends BaseFileRewriteCoordinator { + + private static final FileRewriteCoordinator INSTANCE = new FileRewriteCoordinator(); + + private FileRewriteCoordinator() {} + + public static FileRewriteCoordinator get() { + return INSTANCE; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/IcebergSpark.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/IcebergSpark.java new file mode 100644 index 000000000000..eb2420c0b254 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/IcebergSpark.java @@ -0,0 +1,56 @@ +/* + * 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.function.Function; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Type; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; + +public class IcebergSpark { + private IcebergSpark() {} + + public static void registerBucketUDF( + SparkSession session, String funcName, DataType sourceType, int numBuckets) { + SparkTypeToType typeConverter = new SparkTypeToType(); + Type sourceIcebergType = typeConverter.atomic(sourceType); + Function bucket = Transforms.bucket(numBuckets).bind(sourceIcebergType); + session + .udf() + .register( + funcName, + value -> bucket.apply(SparkValueConverter.convert(sourceIcebergType, value)), + DataTypes.IntegerType); + } + + public static void registerTruncateUDF( + SparkSession session, String funcName, DataType sourceType, int width) { + SparkTypeToType typeConverter = new SparkTypeToType(); + Type sourceIcebergType = typeConverter.atomic(sourceType); + Function truncate = Transforms.truncate(width).bind(sourceIcebergType); + session + .udf() + .register( + funcName, + value -> truncate.apply(SparkValueConverter.convert(sourceIcebergType, value)), + sourceType); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/JobGroupInfo.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/JobGroupInfo.java new file mode 100644 index 000000000000..dc59fc70880e --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/JobGroupInfo.java @@ -0,0 +1,48 @@ +/* + * 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; + +/** Captures information about the current job which is used for displaying on the UI */ +public class JobGroupInfo { + private final String groupId; + private final String description; + private final boolean interruptOnCancel; + + public JobGroupInfo(String groupId, String desc) { + this(groupId, desc, false); + } + + public JobGroupInfo(String groupId, String desc, boolean interruptOnCancel) { + this.groupId = groupId; + this.description = desc; + this.interruptOnCancel = interruptOnCancel; + } + + public String groupId() { + return groupId; + } + + public String description() { + return description; + } + + public boolean interruptOnCancel() { + return interruptOnCancel; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/JobGroupUtils.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/JobGroupUtils.java new file mode 100644 index 000000000000..a6aadf7ebd0e --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/JobGroupUtils.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.function.Supplier; +import org.apache.spark.SparkContext; +import org.apache.spark.SparkContext$; +import org.apache.spark.api.java.JavaSparkContext; + +public class JobGroupUtils { + + private static final String JOB_GROUP_ID = SparkContext$.MODULE$.SPARK_JOB_GROUP_ID(); + private static final String JOB_GROUP_DESC = SparkContext$.MODULE$.SPARK_JOB_DESCRIPTION(); + private static final String JOB_INTERRUPT_ON_CANCEL = + SparkContext$.MODULE$.SPARK_JOB_INTERRUPT_ON_CANCEL(); + + private JobGroupUtils() {} + + public static JobGroupInfo getJobGroupInfo(SparkContext sparkContext) { + String groupId = sparkContext.getLocalProperty(JOB_GROUP_ID); + String description = sparkContext.getLocalProperty(JOB_GROUP_DESC); + String interruptOnCancel = sparkContext.getLocalProperty(JOB_INTERRUPT_ON_CANCEL); + return new JobGroupInfo(groupId, description, Boolean.parseBoolean(interruptOnCancel)); + } + + public static void setJobGroupInfo(SparkContext sparkContext, JobGroupInfo info) { + sparkContext.setLocalProperty(JOB_GROUP_ID, info.groupId()); + sparkContext.setLocalProperty(JOB_GROUP_DESC, info.description()); + sparkContext.setLocalProperty( + JOB_INTERRUPT_ON_CANCEL, String.valueOf(info.interruptOnCancel())); + } + + public static T withJobGroupInfo( + JavaSparkContext sparkContext, JobGroupInfo info, Supplier supplier) { + return withJobGroupInfo(sparkContext.sc(), info, supplier); + } + + public static T withJobGroupInfo( + SparkContext sparkContext, JobGroupInfo info, Supplier supplier) { + JobGroupInfo previousInfo = getJobGroupInfo(sparkContext); + try { + setJobGroupInfo(sparkContext, info); + return supplier.get(); + } finally { + setJobGroupInfo(sparkContext, previousInfo); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/OrcBatchReadConf.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/OrcBatchReadConf.java new file mode 100644 index 000000000000..d3b339d60e3f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/OrcBatchReadConf.java @@ -0,0 +1,27 @@ +/* + * 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.io.Serializable; +import org.immutables.value.Value; + +@Value.Immutable +public interface OrcBatchReadConf extends Serializable { + int batchSize(); +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java new file mode 100644 index 000000000000..442d728d4d69 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java @@ -0,0 +1,29 @@ +/* + * 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.io.Serializable; +import org.immutables.value.Value; + +@Value.Immutable +public interface ParquetBatchReadConf extends Serializable { + int batchSize(); + + ParquetReaderType readerType(); +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java new file mode 100644 index 000000000000..d9742c048251 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java @@ -0,0 +1,47 @@ +/* + * 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 org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Enumerates the types of Parquet readers. */ +public enum ParquetReaderType { + /** ICEBERG type utilizes the built-in Parquet reader. */ + ICEBERG, + + /** + * COMET type changes the Parquet reader to the Apache DataFusion Comet Parquet reader. Comet + * Parquet reader performs I/O and decompression in the JVM but decodes in native to improve + * performance. Additionally, Comet will convert Spark's physical plan into a native physical plan + * and execute this plan natively. + * + *

    TODO: Implement {@link org.apache.comet.parquet.SupportsComet} in SparkScan to convert Spark + * physical plan to native physical plan for native execution. + */ + COMET; + + public static ParquetReaderType fromString(String typeAsString) { + Preconditions.checkArgument(typeAsString != null, "Parquet reader type is null"); + try { + return ParquetReaderType.valueOf(typeAsString.toUpperCase()); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException("Unknown parquet reader type: " + typeAsString); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PathIdentifier.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PathIdentifier.java new file mode 100644 index 000000000000..110af6b87de5 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PathIdentifier.java @@ -0,0 +1,57 @@ +/* + * 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.List; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.base.Splitter; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.spark.sql.connector.catalog.Identifier; + +public class PathIdentifier implements Identifier { + private static final Splitter SPLIT = Splitter.on("/"); + private static final Joiner JOIN = Joiner.on("/"); + private final String[] namespace; + private final String location; + private final String name; + + public PathIdentifier(String location) { + this.location = location; + List pathParts = SPLIT.splitToList(location); + name = Iterables.getLast(pathParts); + namespace = + pathParts.size() > 1 + ? new String[] {JOIN.join(pathParts.subList(0, pathParts.size() - 1))} + : new String[0]; + } + + @Override + public String[] namespace() { + return namespace; + } + + @Override + public String name() { + return name; + } + + public String location() { + return location; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PositionDeletesRewriteCoordinator.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PositionDeletesRewriteCoordinator.java new file mode 100644 index 000000000000..c7568005e22f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PositionDeletesRewriteCoordinator.java @@ -0,0 +1,33 @@ +/* + * 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 org.apache.iceberg.DeleteFile; + +public class PositionDeletesRewriteCoordinator extends BaseFileRewriteCoordinator { + + private static final PositionDeletesRewriteCoordinator INSTANCE = + new PositionDeletesRewriteCoordinator(); + + private PositionDeletesRewriteCoordinator() {} + + public static PositionDeletesRewriteCoordinator get() { + return INSTANCE; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java new file mode 100644 index 000000000000..f4323f1c0350 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithoutReordering.java @@ -0,0 +1,249 @@ +/* + * 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.List; +import java.util.Set; +import java.util.function.Supplier; +import org.apache.iceberg.Schema; +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.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.BinaryType$; +import org.apache.spark.sql.types.BooleanType$; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DateType$; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.DoubleType$; +import org.apache.spark.sql.types.FloatType$; +import org.apache.spark.sql.types.IntegerType$; +import org.apache.spark.sql.types.LongType$; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.NullType$; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.TimestampNTZType$; +import org.apache.spark.sql.types.TimestampType$; + +public class PruneColumnsWithoutReordering extends TypeUtil.CustomOrderSchemaVisitor { + private final StructType requestedType; + private final Set filterRefs; + private DataType current = null; + + PruneColumnsWithoutReordering(StructType requestedType, Set filterRefs) { + this.requestedType = requestedType; + this.filterRefs = filterRefs; + } + + @Override + public Type schema(Schema schema, Supplier structResult) { + this.current = requestedType; + try { + return structResult.get(); + } finally { + this.current = null; + } + } + + @Override + public Type struct(Types.StructType struct, Iterable fieldResults) { + Preconditions.checkNotNull( + struct, "Cannot prune null struct. Pruning must start with a schema."); + Preconditions.checkArgument(current instanceof StructType, "Not a struct: %s", current); + + List fields = struct.fields(); + List types = Lists.newArrayList(fieldResults); + + boolean changed = false; + List newFields = Lists.newArrayListWithExpectedSize(types.size()); + for (int i = 0; i < fields.size(); i += 1) { + Types.NestedField field = fields.get(i); + Type type = types.get(i); + + if (type == null) { + changed = true; + + } else if (field.type() == type) { + newFields.add(field); + + } else if (field.isOptional()) { + changed = true; + newFields.add(Types.NestedField.optional(field.fieldId(), field.name(), type)); + + } else { + changed = true; + newFields.add(Types.NestedField.required(field.fieldId(), field.name(), type)); + } + } + + if (changed) { + return Types.StructType.of(newFields); + } + + return struct; + } + + @Override + public Type field(Types.NestedField field, Supplier fieldResult) { + Preconditions.checkArgument(current instanceof StructType, "Not a struct: %s", current); + StructType requestedStruct = (StructType) current; + + // fields are resolved by name because Spark only sees the current table schema. + if (requestedStruct.getFieldIndex(field.name()).isEmpty()) { + // make sure that filter fields are projected even if they aren't in the requested schema. + if (filterRefs.contains(field.fieldId())) { + return field.type(); + } + return null; + } + + int fieldIndex = requestedStruct.fieldIndex(field.name()); + StructField requestedField = requestedStruct.fields()[fieldIndex]; + + Preconditions.checkArgument( + requestedField.nullable() || field.isRequired(), + "Cannot project an optional field as non-null: %s", + field.name()); + + this.current = requestedField.dataType(); + try { + return fieldResult.get(); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException( + "Invalid projection for field " + field.name() + ": " + e.getMessage(), e); + } finally { + this.current = requestedStruct; + } + } + + @Override + public Type list(Types.ListType list, Supplier elementResult) { + Preconditions.checkArgument(current instanceof ArrayType, "Not an array: %s", current); + ArrayType requestedArray = (ArrayType) current; + + Preconditions.checkArgument( + requestedArray.containsNull() || !list.isElementOptional(), + "Cannot project an array of optional elements as required elements: %s", + requestedArray); + + this.current = requestedArray.elementType(); + try { + Type elementType = elementResult.get(); + if (list.elementType() == elementType) { + return list; + } + + // must be a projected element type, create a new list + if (list.isElementOptional()) { + return Types.ListType.ofOptional(list.elementId(), elementType); + } else { + return Types.ListType.ofRequired(list.elementId(), elementType); + } + } finally { + this.current = requestedArray; + } + } + + @Override + public Type map(Types.MapType map, Supplier keyResult, Supplier valueResult) { + Preconditions.checkArgument(current instanceof MapType, "Not a map: %s", current); + MapType requestedMap = (MapType) current; + + Preconditions.checkArgument( + requestedMap.valueContainsNull() || !map.isValueOptional(), + "Cannot project a map of optional values as required values: %s", + map); + + this.current = requestedMap.valueType(); + try { + Type valueType = valueResult.get(); + if (map.valueType() == valueType) { + return map; + } + + if (map.isValueOptional()) { + return Types.MapType.ofOptional(map.keyId(), map.valueId(), map.keyType(), valueType); + } else { + return Types.MapType.ofRequired(map.keyId(), map.valueId(), map.keyType(), valueType); + } + } finally { + this.current = requestedMap; + } + } + + @Override + public Type variant(Types.VariantType variant) { + return Types.VariantType.get(); + } + + @Override + public Type primitive(Type.PrimitiveType primitive) { + Set> expectedType = TYPES.get(primitive.typeId()); + Preconditions.checkArgument( + expectedType != null && expectedType.contains(current.getClass()), + "Cannot project %s to incompatible type: %s", + primitive, + current); + + // additional checks based on type + switch (primitive.typeId()) { + case DECIMAL: + Types.DecimalType decimal = (Types.DecimalType) primitive; + DecimalType requestedDecimal = (DecimalType) current; + Preconditions.checkArgument( + requestedDecimal.scale() == decimal.scale(), + "Cannot project decimal with incompatible scale: %s != %s", + requestedDecimal.scale(), + decimal.scale()); + Preconditions.checkArgument( + requestedDecimal.precision() >= decimal.precision(), + "Cannot project decimal with incompatible precision: %s < %s", + requestedDecimal.precision(), + decimal.precision()); + break; + default: + } + + return primitive; + } + + private static final ImmutableMap>> TYPES = + ImmutableMap.>>builder() + .put(TypeID.BOOLEAN, ImmutableSet.of(BooleanType$.class)) + .put(TypeID.INTEGER, ImmutableSet.of(IntegerType$.class)) + .put(TypeID.LONG, ImmutableSet.of(LongType$.class)) + .put(TypeID.FLOAT, ImmutableSet.of(FloatType$.class)) + .put(TypeID.DOUBLE, ImmutableSet.of(DoubleType$.class)) + .put(TypeID.DATE, ImmutableSet.of(DateType$.class)) + .put(TypeID.TIMESTAMP, ImmutableSet.of(TimestampType$.class, TimestampNTZType$.class)) + .put(TypeID.DECIMAL, ImmutableSet.of(DecimalType.class)) + .put(TypeID.UUID, ImmutableSet.of(StringType$.class)) + .put(TypeID.STRING, ImmutableSet.of(StringType$.class)) + .put(TypeID.FIXED, ImmutableSet.of(BinaryType$.class)) + .put(TypeID.BINARY, ImmutableSet.of(BinaryType$.class)) + .put(TypeID.UNKNOWN, ImmutableSet.of(NullType$.class)) + .buildOrThrow(); +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java new file mode 100644 index 000000000000..2e90dc7749d1 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RemoveCarryoverIterator.java @@ -0,0 +1,148 @@ +/* + * 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.Iterator; +import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; + +/** + * An iterator that removes the carry-over rows from changelog tables within a single Spark task. It + * assumes that rows are partitioned by identifier(or all) columns, and it is sorted by both + * identifier(or all) columns and change type. + * + *

    Carry-over rows are the result of a removal and insertion of the same row within an operation + * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1, + * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this + * file and preserving row1 in a new file. The change-log table would report this as follows, + * despite it not being an actual change to the table. + * + *

      + *
    • (id=1, data='a', op='DELETE') + *
    • (id=1, data='a', op='INSERT') + *
    • (id=2, data='b', op='DELETE') + *
    + * + * The iterator finds the carry-over rows and removes them from the result. For example, the above + * rows will be converted to: + * + *
      + *
    • (id=2, data='b', op='DELETE') + *
    + */ +class RemoveCarryoverIterator extends ChangelogIterator { + private final int[] indicesToIdentifySameRow; + + private Row cachedDeletedRow = null; + private long deletedRowCount = 0; + private Row cachedNextRecord = null; + + RemoveCarryoverIterator(Iterator rowIterator, StructType rowType) { + super(rowIterator, rowType); + this.indicesToIdentifySameRow = generateIndicesToIdentifySameRow(); + } + + @Override + public boolean hasNext() { + if (hasCachedDeleteRow() || cachedNextRecord != null) { + return true; + } + return rowIterator().hasNext(); + } + + @Override + public Row next() { + Row currentRow; + + if (returnCachedDeleteRow()) { + // Non-carryover delete rows found. One or more identical delete rows were seen followed by a + // non-identical row. This means none of the delete rows were carry over rows. Emit one + // delete row and decrease the amount of delete rows seen. + deletedRowCount--; + currentRow = cachedDeletedRow; + if (deletedRowCount == 0) { + cachedDeletedRow = null; + } + return currentRow; + } else if (cachedNextRecord != null) { + currentRow = cachedNextRecord; + cachedNextRecord = null; + } else { + currentRow = rowIterator().next(); + } + + // If the current row is a delete row, drain all identical delete rows + if (changeType(currentRow).equals(DELETE) && rowIterator().hasNext()) { + cachedDeletedRow = currentRow; + deletedRowCount = 1; + + Row nextRow = rowIterator().next(); + + // drain all identical delete rows when there is at least one cached delete row and the next + // row is the same record + while (nextRow != null + && cachedDeletedRow != null + && isSameRecord(cachedDeletedRow, nextRow, indicesToIdentifySameRow)) { + if (changeType(nextRow).equals(INSERT)) { + deletedRowCount--; + if (deletedRowCount == 0) { + cachedDeletedRow = null; + } + } else { + deletedRowCount++; + } + + if (rowIterator().hasNext()) { + nextRow = rowIterator().next(); + } else { + nextRow = null; + } + } + + cachedNextRecord = nextRow; + return null; + } else { + // either there is no cached delete row or the current row is not a delete row + return currentRow; + } + } + + /** + * The iterator returns a cached delete row if there are delete rows cached and the next row is + * not the same record or there is no next row. + */ + private boolean returnCachedDeleteRow() { + return hitBoundary() && hasCachedDeleteRow(); + } + + private boolean hitBoundary() { + return !rowIterator().hasNext() || cachedNextRecord != null; + } + + private boolean hasCachedDeleteRow() { + return cachedDeletedRow != null; + } + + private int[] generateIndicesToIdentifySameRow() { + Set metadataColumnIndices = Sets.newHashSet(changeTypeIndex()); + return generateIndicesToIdentifySameRow(rowType().size(), metadataColumnIndices); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java new file mode 100644 index 000000000000..941e4a4731e2 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RemoveNetCarryoverIterator.java @@ -0,0 +1,129 @@ +/* + * 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.Iterator; +import java.util.Set; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; + +/** + * This class computes the net changes across multiple snapshots. It is different from {@link + * org.apache.iceberg.spark.RemoveCarryoverIterator}, which only removes carry-over rows within a + * single snapshot. It takes a row iterator, and assumes the following: + * + *
      + *
    • The row iterator is partitioned by all columns. + *
    • The row iterator is sorted by all columns, change order, and change type. The change order + * is 1-to-1 mapping to snapshot id. + *
    + */ +public class RemoveNetCarryoverIterator extends ChangelogIterator { + + private final int[] indicesToIdentifySameRow; + + private Row cachedNextRow; + private Row cachedRow; + private long cachedRowCount; + + protected RemoveNetCarryoverIterator(Iterator rowIterator, StructType rowType) { + super(rowIterator, rowType); + this.indicesToIdentifySameRow = generateIndicesToIdentifySameRow(); + } + + @Override + public boolean hasNext() { + if (cachedRowCount > 0) { + return true; + } + + if (cachedNextRow != null) { + return true; + } + + return rowIterator().hasNext(); + } + + @Override + public Row next() { + // if there are cached rows, return one of them from the beginning + if (cachedRowCount > 0) { + cachedRowCount--; + return cachedRow; + } + + cachedRow = getCurrentRow(); + // return it directly if there is no more rows + if (!rowIterator().hasNext()) { + return cachedRow; + } + cachedRowCount = 1; + + cachedNextRow = rowIterator().next(); + + // pull rows from the iterator until two consecutive rows are different + while (isSameRecord(cachedRow, cachedNextRow, indicesToIdentifySameRow)) { + if (oppositeChangeType(cachedRow, cachedNextRow)) { + // two rows with opposite change types means no net changes, remove both + cachedRowCount--; + } else { + // two rows with same change types means potential net changes, cache the next row + cachedRowCount++; + } + + // stop pulling rows if there is no more rows or the next row is different + if (cachedRowCount <= 0 || !rowIterator().hasNext()) { + // reset the cached next row if there is no more rows + cachedNextRow = null; + break; + } + + cachedNextRow = rowIterator().next(); + } + + return null; + } + + private Row getCurrentRow() { + Row currentRow; + if (cachedNextRow != null) { + currentRow = cachedNextRow; + cachedNextRow = null; + } else { + currentRow = rowIterator().next(); + } + return currentRow; + } + + private boolean oppositeChangeType(Row currentRow, Row nextRow) { + return (changeType(nextRow).equals(INSERT) && changeType(currentRow).equals(DELETE)) + || (changeType(nextRow).equals(DELETE) && changeType(currentRow).equals(INSERT)); + } + + private int[] generateIndicesToIdentifySameRow() { + Set metadataColumnIndices = + Sets.newHashSet( + rowType().fieldIndex(MetadataColumns.CHANGE_ORDINAL.name()), + rowType().fieldIndex(MetadataColumns.COMMIT_SNAPSHOT_ID.name()), + changeTypeIndex()); + return generateIndicesToIdentifySameRow(rowType().size(), metadataColumnIndices); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RollbackStagedTable.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RollbackStagedTable.java new file mode 100644 index 000000000000..bc8a966488ee --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/RollbackStagedTable.java @@ -0,0 +1,142 @@ +/* + * 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.Map; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Function; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.StagedTable; +import org.apache.spark.sql.connector.catalog.SupportsDelete; +import org.apache.spark.sql.connector.catalog.SupportsRead; +import org.apache.spark.sql.connector.catalog.SupportsWrite; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCapability; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.WriteBuilder; +import org.apache.spark.sql.sources.Filter; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * An implementation of StagedTable that mimics the behavior of Spark's non-atomic CTAS and RTAS. + * + *

    A Spark catalog can implement StagingTableCatalog to support atomic operations by producing + * StagedTable. But if a catalog implements StagingTableCatalog, Spark expects the catalog to be + * able to produce a StagedTable for any table loaded by the catalog. This assumption doesn't always + * work, as in the case of {@link SparkSessionCatalog}, which supports atomic operations can produce + * a StagedTable for Iceberg tables, but wraps the session catalog and cannot necessarily produce a + * working StagedTable implementation for tables that it loads. + * + *

    The work-around is this class, which implements the StagedTable interface but does not have + * atomic behavior. Instead, the StagedTable interface is used to implement the behavior of the + * non-atomic SQL plans that will create a table, write, and will drop the table to roll back. + * + *

    This StagedTable implements SupportsRead, SupportsWrite, and SupportsDelete by passing the + * calls to the real table. Implementing those interfaces is safe because Spark will not use them + * unless the table supports them and returns the corresponding capabilities from {@link + * #capabilities()}. + */ +public class RollbackStagedTable + implements StagedTable, SupportsRead, SupportsWrite, SupportsDelete { + private final TableCatalog catalog; + private final Identifier ident; + private final Table table; + + public RollbackStagedTable(TableCatalog catalog, Identifier ident, Table table) { + this.catalog = catalog; + this.ident = ident; + this.table = table; + } + + @Override + public void commitStagedChanges() { + // the changes have already been committed to the table at the end of the write + } + + @Override + public void abortStagedChanges() { + // roll back changes by dropping the table + catalog.dropTable(ident); + } + + @Override + public String name() { + return table.name(); + } + + @Override + public StructType schema() { + return table.schema(); + } + + @Override + public Transform[] partitioning() { + return table.partitioning(); + } + + @Override + public Map properties() { + return table.properties(); + } + + @Override + public Set capabilities() { + return table.capabilities(); + } + + @Override + public void deleteWhere(Filter[] filters) { + call(SupportsDelete.class, t -> t.deleteWhere(filters)); + } + + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + return callReturning(SupportsRead.class, t -> t.newScanBuilder(options)); + } + + @Override + public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { + return callReturning(SupportsWrite.class, t -> t.newWriteBuilder(info)); + } + + private void call(Class requiredClass, Consumer task) { + callReturning( + requiredClass, + inst -> { + task.accept(inst); + return null; + }); + } + + private R callReturning(Class requiredClass, Function task) { + if (requiredClass.isInstance(table)) { + return task.apply(requiredClass.cast(table)); + } else { + throw new UnsupportedOperationException( + String.format( + "Table does not implement %s: %s (%s)", + requiredClass.getSimpleName(), table.name(), table.getClass().getName())); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java new file mode 100644 index 000000000000..cab40d103171 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ScanTaskSetManager.java @@ -0,0 +1,73 @@ +/* + * 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.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; + +public class ScanTaskSetManager { + + private static final ScanTaskSetManager INSTANCE = new ScanTaskSetManager(); + + private final Map, List> tasksMap = + Maps.newConcurrentMap(); + + private ScanTaskSetManager() {} + + public static ScanTaskSetManager get() { + return INSTANCE; + } + + public void stageTasks(Table table, String setId, List tasks) { + Preconditions.checkArgument( + tasks != null && !tasks.isEmpty(), "Cannot stage null or empty tasks"); + Pair id = toId(table, setId); + tasksMap.put(id, tasks); + } + + @SuppressWarnings("unchecked") + public List fetchTasks(Table table, String setId) { + Pair id = toId(table, setId); + return (List) tasksMap.get(id); + } + + @SuppressWarnings("unchecked") + public List removeTasks(Table table, String setId) { + Pair id = toId(table, setId); + return (List) tasksMap.remove(id); + } + + public Set fetchSetIds(Table table) { + return tasksMap.keySet().stream() + .filter(e -> e.first().equals(Spark3Util.baseTableUUID(table))) + .map(Pair::second) + .collect(Collectors.toSet()); + } + + private Pair toId(Table table, String setId) { + return Pair.of(Spark3Util.baseTableUUID(table), setId); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SortOrderToSpark.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SortOrderToSpark.java new file mode 100644 index 000000000000..781f61b33f0e --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SortOrderToSpark.java @@ -0,0 +1,101 @@ +/* + * 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.Map; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.transforms.SortOrderVisitor; +import org.apache.spark.sql.connector.expressions.Expressions; +import org.apache.spark.sql.connector.expressions.NullOrdering; +import org.apache.spark.sql.connector.expressions.SortOrder; + +class SortOrderToSpark implements SortOrderVisitor { + + private final Map quotedNameById; + + SortOrderToSpark(Schema schema) { + this.quotedNameById = SparkSchemaUtil.indexQuotedNameById(schema); + } + + @Override + public SortOrder field(String sourceName, int id, SortDirection direction, NullOrder nullOrder) { + return Expressions.sort( + Expressions.column(quotedName(id)), toSpark(direction), toSpark(nullOrder)); + } + + @Override + public SortOrder bucket( + String sourceName, int id, int width, SortDirection direction, NullOrder nullOrder) { + return Expressions.sort( + Expressions.bucket(width, quotedName(id)), toSpark(direction), toSpark(nullOrder)); + } + + @Override + public SortOrder truncate( + String sourceName, int id, int width, SortDirection direction, NullOrder nullOrder) { + return Expressions.sort( + Expressions.apply( + "truncate", Expressions.literal(width), Expressions.column(quotedName(id))), + toSpark(direction), + toSpark(nullOrder)); + } + + @Override + public SortOrder year(String sourceName, int id, SortDirection direction, NullOrder nullOrder) { + return Expressions.sort( + Expressions.years(quotedName(id)), toSpark(direction), toSpark(nullOrder)); + } + + @Override + public SortOrder month(String sourceName, int id, SortDirection direction, NullOrder nullOrder) { + return Expressions.sort( + Expressions.months(quotedName(id)), toSpark(direction), toSpark(nullOrder)); + } + + @Override + public SortOrder day(String sourceName, int id, SortDirection direction, NullOrder nullOrder) { + return Expressions.sort( + Expressions.days(quotedName(id)), toSpark(direction), toSpark(nullOrder)); + } + + @Override + public SortOrder hour(String sourceName, int id, SortDirection direction, NullOrder nullOrder) { + return Expressions.sort( + Expressions.hours(quotedName(id)), toSpark(direction), toSpark(nullOrder)); + } + + private String quotedName(int id) { + return quotedNameById.get(id); + } + + private org.apache.spark.sql.connector.expressions.SortDirection toSpark( + SortDirection direction) { + if (direction == SortDirection.ASC) { + return org.apache.spark.sql.connector.expressions.SortDirection.ASCENDING; + } else { + return org.apache.spark.sql.connector.expressions.SortDirection.DESCENDING; + } + } + + private NullOrdering toSpark(NullOrder nullOrder) { + return nullOrder == NullOrder.NULLS_FIRST ? NullOrdering.NULLS_FIRST : NullOrdering.NULLS_LAST; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java new file mode 100644 index 000000000000..df42175c3476 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java @@ -0,0 +1,1079 @@ +/* + * 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.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.BaseMetadataTable; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.UpdateProperties; +import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.expressions.BoundPredicate; +import org.apache.iceberg.expressions.ExpressionVisitors; +import org.apache.iceberg.expressions.Term; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.expressions.UnboundTerm; +import org.apache.iceberg.expressions.UnboundTransform; +import org.apache.iceberg.expressions.Zorder; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +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.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; +import org.apache.iceberg.spark.SparkTableUtil.SparkPartition; +import org.apache.iceberg.spark.source.HasIcebergCatalog; +import org.apache.iceberg.spark.source.SparkTable; +import org.apache.iceberg.transforms.PartitionSpecVisitor; +import org.apache.iceberg.transforms.SortOrderVisitor; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.CatalystTypeConverters; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.catalyst.parser.ParserInterface; +import org.apache.spark.sql.connector.catalog.CatalogManager; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.TableChange; +import org.apache.spark.sql.connector.expressions.Expression; +import org.apache.spark.sql.connector.expressions.Expressions; +import org.apache.spark.sql.connector.expressions.Literal; +import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.expressions.SortOrder; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.execution.datasources.FileStatusCache; +import org.apache.spark.sql.execution.datasources.FileStatusWithMetadata; +import org.apache.spark.sql.execution.datasources.InMemoryFileIndex; +import org.apache.spark.sql.execution.datasources.PartitionDirectory; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.types.LongType; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Option; +import scala.collection.JavaConverters; +import scala.collection.immutable.Seq; + +public class Spark3Util { + + private static final Logger LOG = LoggerFactory.getLogger(Spark3Util.class); + + private static final Set RESERVED_PROPERTIES = + ImmutableSet.of(TableCatalog.PROP_LOCATION, TableCatalog.PROP_PROVIDER); + private static final Joiner DOT = Joiner.on("."); + private static final String HIVE_NULL = "__HIVE_DEFAULT_PARTITION__"; + + 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() + .filter(entry -> !RESERVED_PROPERTIES.contains(entry.getKey())) + .forEach(tableProperties::put); + + String provider = createProperties.get(TableCatalog.PROP_PROVIDER); + if ("parquet".equalsIgnoreCase(provider)) { + tableProperties.put(TableProperties.DEFAULT_FILE_FORMAT, "parquet"); + } else if ("avro".equalsIgnoreCase(provider)) { + tableProperties.put(TableProperties.DEFAULT_FILE_FORMAT, "avro"); + } else if ("orc".equalsIgnoreCase(provider)) { + tableProperties.put(TableProperties.DEFAULT_FILE_FORMAT, "orc"); + } else if (provider != null && !"iceberg".equalsIgnoreCase(provider)) { + throw new IllegalArgumentException("Unsupported format in USING: " + provider); + } + + return tableProperties.build(); + } + + /** + * Applies a list of Spark table changes to an {@link UpdateProperties} operation. + * + * @param pendingUpdate an uncommitted UpdateProperties operation to configure + * @param changes a list of Spark table changes + * @return the UpdateProperties operation configured with the changes + */ + public static UpdateProperties applyPropertyChanges( + UpdateProperties pendingUpdate, List changes) { + for (TableChange change : changes) { + if (change instanceof TableChange.SetProperty) { + TableChange.SetProperty set = (TableChange.SetProperty) change; + pendingUpdate.set(set.property(), set.value()); + + } else if (change instanceof TableChange.RemoveProperty) { + TableChange.RemoveProperty remove = (TableChange.RemoveProperty) change; + pendingUpdate.remove(remove.property()); + + } else { + throw new UnsupportedOperationException("Cannot apply unknown table change: " + change); + } + } + + return pendingUpdate; + } + + /** + * Applies a list of Spark table changes to an {@link UpdateSchema} operation. + * + * @param pendingUpdate an uncommitted UpdateSchema operation to configure + * @param changes a list of Spark table changes + * @return the UpdateSchema operation configured with the changes + */ + public static UpdateSchema applySchemaChanges( + UpdateSchema pendingUpdate, List changes) { + for (TableChange change : changes) { + if (change instanceof TableChange.AddColumn) { + apply(pendingUpdate, (TableChange.AddColumn) change); + + } else if (change instanceof TableChange.UpdateColumnType) { + TableChange.UpdateColumnType update = (TableChange.UpdateColumnType) change; + Type newType = SparkSchemaUtil.convert(update.newDataType()); + Preconditions.checkArgument( + newType.isPrimitiveType(), + "Cannot update '%s', not a primitive type: %s", + DOT.join(update.fieldNames()), + update.newDataType()); + pendingUpdate.updateColumn(DOT.join(update.fieldNames()), newType.asPrimitiveType()); + + } else if (change instanceof TableChange.UpdateColumnComment) { + TableChange.UpdateColumnComment update = (TableChange.UpdateColumnComment) change; + pendingUpdate.updateColumnDoc(DOT.join(update.fieldNames()), update.newComment()); + + } else if (change instanceof TableChange.RenameColumn) { + TableChange.RenameColumn rename = (TableChange.RenameColumn) change; + pendingUpdate.renameColumn(DOT.join(rename.fieldNames()), rename.newName()); + + } else if (change instanceof TableChange.DeleteColumn) { + TableChange.DeleteColumn delete = (TableChange.DeleteColumn) change; + pendingUpdate.deleteColumn(DOT.join(delete.fieldNames())); + + } else if (change instanceof TableChange.UpdateColumnNullability) { + TableChange.UpdateColumnNullability update = (TableChange.UpdateColumnNullability) change; + if (update.nullable()) { + pendingUpdate.makeColumnOptional(DOT.join(update.fieldNames())); + } else { + pendingUpdate.requireColumn(DOT.join(update.fieldNames())); + } + + } else if (change instanceof TableChange.UpdateColumnPosition) { + apply(pendingUpdate, (TableChange.UpdateColumnPosition) change); + + } else { + throw new UnsupportedOperationException("Cannot apply unknown table change: " + change); + } + } + + return pendingUpdate; + } + + private static void apply(UpdateSchema pendingUpdate, TableChange.UpdateColumnPosition update) { + Preconditions.checkArgument(update.position() != null, "Invalid position: null"); + + if (update.position() instanceof TableChange.After) { + TableChange.After after = (TableChange.After) update.position(); + String referenceField = peerName(update.fieldNames(), after.column()); + pendingUpdate.moveAfter(DOT.join(update.fieldNames()), referenceField); + + } else if (update.position() instanceof TableChange.First) { + pendingUpdate.moveFirst(DOT.join(update.fieldNames())); + + } else { + throw new IllegalArgumentException("Unknown position for reorder: " + update.position()); + } + } + + private static void apply(UpdateSchema pendingUpdate, TableChange.AddColumn add) { + Preconditions.checkArgument( + add.isNullable(), + "Incompatible change: cannot add required column: %s", + leafName(add.fieldNames())); + if (add.defaultValue() != null) { + throw new UnsupportedOperationException( + String.format( + "Cannot add column %s since setting default values in Spark is currently unsupported", + leafName(add.fieldNames()))); + } + + Type type = SparkSchemaUtil.convert(add.dataType()); + pendingUpdate.addColumn( + parentName(add.fieldNames()), leafName(add.fieldNames()), type, add.comment()); + + if (add.position() instanceof TableChange.After) { + TableChange.After after = (TableChange.After) add.position(); + String referenceField = peerName(add.fieldNames(), after.column()); + pendingUpdate.moveAfter(DOT.join(add.fieldNames()), referenceField); + + } else if (add.position() instanceof TableChange.First) { + pendingUpdate.moveFirst(DOT.join(add.fieldNames())); + + } else { + Preconditions.checkArgument( + add.position() == null, + "Cannot add '%s' at unknown position: %s", + DOT.join(add.fieldNames()), + add.position()); + } + } + + public static org.apache.iceberg.Table toIcebergTable(Table table) { + Preconditions.checkArgument( + table instanceof SparkTable, "Table %s is not an Iceberg table", table); + SparkTable sparkTable = (SparkTable) table; + return sparkTable.table(); + } + + public static SortOrder[] toOrdering(org.apache.iceberg.SortOrder sortOrder) { + SortOrderToSpark visitor = new SortOrderToSpark(sortOrder.schema()); + List ordering = SortOrderVisitor.visit(sortOrder, visitor); + return ordering.toArray(new SortOrder[0]); + } + + public static Transform[] toTransforms(Schema schema, List fields) { + SpecTransformToSparkTransform visitor = new SpecTransformToSparkTransform(schema); + + List transforms = Lists.newArrayList(); + + for (PartitionField field : fields) { + Transform transform = PartitionSpecVisitor.visit(schema, field, visitor); + if (transform != null) { + transforms.add(transform); + } + } + + return transforms.toArray(new Transform[0]); + } + + /** + * Converts a PartitionSpec to Spark transforms. + * + * @param spec a PartitionSpec + * @return an array of Transforms + */ + public static Transform[] toTransforms(PartitionSpec spec) { + SpecTransformToSparkTransform visitor = new SpecTransformToSparkTransform(spec.schema()); + List transforms = PartitionSpecVisitor.visit(spec, visitor); + return transforms.stream().filter(Objects::nonNull).toArray(Transform[]::new); + } + + private static class SpecTransformToSparkTransform implements PartitionSpecVisitor { + private final Map quotedNameById; + + SpecTransformToSparkTransform(Schema schema) { + this.quotedNameById = SparkSchemaUtil.indexQuotedNameById(schema); + } + + @Override + public Transform identity(String sourceName, int sourceId) { + return Expressions.identity(quotedName(sourceId)); + } + + @Override + public Transform bucket(String sourceName, int sourceId, int numBuckets) { + return Expressions.bucket(numBuckets, quotedName(sourceId)); + } + + @Override + public Transform truncate(String sourceName, int sourceId, int width) { + NamedReference column = Expressions.column(quotedName(sourceId)); + return Expressions.apply("truncate", Expressions.literal(width), column); + } + + @Override + public Transform year(String sourceName, int sourceId) { + return Expressions.years(quotedName(sourceId)); + } + + @Override + public Transform month(String sourceName, int sourceId) { + return Expressions.months(quotedName(sourceId)); + } + + @Override + public Transform day(String sourceName, int sourceId) { + return Expressions.days(quotedName(sourceId)); + } + + @Override + public Transform hour(String sourceName, int sourceId) { + return Expressions.hours(quotedName(sourceId)); + } + + @Override + public Transform alwaysNull(int fieldId, String sourceName, int sourceId) { + // do nothing for alwaysNull, it doesn't need to be converted to a transform + return null; + } + + @Override + public Transform unknown(int fieldId, String sourceName, int sourceId, String transform) { + return Expressions.apply(transform, Expressions.column(quotedName(sourceId))); + } + + private String quotedName(int id) { + return quotedNameById.get(id); + } + } + + public static NamedReference toNamedReference(String name) { + return Expressions.column(name); + } + + public static Term toIcebergTerm(Expression expr) { + if (expr instanceof Transform) { + Transform transform = (Transform) expr; + Preconditions.checkArgument( + "zorder".equals(transform.name()) || transform.references().length == 1, + "Cannot convert transform with more than one column reference: %s", + transform); + String colName = DOT.join(transform.references()[0].fieldNames()); + switch (transform.name().toLowerCase(Locale.ROOT)) { + case "identity": + return org.apache.iceberg.expressions.Expressions.ref(colName); + case "bucket": + return org.apache.iceberg.expressions.Expressions.bucket(colName, findWidth(transform)); + case "year": + case "years": + return org.apache.iceberg.expressions.Expressions.year(colName); + case "month": + case "months": + return org.apache.iceberg.expressions.Expressions.month(colName); + case "date": + case "day": + case "days": + return org.apache.iceberg.expressions.Expressions.day(colName); + case "date_hour": + case "hour": + case "hours": + return org.apache.iceberg.expressions.Expressions.hour(colName); + case "truncate": + return org.apache.iceberg.expressions.Expressions.truncate(colName, findWidth(transform)); + case "zorder": + return new Zorder( + Stream.of(transform.references()) + .map(ref -> DOT.join(ref.fieldNames())) + .map(org.apache.iceberg.expressions.Expressions::ref) + .collect(Collectors.toList())); + default: + throw new UnsupportedOperationException("Transform is not supported: " + transform); + } + + } else if (expr instanceof NamedReference) { + NamedReference ref = (NamedReference) expr; + return org.apache.iceberg.expressions.Expressions.ref(DOT.join(ref.fieldNames())); + + } else { + throw new UnsupportedOperationException("Cannot convert unknown expression: " + expr); + } + } + + /** + * Converts Spark transforms into a {@link PartitionSpec}. + * + * @param schema the table schema + * @param partitioning Spark Transforms + * @return a PartitionSpec + */ + public static PartitionSpec toPartitionSpec(Schema schema, Transform[] partitioning) { + if (partitioning == null || partitioning.length == 0) { + return PartitionSpec.unpartitioned(); + } + + PartitionSpec.Builder builder = PartitionSpec.builderFor(schema); + for (Transform transform : partitioning) { + Preconditions.checkArgument( + transform.references().length == 1, + "Cannot convert transform with more than one column reference: %s", + transform); + String colName = DOT.join(transform.references()[0].fieldNames()); + switch (transform.name().toLowerCase(Locale.ROOT)) { + case "identity": + builder.identity(colName); + break; + case "bucket": + builder.bucket(colName, findWidth(transform)); + break; + case "year": + case "years": + builder.year(colName); + break; + case "month": + case "months": + builder.month(colName); + break; + case "date": + case "day": + case "days": + builder.day(colName); + break; + case "date_hour": + case "hour": + case "hours": + builder.hour(colName); + break; + case "truncate": + builder.truncate(colName, findWidth(transform)); + break; + default: + throw new UnsupportedOperationException("Transform is not supported: " + transform); + } + } + + return builder.build(); + } + + @SuppressWarnings("unchecked") + private static int findWidth(Transform transform) { + for (Expression expr : transform.arguments()) { + if (expr instanceof Literal) { + if (((Literal) expr).dataType() instanceof IntegerType) { + Literal lit = (Literal) expr; + Preconditions.checkArgument( + lit.value() > 0, "Unsupported width for transform: %s", transform.describe()); + return lit.value(); + + } else if (((Literal) expr).dataType() instanceof LongType) { + Literal lit = (Literal) expr; + Preconditions.checkArgument( + lit.value() > 0 && lit.value() < Integer.MAX_VALUE, + "Unsupported width for transform: %s", + transform.describe()); + if (lit.value() > Integer.MAX_VALUE) { + throw new IllegalArgumentException(); + } + return lit.value().intValue(); + } + } + } + + throw new IllegalArgumentException("Cannot find width for transform: " + transform.describe()); + } + + private static String leafName(String[] fieldNames) { + Preconditions.checkArgument( + fieldNames.length > 0, "Invalid field name: at least one name is required"); + return fieldNames[fieldNames.length - 1]; + } + + private static String peerName(String[] fieldNames, String fieldName) { + if (fieldNames.length > 1) { + String[] peerNames = Arrays.copyOf(fieldNames, fieldNames.length); + peerNames[fieldNames.length - 1] = fieldName; + return DOT.join(peerNames); + } + return fieldName; + } + + private static String parentName(String[] fieldNames) { + if (fieldNames.length > 1) { + return DOT.join(Arrays.copyOfRange(fieldNames, 0, fieldNames.length - 1)); + } + return null; + } + + public static String describe(List exprs) { + return exprs.stream().map(Spark3Util::describe).collect(Collectors.joining(", ")); + } + + public static String describe(org.apache.iceberg.expressions.Expression expr) { + return ExpressionVisitors.visit(expr, DescribeExpressionVisitor.INSTANCE); + } + + public static String describe(Schema schema) { + return TypeUtil.visit(schema, DescribeSchemaVisitor.INSTANCE); + } + + public static String describe(Type type) { + return TypeUtil.visit(type, DescribeSchemaVisitor.INSTANCE); + } + + public static String describe(org.apache.iceberg.SortOrder order) { + return Joiner.on(", ").join(SortOrderVisitor.visit(order, DescribeSortOrderVisitor.INSTANCE)); + } + + public static boolean extensionsEnabled(SparkSession spark) { + String extensions = spark.conf().get("spark.sql.extensions", ""); + return extensions.contains("IcebergSparkSessionExtensions"); + } + + public static class DescribeSchemaVisitor extends TypeUtil.SchemaVisitor { + private static final Joiner COMMA = Joiner.on(','); + private static final DescribeSchemaVisitor INSTANCE = new DescribeSchemaVisitor(); + + private DescribeSchemaVisitor() {} + + @Override + public String schema(Schema schema, String structResult) { + return structResult; + } + + @Override + public String struct(Types.StructType struct, List fieldResults) { + return "struct<" + COMMA.join(fieldResults) + ">"; + } + + @Override + public String field(Types.NestedField field, String fieldResult) { + return field.name() + ": " + fieldResult + (field.isRequired() ? " not null" : ""); + } + + @Override + public String list(Types.ListType list, String elementResult) { + return "list<" + elementResult + ">"; + } + + @Override + public String map(Types.MapType map, String keyResult, String valueResult) { + return "map<" + keyResult + ", " + valueResult + ">"; + } + + @Override + public String variant(Types.VariantType variant) { + return "variant"; + } + + @Override + public String primitive(Type.PrimitiveType primitive) { + switch (primitive.typeId()) { + case BOOLEAN: + return "boolean"; + case INTEGER: + return "int"; + case LONG: + return "bigint"; + case FLOAT: + return "float"; + case DOUBLE: + return "double"; + case DATE: + return "date"; + case TIME: + return "time"; + case TIMESTAMP: + return "timestamp"; + case STRING: + case UUID: + return "string"; + case FIXED: + case BINARY: + return "binary"; + case DECIMAL: + Types.DecimalType decimal = (Types.DecimalType) primitive; + return "decimal(" + decimal.precision() + "," + decimal.scale() + ")"; + } + throw new UnsupportedOperationException("Cannot convert type to SQL: " + primitive); + } + } + + private static class DescribeExpressionVisitor + extends ExpressionVisitors.ExpressionVisitor { + private static final DescribeExpressionVisitor INSTANCE = new DescribeExpressionVisitor(); + + private DescribeExpressionVisitor() {} + + @Override + public String alwaysTrue() { + return "true"; + } + + @Override + public String alwaysFalse() { + return "false"; + } + + @Override + public String not(String result) { + return "NOT (" + result + ")"; + } + + @Override + public String and(String leftResult, String rightResult) { + return "(" + leftResult + " AND " + rightResult + ")"; + } + + @Override + public String or(String leftResult, String rightResult) { + return "(" + leftResult + " OR " + rightResult + ")"; + } + + @Override + public String predicate(BoundPredicate pred) { + throw new UnsupportedOperationException("Cannot convert bound predicates to SQL"); + } + + @Override + public String predicate(UnboundPredicate pred) { + switch (pred.op()) { + case IS_NULL: + return sqlString(pred.term()) + " IS NULL"; + case NOT_NULL: + return sqlString(pred.term()) + " IS NOT NULL"; + case IS_NAN: + return "is_nan(" + sqlString(pred.term()) + ")"; + case NOT_NAN: + return "not_nan(" + sqlString(pred.term()) + ")"; + case LT: + return sqlString(pred.term()) + " < " + sqlString(pred.literal()); + case LT_EQ: + return sqlString(pred.term()) + " <= " + sqlString(pred.literal()); + case GT: + return sqlString(pred.term()) + " > " + sqlString(pred.literal()); + case GT_EQ: + return sqlString(pred.term()) + " >= " + sqlString(pred.literal()); + case EQ: + return sqlString(pred.term()) + " = " + sqlString(pred.literal()); + case NOT_EQ: + return sqlString(pred.term()) + " != " + sqlString(pred.literal()); + case STARTS_WITH: + return sqlString(pred.term()) + " LIKE '" + pred.literal().value() + "%'"; + case NOT_STARTS_WITH: + return sqlString(pred.term()) + " NOT LIKE '" + pred.literal().value() + "%'"; + case IN: + return sqlString(pred.term()) + " IN (" + sqlString(pred.literals()) + ")"; + case NOT_IN: + return sqlString(pred.term()) + " NOT IN (" + sqlString(pred.literals()) + ")"; + default: + throw new UnsupportedOperationException("Cannot convert predicate to SQL: " + pred); + } + } + + private static String sqlString(UnboundTerm term) { + if (term instanceof org.apache.iceberg.expressions.NamedReference) { + return term.ref().name(); + } else if (term instanceof UnboundTransform) { + UnboundTransform transform = (UnboundTransform) term; + return transform.transform().toString() + "(" + transform.ref().name() + ")"; + } else { + throw new UnsupportedOperationException("Cannot convert term to SQL: " + term); + } + } + + private static String sqlString(List> literals) { + return literals.stream() + .map(DescribeExpressionVisitor::sqlString) + .collect(Collectors.joining(", ")); + } + + private static String sqlString(org.apache.iceberg.expressions.Literal lit) { + if (lit.value() instanceof String) { + return "'" + lit.value() + "'"; + } else if (lit.value() instanceof ByteBuffer) { + byte[] bytes = ByteBuffers.toByteArray((ByteBuffer) lit.value()); + return "X'" + BaseEncoding.base16().encode(bytes) + "'"; + } else { + return lit.value().toString(); + } + } + } + + /** + * Returns an Iceberg Table by its name from a Spark V2 Catalog. If cache is enabled in {@link + * SparkCatalog}, the {@link TableOperations} of the table may be stale, please refresh the table + * to get the latest one. + * + * @param spark SparkSession used for looking up catalog references and tables + * @param name The multipart identifier of the Iceberg table + * @return an Iceberg table + */ + public static org.apache.iceberg.Table loadIcebergTable(SparkSession spark, String name) + throws ParseException, NoSuchTableException { + CatalogAndIdentifier catalogAndIdentifier = catalogAndIdentifier(spark, name); + + TableCatalog catalog = asTableCatalog(catalogAndIdentifier.catalog); + Table sparkTable = catalog.loadTable(catalogAndIdentifier.identifier); + return toIcebergTable(sparkTable); + } + + /** + * Returns the underlying Iceberg Catalog object represented by a Spark Catalog + * + * @param spark SparkSession used for looking up catalog reference + * @param catalogName The name of the Spark Catalog being referenced + * @return the Iceberg catalog class being wrapped by the Spark Catalog + */ + public static Catalog loadIcebergCatalog(SparkSession spark, String catalogName) { + CatalogPlugin catalogPlugin = spark.sessionState().catalogManager().catalog(catalogName); + Preconditions.checkArgument( + catalogPlugin instanceof HasIcebergCatalog, + String.format( + "Cannot load Iceberg catalog from catalog %s because it does not contain an Iceberg Catalog. " + + "Actual Class: %s", + catalogName, catalogPlugin.getClass().getName())); + return ((HasIcebergCatalog) catalogPlugin).icebergCatalog(); + } + + public static CatalogAndIdentifier catalogAndIdentifier(SparkSession spark, String name) + throws ParseException { + return catalogAndIdentifier( + spark, name, spark.sessionState().catalogManager().currentCatalog()); + } + + public static CatalogAndIdentifier catalogAndIdentifier( + SparkSession spark, String name, CatalogPlugin defaultCatalog) throws ParseException { + ParserInterface parser = spark.sessionState().sqlParser(); + Seq multiPartIdentifier = parser.parseMultipartIdentifier(name).toIndexedSeq(); + List javaMultiPartIdentifier = JavaConverters.seqAsJavaList(multiPartIdentifier); + return catalogAndIdentifier(spark, javaMultiPartIdentifier, defaultCatalog); + } + + public static CatalogAndIdentifier catalogAndIdentifier( + String description, SparkSession spark, String name) { + return catalogAndIdentifier( + description, spark, name, spark.sessionState().catalogManager().currentCatalog()); + } + + public static CatalogAndIdentifier catalogAndIdentifier( + String description, SparkSession spark, String name, CatalogPlugin defaultCatalog) { + try { + return catalogAndIdentifier(spark, name, defaultCatalog); + } catch (ParseException e) { + throw new IllegalArgumentException("Cannot parse " + description + ": " + name, e); + } + } + + public static CatalogAndIdentifier catalogAndIdentifier( + SparkSession spark, List nameParts) { + return catalogAndIdentifier( + spark, nameParts, spark.sessionState().catalogManager().currentCatalog()); + } + + /** + * A modified version of Spark's LookupCatalog.CatalogAndIdentifier.unapply Attempts to find the + * catalog and identifier a multipart identifier represents + * + * @param spark Spark session to use for resolution + * @param nameParts Multipart identifier representing a table + * @param defaultCatalog Catalog to use if none is specified + * @return The CatalogPlugin and Identifier for the table + */ + public static CatalogAndIdentifier catalogAndIdentifier( + SparkSession spark, List nameParts, CatalogPlugin defaultCatalog) { + CatalogManager catalogManager = spark.sessionState().catalogManager(); + + String[] currentNamespace; + if (defaultCatalog.equals(catalogManager.currentCatalog())) { + currentNamespace = catalogManager.currentNamespace(); + } else { + currentNamespace = defaultCatalog.defaultNamespace(); + } + + Pair catalogIdentifier = + SparkUtil.catalogAndIdentifier( + nameParts, + catalogName -> { + try { + return catalogManager.catalog(catalogName); + } catch (Exception e) { + LOG.warn("Failed to load catalog: {}", catalogName, e); + return null; + } + }, + Identifier::of, + defaultCatalog, + currentNamespace); + return new CatalogAndIdentifier(catalogIdentifier); + } + + private static TableCatalog asTableCatalog(CatalogPlugin catalog) { + if (catalog instanceof TableCatalog) { + return (TableCatalog) catalog; + } + + throw new IllegalArgumentException( + String.format( + "Cannot use catalog %s(%s): not a TableCatalog", + catalog.name(), catalog.getClass().getName())); + } + + /** This mimics a class inside of Spark which is private inside of LookupCatalog. */ + public static class CatalogAndIdentifier { + private final CatalogPlugin catalog; + private final Identifier identifier; + + public CatalogAndIdentifier(CatalogPlugin catalog, Identifier identifier) { + this.catalog = catalog; + this.identifier = identifier; + } + + public CatalogAndIdentifier(Pair identifier) { + this.catalog = identifier.first(); + this.identifier = identifier.second(); + } + + public CatalogPlugin catalog() { + return catalog; + } + + public Identifier identifier() { + return identifier; + } + } + + public static TableIdentifier identifierToTableIdentifier(Identifier identifier) { + return TableIdentifier.of(Namespace.of(identifier.namespace()), identifier.name()); + } + + public static String quotedFullIdentifier(String catalogName, Identifier identifier) { + List parts = + ImmutableList.builder() + .add(catalogName) + .addAll(Arrays.asList(identifier.namespace())) + .add(identifier.name()) + .build(); + + return CatalogV2Implicits.MultipartIdentifierHelper( + JavaConverters.asScalaIteratorConverter(parts.iterator()).asScala().toSeq()) + .quoted(); + } + + public static org.apache.spark.sql.execution.datasources.PartitionSpec getInferredSpec( + SparkSession spark, Path rootPath) { + FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark); + InMemoryFileIndex fileIndex = + new InMemoryFileIndex( + spark, + JavaConverters.collectionAsScalaIterableConverter(ImmutableList.of(rootPath)) + .asScala() + .toSeq(), + scala.collection.immutable.Map$.MODULE$.empty(), + Option.empty(), // Pass empty so that automatic schema inference is used + fileStatusCache, + Option.empty(), + Option.empty()); + return fileIndex.partitionSpec(); + } + + /** + * Use Spark to list all partitions in the table. + * + * @param spark a Spark session + * @param rootPath a table identifier + * @param format format of the file + * @param partitionFilter partitionFilter of the file + * @param partitionSpec partitionSpec of the table + * @return all table's partitions + */ + public static List getPartitions( + SparkSession spark, + Path rootPath, + String format, + Map partitionFilter, + PartitionSpec partitionSpec) { + FileStatusCache fileStatusCache = FileStatusCache.getOrCreate(spark); + + Option userSpecifiedSchema = + partitionSpec == null + ? Option.empty() + : Option.apply( + SparkSchemaUtil.convert(new Schema(partitionSpec.partitionType().fields()))); + + InMemoryFileIndex fileIndex = + new InMemoryFileIndex( + spark, + JavaConverters.collectionAsScalaIterableConverter(ImmutableList.of(rootPath)) + .asScala() + .toSeq(), + scala.collection.immutable.Map$.MODULE$.empty(), + userSpecifiedSchema, + fileStatusCache, + Option.empty(), + Option.empty()); + + org.apache.spark.sql.execution.datasources.PartitionSpec spec = fileIndex.partitionSpec(); + StructType schema = spec.partitionColumns(); + if (schema.isEmpty()) { + return Lists.newArrayList(); + } + + List filterExpressions = + SparkUtil.partitionMapToExpression(schema, partitionFilter); + Seq scalaPartitionFilters = + JavaConverters.asScalaBufferConverter(filterExpressions).asScala().toIndexedSeq(); + + List dataFilters = Lists.newArrayList(); + Seq scalaDataFilters = + JavaConverters.asScalaBufferConverter(dataFilters).asScala().toIndexedSeq(); + + Seq filteredPartitions = + fileIndex.listFiles(scalaPartitionFilters, scalaDataFilters).toIndexedSeq(); + + return JavaConverters.seqAsJavaListConverter(filteredPartitions).asJava().stream() + .map( + partition -> { + Map values = Maps.newHashMap(); + JavaConverters.asJavaIterableConverter(schema) + .asJava() + .forEach( + field -> { + int fieldIndex = schema.fieldIndex(field.name()); + Object catalystValue = partition.values().get(fieldIndex, field.dataType()); + Object value = + CatalystTypeConverters.convertToScala(catalystValue, field.dataType()); + values.put(field.name(), (value == null) ? HIVE_NULL : value.toString()); + }); + + FileStatusWithMetadata fileStatus = + JavaConverters.seqAsJavaListConverter(partition.files()).asJava().get(0); + + return new SparkPartition( + values, fileStatus.getPath().getParent().toString(), format); + }) + .collect(Collectors.toList()); + } + + public static org.apache.spark.sql.catalyst.TableIdentifier toV1TableIdentifier( + Identifier identifier) { + String[] namespace = identifier.namespace(); + + Preconditions.checkArgument( + namespace.length <= 1, + "Cannot convert %s to a Spark v1 identifier, namespace contains more than 1 part", + identifier); + + String table = identifier.name(); + Option database = namespace.length == 1 ? Option.apply(namespace[0]) : Option.empty(); + return org.apache.spark.sql.catalyst.TableIdentifier.apply(table, database); + } + + static String baseTableUUID(org.apache.iceberg.Table table) { + if (table instanceof HasTableOperations) { + TableOperations ops = ((HasTableOperations) table).operations(); + return ops.current().uuid(); + } else if (table instanceof BaseMetadataTable) { + return ((BaseMetadataTable) table).table().operations().current().uuid(); + } else { + throw new UnsupportedOperationException("Cannot retrieve UUID for table " + table.name()); + } + } + + private static class DescribeSortOrderVisitor implements SortOrderVisitor { + private static final DescribeSortOrderVisitor INSTANCE = new DescribeSortOrderVisitor(); + + private DescribeSortOrderVisitor() {} + + @Override + public String field( + String sourceName, + int sourceId, + org.apache.iceberg.SortDirection direction, + NullOrder nullOrder) { + return String.format("%s %s %s", sourceName, direction, nullOrder); + } + + @Override + public String bucket( + String sourceName, + int sourceId, + int numBuckets, + org.apache.iceberg.SortDirection direction, + NullOrder nullOrder) { + return String.format("bucket(%s, %s) %s %s", numBuckets, sourceName, direction, nullOrder); + } + + @Override + public String truncate( + String sourceName, + int sourceId, + int width, + org.apache.iceberg.SortDirection direction, + NullOrder nullOrder) { + return String.format("truncate(%s, %s) %s %s", sourceName, width, direction, nullOrder); + } + + @Override + public String year( + String sourceName, + int sourceId, + org.apache.iceberg.SortDirection direction, + NullOrder nullOrder) { + return String.format("years(%s) %s %s", sourceName, direction, nullOrder); + } + + @Override + public String month( + String sourceName, + int sourceId, + org.apache.iceberg.SortDirection direction, + NullOrder nullOrder) { + return String.format("months(%s) %s %s", sourceName, direction, nullOrder); + } + + @Override + public String day( + String sourceName, + int sourceId, + org.apache.iceberg.SortDirection direction, + NullOrder nullOrder) { + return String.format("days(%s) %s %s", sourceName, direction, nullOrder); + } + + @Override + public String hour( + String sourceName, + int sourceId, + org.apache.iceberg.SortDirection direction, + NullOrder nullOrder) { + return String.format("hours(%s) %s %s", sourceName, direction, nullOrder); + } + + @Override + public String unknown( + String sourceName, + int sourceId, + String transform, + org.apache.iceberg.SortDirection direction, + NullOrder nullOrder) { + return String.format("%s(%s) %s %s", transform, sourceName, direction, nullOrder); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkAggregates.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkAggregates.java new file mode 100644 index 000000000000..153ef11a9eb6 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkAggregates.java @@ -0,0 +1,84 @@ +/* + * 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.Map; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.expressions.aggregate.AggregateFunc; +import org.apache.spark.sql.connector.expressions.aggregate.Count; +import org.apache.spark.sql.connector.expressions.aggregate.CountStar; +import org.apache.spark.sql.connector.expressions.aggregate.Max; +import org.apache.spark.sql.connector.expressions.aggregate.Min; + +public class SparkAggregates { + private SparkAggregates() {} + + private static final Map, Operation> AGGREGATES = + ImmutableMap., Operation>builder() + .put(Count.class, Operation.COUNT) + .put(CountStar.class, Operation.COUNT_STAR) + .put(Max.class, Operation.MAX) + .put(Min.class, Operation.MIN) + .buildOrThrow(); + + public static Expression convert(AggregateFunc aggregate) { + Operation op = AGGREGATES.get(aggregate.getClass()); + if (op != null) { + switch (op) { + case COUNT: + Count countAgg = (Count) aggregate; + if (countAgg.isDistinct()) { + // manifest file doesn't have count distinct so this can't be pushed down + return null; + } + + if (countAgg.column() instanceof NamedReference) { + return Expressions.count(SparkUtil.toColumnName((NamedReference) countAgg.column())); + } else { + return null; + } + + case COUNT_STAR: + return Expressions.countStar(); + + case MAX: + Max maxAgg = (Max) aggregate; + if (maxAgg.column() instanceof NamedReference) { + return Expressions.max(SparkUtil.toColumnName((NamedReference) maxAgg.column())); + } else { + return null; + } + + case MIN: + Min minAgg = (Min) aggregate; + if (minAgg.column() instanceof NamedReference) { + return Expressions.min(SparkUtil.toColumnName((NamedReference) minAgg.column())); + } else { + return null; + } + } + } + + return null; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCachedTableCatalog.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCachedTableCatalog.java new file mode 100644 index 000000000000..28427f597b06 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCachedTableCatalog.java @@ -0,0 +1,257 @@ +/* + * 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.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Stream; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Splitter; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.source.SparkTable; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.TableChange; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** An internal table catalog that is capable of loading tables from a cache. */ +public class SparkCachedTableCatalog implements TableCatalog, SupportsFunctions { + + private static final String CLASS_NAME = SparkCachedTableCatalog.class.getName(); + private static final Splitter COMMA = Splitter.on(","); + private static final Pattern AT_TIMESTAMP = Pattern.compile("at_timestamp_(\\d+)"); + 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 static final SparkTableCache TABLE_CACHE = SparkTableCache.get(); + + private String name = null; + + @Override + public Identifier[] listTables(String[] namespace) { + throw new UnsupportedOperationException(CLASS_NAME + " does not support listing tables"); + } + + @Override + public SparkTable loadTable(Identifier ident) throws NoSuchTableException { + return load(ident); + } + + @Override + public SparkTable loadTable(Identifier ident, String version) throws NoSuchTableException { + SparkTable table = load(ident); + Preconditions.checkArgument( + table.snapshotId() == null, "Cannot time travel based on both table identifier and AS OF"); + return table.copyWithSnapshotId(Long.parseLong(version)); + } + + @Override + public SparkTable loadTable(Identifier ident, long timestampMicros) throws NoSuchTableException { + SparkTable table = load(ident); + Preconditions.checkArgument( + table.snapshotId() == null, "Cannot time travel based on both table identifier and AS OF"); + // Spark passes microseconds but Iceberg uses milliseconds for snapshots + long timestampMillis = TimeUnit.MICROSECONDS.toMillis(timestampMicros); + long snapshotId = SnapshotUtil.snapshotIdAsOfTime(table.table(), timestampMillis); + return table.copyWithSnapshotId(snapshotId); + } + + @Override + public void invalidateTable(Identifier ident) { + throw new UnsupportedOperationException(CLASS_NAME + " does not support table invalidation"); + } + + @Override + public SparkTable createTable( + Identifier ident, StructType schema, Transform[] partitions, Map properties) + throws TableAlreadyExistsException { + throw new UnsupportedOperationException(CLASS_NAME + " does not support creating tables"); + } + + @Override + public SparkTable alterTable(Identifier ident, TableChange... changes) { + throw new UnsupportedOperationException(CLASS_NAME + " does not support altering tables"); + } + + @Override + public boolean dropTable(Identifier ident) { + throw new UnsupportedOperationException(CLASS_NAME + " does not support dropping tables"); + } + + @Override + public boolean purgeTable(Identifier ident) throws UnsupportedOperationException { + throw new UnsupportedOperationException(CLASS_NAME + " does not support purging tables"); + } + + @Override + public void renameTable(Identifier oldIdent, Identifier newIdent) { + throw new UnsupportedOperationException(CLASS_NAME + " does not support renaming tables"); + } + + @Override + public void initialize(String catalogName, CaseInsensitiveStringMap options) { + this.name = catalogName; + } + + @Override + public String name() { + return name; + } + + private SparkTable load(Identifier ident) throws NoSuchTableException { + Preconditions.checkArgument( + ident.namespace().length == 0, CLASS_NAME + " does not support namespaces"); + + Pair> parsedIdent = parseIdent(ident); + String key = parsedIdent.first(); + TableLoadOptions options = parseLoadOptions(parsedIdent.second()); + + Table table = TABLE_CACHE.get(key); + + if (table == null) { + throw new NoSuchTableException(ident); + } + + if (options.isTableRewrite()) { + return new SparkTable(table, null, false, true); + } + + if (options.snapshotId() != null) { + return new SparkTable(table, options.snapshotId(), false); + } else if (options.asOfTimestamp() != null) { + return new SparkTable( + table, SnapshotUtil.snapshotIdAsOfTime(table, options.asOfTimestamp()), false); + } else if (options.branch() != null) { + Snapshot branchSnapshot = table.snapshot(options.branch()); + Preconditions.checkArgument( + branchSnapshot != null, + "Cannot find snapshot associated with branch name: %s", + options.branch()); + return new SparkTable(table, branchSnapshot.snapshotId(), false); + } else if (options.tag() != null) { + Snapshot tagSnapshot = table.snapshot(options.tag()); + Preconditions.checkArgument( + tagSnapshot != null, "Cannot find snapshot associated with tag name: %s", options.tag()); + return new SparkTable(table, tagSnapshot.snapshotId(), false); + } else { + return new SparkTable(table, false); + } + } + + private static class TableLoadOptions { + private Long asOfTimestamp; + private Long snapshotId; + private String branch; + private String tag; + private Boolean isTableRewrite; + + Long asOfTimestamp() { + return asOfTimestamp; + } + + Long snapshotId() { + return snapshotId; + } + + String branch() { + return branch; + } + + String tag() { + return tag; + } + + boolean isTableRewrite() { + return Boolean.TRUE.equals(isTableRewrite); + } + } + + /** Extracts table load options from metadata. */ + private TableLoadOptions parseLoadOptions(List metadata) { + TableLoadOptions opts = new TableLoadOptions(); + for (String meta : metadata) { + Matcher timeBasedMatcher = AT_TIMESTAMP.matcher(meta); + if (timeBasedMatcher.matches()) { + opts.asOfTimestamp = Long.parseLong(timeBasedMatcher.group(1)); + continue; + } + + Matcher snapshotBasedMatcher = SNAPSHOT_ID.matcher(meta); + if (snapshotBasedMatcher.matches()) { + opts.snapshotId = Long.parseLong(snapshotBasedMatcher.group(1)); + continue; + } + + Matcher branchBasedMatcher = BRANCH.matcher(meta); + if (branchBasedMatcher.matches()) { + opts.branch = branchBasedMatcher.group(1); + continue; + } + + Matcher tagBasedMatcher = TAG.matcher(meta); + if (tagBasedMatcher.matches()) { + opts.tag = tagBasedMatcher.group(1); + } + + if (meta.equalsIgnoreCase(REWRITE)) { + opts.isTableRewrite = true; + } + } + + long numberOptions = + Stream.of(opts.snapshotId, opts.asOfTimestamp, opts.branch, opts.tag, opts.isTableRewrite) + .filter(Objects::nonNull) + .count(); + Preconditions.checkArgument( + numberOptions <= 1, + "Can specify only one of snapshot-id (%s), as-of-timestamp (%s), branch (%s), tag (%s), is-table-rewrite (%s)", + opts.snapshotId, + opts.asOfTimestamp, + opts.branch, + opts.tag, + opts.isTableRewrite); + + return opts; + } + + private Pair> parseIdent(Identifier ident) { + int hashIndex = ident.name().lastIndexOf('#'); + if (hashIndex != -1 && !ident.name().endsWith("#")) { + String key = ident.name().substring(0, hashIndex); + List metadata = COMMA.splitToList(ident.name().substring(hashIndex + 1)); + return Pair.of(key, metadata); + } else { + return Pair.of(ident.name(), ImmutableList.of()); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java new file mode 100644 index 000000000000..dba3ea6ecc1c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -0,0 +1,1049 @@ +/* + * 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 static org.apache.iceberg.TableProperties.GC_ENABLED; +import static org.apache.iceberg.TableProperties.GC_ENABLED_DEFAULT; + +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; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.EnvironmentContext; +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; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.catalog.ViewCatalog; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Splitter; +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.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.actions.SparkActions; +import org.apache.iceberg.spark.source.SparkChangelogTable; +import org.apache.iceberg.spark.source.SparkTable; +import org.apache.iceberg.spark.source.SparkView; +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; +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.analysis.NoSuchViewException; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.catalyst.analysis.ViewAlreadyExistsException; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.NamespaceChange; +import org.apache.spark.sql.connector.catalog.StagedTable; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.TableChange; +import org.apache.spark.sql.connector.catalog.TableChange.ColumnChange; +import org.apache.spark.sql.connector.catalog.TableChange.RemoveProperty; +import org.apache.spark.sql.connector.catalog.TableChange.SetProperty; +import org.apache.spark.sql.connector.catalog.View; +import org.apache.spark.sql.connector.catalog.ViewChange; +import org.apache.spark.sql.connector.catalog.ViewInfo; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * A Spark TableCatalog implementation that wraps an Iceberg {@link Catalog}. + * + *

    This supports the following catalog configuration options: + * + *

      + *
    • type - catalog type, "hive" or "hadoop" or "rest". To specify a non-hive or + * hadoop catalog, use the catalog-impl option. + *
    • uri - the Hive Metastore URI for Hive catalog or REST URI for REST catalog + *
    • warehouse - the warehouse path (Hadoop catalog only) + *
    • catalog-impl - a custom {@link Catalog} implementation to use + *
    • io-impl - a custom {@link org.apache.iceberg.io.FileIO} implementation to use + *
    • metrics-reporter-impl - a custom {@link + * org.apache.iceberg.metrics.MetricsReporter} implementation to use + *
    • default-namespace - a namespace to use as the default + *
    • cache-enabled - whether to enable catalog cache + *
    • cache.case-sensitive - whether the catalog cache should compare table + * identifiers in a case sensitive way + *
    • cache.expiration-interval-ms - interval in millis before expiring tables from + * catalog cache. Refer to {@link CatalogProperties#CACHE_EXPIRATION_INTERVAL_MS} for further + * details and significant values. + *
    • table-default.$tablePropertyKey - table property $tablePropertyKey default at + * catalog level + *
    • table-override.$tablePropertyKey - table property $tablePropertyKey enforced + * at catalog level + *
    + * + *

    + */ +public class SparkCatalog extends BaseCatalog { + private static final Set DEFAULT_NS_KEYS = ImmutableSet.of(TableCatalog.PROP_OWNER); + private static final Splitter COMMA = Splitter.on(","); + private static final Joiner COMMA_JOINER = Joiner.on(","); + private static final Pattern AT_TIMESTAMP = Pattern.compile("at_timestamp_(\\d+)"); + 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; + private HadoopTables tables; + + /** + * Build an Iceberg {@link Catalog} to be used by this Spark catalog adapter. + * + * @param name Spark's catalog name + * @param options Spark's catalog options + * @return an Iceberg catalog + */ + protected Catalog buildIcebergCatalog(String name, CaseInsensitiveStringMap options) { + Configuration conf = SparkUtil.hadoopConfCatalogOverrides(SparkSession.active(), name); + Map optionsMap = new TreeMap<>(String.CASE_INSENSITIVE_ORDER); + optionsMap.putAll(options.asCaseSensitiveMap()); + optionsMap.put(CatalogProperties.APP_ID, SparkSession.active().sparkContext().applicationId()); + optionsMap.put(CatalogProperties.USER, SparkSession.active().sparkContext().sparkUser()); + return CatalogUtil.buildIcebergCatalog(name, optionsMap, conf); + } + + /** + * Build an Iceberg {@link TableIdentifier} for the given Spark identifier. + * + * @param identifier Spark's identifier + * @return an Iceberg identifier + */ + protected TableIdentifier buildIdentifier(Identifier identifier) { + return Spark3Util.identifierToTableIdentifier(identifier); + } + + @Override + public Table loadTable(Identifier ident) throws NoSuchTableException { + try { + return load(ident); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + throw new NoSuchTableException(ident); + } + } + + @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()); + } + } + + @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()); + } + } + + @Override + public boolean tableExists(Identifier ident) { + if (isPathIdentifier(ident)) { + return tables.exists(((PathIdentifier) ident).location()); + } else { + return icebergCatalog.tableExists(buildIdentifier(ident)); + } + } + + @Override + public Table createTable( + Identifier ident, StructType schema, Transform[] transforms, Map properties) + throws TableAlreadyExistsException { + Schema icebergSchema = SparkSchemaUtil.convert(schema); + try { + Catalog.TableBuilder builder = newBuilder(ident, icebergSchema); + org.apache.iceberg.Table icebergTable = + builder + .withPartitionSpec(Spark3Util.toPartitionSpec(icebergSchema, transforms)) + .withLocation(properties.get("location")) + .withProperties(Spark3Util.rebuildCreateProperties(properties)) + .create(); + return new SparkTable(icebergTable, !cacheEnabled); + } catch (AlreadyExistsException e) { + throw new TableAlreadyExistsException(ident); + } + } + + @Override + public StagedTable stageCreate( + Identifier ident, StructType schema, Transform[] transforms, Map properties) + throws TableAlreadyExistsException { + Schema icebergSchema = SparkSchemaUtil.convert(schema); + try { + Catalog.TableBuilder builder = newBuilder(ident, icebergSchema); + Transaction transaction = + builder + .withPartitionSpec(Spark3Util.toPartitionSpec(icebergSchema, transforms)) + .withLocation(properties.get("location")) + .withProperties(Spark3Util.rebuildCreateProperties(properties)) + .createTransaction(); + return new StagedSparkTable(transaction); + } catch (AlreadyExistsException e) { + throw new TableAlreadyExistsException(ident); + } + } + + @Override + public StagedTable stageReplace( + Identifier ident, StructType schema, Transform[] transforms, Map properties) + throws NoSuchTableException { + Schema icebergSchema = SparkSchemaUtil.convert(schema); + try { + Catalog.TableBuilder builder = newBuilder(ident, icebergSchema); + Transaction transaction = + builder + .withPartitionSpec(Spark3Util.toPartitionSpec(icebergSchema, transforms)) + .withLocation(properties.get("location")) + .withProperties(Spark3Util.rebuildCreateProperties(properties)) + .replaceTransaction(); + return new StagedSparkTable(transaction); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + throw new NoSuchTableException(ident); + } + } + + @Override + public StagedTable stageCreateOrReplace( + Identifier ident, StructType schema, Transform[] transforms, Map properties) { + Schema icebergSchema = SparkSchemaUtil.convert(schema); + Catalog.TableBuilder builder = newBuilder(ident, icebergSchema); + Transaction transaction = + builder + .withPartitionSpec(Spark3Util.toPartitionSpec(icebergSchema, transforms)) + .withLocation(properties.get("location")) + .withProperties(Spark3Util.rebuildCreateProperties(properties)) + .createOrReplaceTransaction(); + return new StagedSparkTable(transaction); + } + + @Override + public Table alterTable(Identifier ident, TableChange... changes) throws NoSuchTableException { + SetProperty setLocation = null; + SetProperty setSnapshotId = null; + SetProperty pickSnapshotId = null; + List propertyChanges = Lists.newArrayList(); + List schemaChanges = Lists.newArrayList(); + + for (TableChange change : changes) { + if (change instanceof SetProperty) { + SetProperty set = (SetProperty) change; + if (TableCatalog.PROP_LOCATION.equalsIgnoreCase(set.property())) { + setLocation = set; + } else if ("current-snapshot-id".equalsIgnoreCase(set.property())) { + setSnapshotId = set; + } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(set.property())) { + pickSnapshotId = set; + } else if ("sort-order".equalsIgnoreCase(set.property())) { + throw new UnsupportedOperationException( + "Cannot specify the 'sort-order' because it's a reserved table " + + "property. Please use the command 'ALTER TABLE ... WRITE ORDERED BY' to specify write sort-orders."); + } else if ("identifier-fields".equalsIgnoreCase(set.property())) { + throw new UnsupportedOperationException( + "Cannot specify the 'identifier-fields' because it's a reserved table property. " + + "Please use the command 'ALTER TABLE ... SET IDENTIFIER FIELDS' to specify identifier fields."); + } else { + propertyChanges.add(set); + } + } else if (change instanceof RemoveProperty) { + propertyChanges.add(change); + } else if (change instanceof ColumnChange) { + schemaChanges.add(change); + } else { + throw new UnsupportedOperationException("Cannot apply unknown table change: " + change); + } + } + + try { + org.apache.iceberg.Table table = icebergCatalog.loadTable(buildIdentifier(ident)); + commitChanges( + table, setLocation, setSnapshotId, pickSnapshotId, propertyChanges, schemaChanges); + return new SparkTable(table, true /* refreshEagerly */); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + throw new NoSuchTableException(ident); + } + } + + @Override + public boolean dropTable(Identifier ident) { + return dropTableWithoutPurging(ident); + } + + @Override + public boolean purgeTable(Identifier ident) { + try { + org.apache.iceberg.Table table = icebergCatalog.loadTable(buildIdentifier(ident)); + ValidationException.check( + PropertyUtil.propertyAsBoolean(table.properties(), GC_ENABLED, GC_ENABLED_DEFAULT), + "Cannot purge table: GC is disabled (deleting files may corrupt other tables)"); + String metadataFileLocation = + ((HasTableOperations) table).operations().current().metadataFileLocation(); + + boolean dropped = dropTableWithoutPurging(ident); + + if (dropped) { + // check whether the metadata file exists because HadoopCatalog/HadoopTables + // will drop the warehouse directly and ignore the `purge` argument + boolean metadataFileExists = table.io().newInputFile(metadataFileLocation).exists(); + + if (metadataFileExists) { + SparkActions.get().deleteReachableFiles(metadataFileLocation).io(table.io()).execute(); + } + } + + return dropped; + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + return false; + } + } + + private boolean dropTableWithoutPurging(Identifier ident) { + if (isPathIdentifier(ident)) { + return tables.dropTable(((PathIdentifier) ident).location(), false /* don't purge data */); + } else { + return icebergCatalog.dropTable(buildIdentifier(ident), false /* don't purge data */); + } + } + + @Override + public void renameTable(Identifier from, Identifier to) + throws NoSuchTableException, TableAlreadyExistsException { + try { + checkNotPathIdentifier(from, "renameTable"); + checkNotPathIdentifier(to, "renameTable"); + icebergCatalog.renameTable(buildIdentifier(from), buildIdentifier(to)); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + throw new NoSuchTableException(from); + } catch (AlreadyExistsException e) { + throw new TableAlreadyExistsException(to); + } + } + + @Override + public void invalidateTable(Identifier ident) { + if (!isPathIdentifier(ident)) { + icebergCatalog.invalidateTable(buildIdentifier(ident)); + } + } + + @Override + public Identifier[] listTables(String[] namespace) { + return icebergCatalog.listTables(Namespace.of(namespace)).stream() + .map(ident -> Identifier.of(ident.namespace().levels(), ident.name())) + .toArray(Identifier[]::new); + } + + @Override + public String[] defaultNamespace() { + if (defaultNamespace != null) { + return defaultNamespace; + } + + return new String[0]; + } + + @Override + public String[][] listNamespaces() { + if (asNamespaceCatalog != null) { + return asNamespaceCatalog.listNamespaces().stream() + .map(Namespace::levels) + .toArray(String[][]::new); + } + + return new String[0][]; + } + + @Override + public String[][] listNamespaces(String[] namespace) throws NoSuchNamespaceException { + if (asNamespaceCatalog != null) { + try { + return asNamespaceCatalog.listNamespaces(Namespace.of(namespace)).stream() + .map(Namespace::levels) + .toArray(String[][]::new); + } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) { + throw new NoSuchNamespaceException(namespace); + } + } + + throw new NoSuchNamespaceException(namespace); + } + + @Override + public boolean namespaceExists(String[] namespace) { + return asNamespaceCatalog != null + && asNamespaceCatalog.namespaceExists(Namespace.of(namespace)); + } + + @Override + public Map loadNamespaceMetadata(String[] namespace) + throws NoSuchNamespaceException { + if (asNamespaceCatalog != null) { + try { + return asNamespaceCatalog.loadNamespaceMetadata(Namespace.of(namespace)); + } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) { + throw new NoSuchNamespaceException(namespace); + } + } + + throw new NoSuchNamespaceException(namespace); + } + + @Override + public void createNamespace(String[] namespace, Map metadata) + throws NamespaceAlreadyExistsException { + if (asNamespaceCatalog != null) { + try { + if (asNamespaceCatalog instanceof HadoopCatalog + && DEFAULT_NS_KEYS.equals(metadata.keySet())) { + // Hadoop catalog will reject metadata properties, but Spark automatically adds "owner". + // If only the automatic properties are present, replace metadata with an empty map. + asNamespaceCatalog.createNamespace(Namespace.of(namespace), ImmutableMap.of()); + } else { + asNamespaceCatalog.createNamespace(Namespace.of(namespace), metadata); + } + } catch (AlreadyExistsException e) { + throw new NamespaceAlreadyExistsException(namespace); + } + } else { + throw new UnsupportedOperationException( + "Namespaces are not supported by catalog: " + catalogName); + } + } + + @Override + public void alterNamespace(String[] namespace, NamespaceChange... changes) + throws NoSuchNamespaceException { + if (asNamespaceCatalog != null) { + Map updates = Maps.newHashMap(); + Set removals = Sets.newHashSet(); + for (NamespaceChange change : changes) { + if (change instanceof NamespaceChange.SetProperty) { + NamespaceChange.SetProperty set = (NamespaceChange.SetProperty) change; + updates.put(set.property(), set.value()); + } else if (change instanceof NamespaceChange.RemoveProperty) { + removals.add(((NamespaceChange.RemoveProperty) change).property()); + } else { + throw new UnsupportedOperationException( + "Cannot apply unknown namespace change: " + change); + } + } + + try { + if (!updates.isEmpty()) { + asNamespaceCatalog.setProperties(Namespace.of(namespace), updates); + } + + if (!removals.isEmpty()) { + asNamespaceCatalog.removeProperties(Namespace.of(namespace), removals); + } + + } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) { + throw new NoSuchNamespaceException(namespace); + } + } else { + throw new NoSuchNamespaceException(namespace); + } + } + + @Override + public boolean dropNamespace(String[] namespace, boolean cascade) + throws NoSuchNamespaceException { + if (asNamespaceCatalog != null) { + try { + return asNamespaceCatalog.dropNamespace(Namespace.of(namespace)); + } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) { + throw new NoSuchNamespaceException(namespace); + } + } + + return false; + } + + @Override + public Identifier[] listViews(String... namespace) { + if (null != asViewCatalog) { + return asViewCatalog.listViews(Namespace.of(namespace)).stream() + .map(ident -> Identifier.of(ident.namespace().levels(), ident.name())) + .toArray(Identifier[]::new); + } + + return new Identifier[0]; + } + + @Override + public boolean viewExists(Identifier ident) { + return asViewCatalog != null && asViewCatalog.viewExists(buildIdentifier(ident)); + } + + @Override + public View loadView(Identifier ident) throws NoSuchViewException { + if (null != asViewCatalog) { + try { + org.apache.iceberg.view.View view = asViewCatalog.loadView(buildIdentifier(ident)); + return new SparkView(catalogName, view); + } catch (org.apache.iceberg.exceptions.NoSuchViewException e) { + throw new NoSuchViewException(ident); + } + } + + throw new NoSuchViewException(ident); + } + + @Override + public View createView(ViewInfo viewInfo) + throws ViewAlreadyExistsException, NoSuchNamespaceException { + if (null != asViewCatalog && viewInfo != null) { + Identifier ident = viewInfo.ident(); + String sql = viewInfo.sql(); + String currentCatalog = viewInfo.currentCatalog(); + String[] currentNamespace = viewInfo.currentNamespace(); + StructType schema = viewInfo.schema(); + String[] queryColumnNames = viewInfo.queryColumnNames(); + Map properties = viewInfo.properties(); + Schema icebergSchema = SparkSchemaUtil.convert(schema); + + try { + Map props = + ImmutableMap.builder() + .putAll(Spark3Util.rebuildCreateProperties(properties)) + .put(SparkView.QUERY_COLUMN_NAMES, COMMA_JOINER.join(queryColumnNames)) + .buildKeepingLast(); + + org.apache.iceberg.view.View view = + asViewCatalog + .buildView(buildIdentifier(ident)) + .withDefaultCatalog(currentCatalog) + .withDefaultNamespace(Namespace.of(currentNamespace)) + .withQuery("spark", sql) + .withSchema(icebergSchema) + .withLocation(properties.get("location")) + .withProperties(props) + .create(); + return new SparkView(catalogName, view); + } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) { + throw new NoSuchNamespaceException(currentNamespace); + } catch (AlreadyExistsException e) { + throw new ViewAlreadyExistsException(ident); + } + } + + throw new UnsupportedOperationException( + "Creating a view is not supported by catalog: " + catalogName); + } + + @Override + public View replaceView( + Identifier ident, + String sql, + String currentCatalog, + String[] currentNamespace, + StructType schema, + String[] queryColumnNames, + String[] columnAliases, + String[] columnComments, + Map properties) + throws NoSuchNamespaceException, NoSuchViewException { + if (null != asViewCatalog) { + Schema icebergSchema = SparkSchemaUtil.convert(schema); + + try { + Map props = + ImmutableMap.builder() + .putAll(Spark3Util.rebuildCreateProperties(properties)) + .put(SparkView.QUERY_COLUMN_NAMES, COMMA_JOINER.join(queryColumnNames)) + .buildKeepingLast(); + + org.apache.iceberg.view.View view = + asViewCatalog + .buildView(buildIdentifier(ident)) + .withDefaultCatalog(currentCatalog) + .withDefaultNamespace(Namespace.of(currentNamespace)) + .withQuery("spark", sql) + .withSchema(icebergSchema) + .withLocation(properties.get("location")) + .withProperties(props) + .createOrReplace(); + return new SparkView(catalogName, view); + } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) { + throw new NoSuchNamespaceException(currentNamespace); + } catch (org.apache.iceberg.exceptions.NoSuchViewException e) { + throw new NoSuchViewException(ident); + } + } + + throw new UnsupportedOperationException( + "Replacing a view is not supported by catalog: " + catalogName); + } + + @Override + public View alterView(Identifier ident, ViewChange... changes) + throws NoSuchViewException, IllegalArgumentException { + if (null != asViewCatalog) { + try { + org.apache.iceberg.view.View view = asViewCatalog.loadView(buildIdentifier(ident)); + UpdateViewProperties updateViewProperties = view.updateProperties(); + + for (ViewChange change : changes) { + if (change instanceof ViewChange.SetProperty) { + ViewChange.SetProperty property = (ViewChange.SetProperty) change; + verifyNonReservedPropertyIsSet(property.property()); + updateViewProperties.set(property.property(), property.value()); + } else if (change instanceof ViewChange.RemoveProperty) { + ViewChange.RemoveProperty remove = (ViewChange.RemoveProperty) change; + verifyNonReservedPropertyIsUnset(remove.property()); + updateViewProperties.remove(remove.property()); + } + } + + updateViewProperties.commit(); + + return new SparkView(catalogName, view); + } catch (org.apache.iceberg.exceptions.NoSuchViewException e) { + throw new NoSuchViewException(ident); + } + } + + throw new UnsupportedOperationException( + "Altering a view is not supported by catalog: " + catalogName); + } + + private static void verifyNonReservedProperty(String property, String errorMsg) { + if (SparkView.RESERVED_PROPERTIES.contains(property)) { + throw new UnsupportedOperationException(String.format(errorMsg, property)); + } + } + + private static void verifyNonReservedPropertyIsUnset(String property) { + verifyNonReservedProperty(property, "Cannot unset reserved property: '%s'"); + } + + private static void verifyNonReservedPropertyIsSet(String property) { + verifyNonReservedProperty(property, "Cannot set reserved property: '%s'"); + } + + @Override + public boolean dropView(Identifier ident) { + if (null != asViewCatalog) { + return asViewCatalog.dropView(buildIdentifier(ident)); + } + + return false; + } + + @Override + public void renameView(Identifier fromIdentifier, Identifier toIdentifier) + throws NoSuchViewException, ViewAlreadyExistsException { + if (null != asViewCatalog) { + try { + asViewCatalog.renameView(buildIdentifier(fromIdentifier), buildIdentifier(toIdentifier)); + } catch (org.apache.iceberg.exceptions.NoSuchViewException e) { + throw new NoSuchViewException(fromIdentifier); + } catch (org.apache.iceberg.exceptions.AlreadyExistsException e) { + throw new ViewAlreadyExistsException(toIdentifier); + } + } else { + throw new UnsupportedOperationException( + "Renaming a view is not supported by catalog: " + catalogName); + } + } + + @Override + public final void initialize(String name, CaseInsensitiveStringMap options) { + super.initialize(name, options); + + this.cacheEnabled = + PropertyUtil.propertyAsBoolean( + options, CatalogProperties.CACHE_ENABLED, CatalogProperties.CACHE_ENABLED_DEFAULT); + + boolean cacheCaseSensitive = + PropertyUtil.propertyAsBoolean( + options, + CatalogProperties.CACHE_CASE_SENSITIVE, + CatalogProperties.CACHE_CASE_SENSITIVE_DEFAULT); + + long cacheExpirationIntervalMs = + PropertyUtil.propertyAsLong( + options, + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS, + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_DEFAULT); + + // An expiration interval of 0ms effectively disables caching. + // Do not wrap with CachingCatalog. + if (cacheExpirationIntervalMs == 0) { + this.cacheEnabled = false; + } + + Catalog catalog = buildIcebergCatalog(name, options); + + this.catalogName = name; + SparkSession sparkSession = SparkSession.getActiveSession().get(); + this.tables = + new HadoopTables( + SparkUtil.hadoopConfCatalogOverrides(SparkSession.getActiveSession().get(), name)); + this.icebergCatalog = + cacheEnabled + ? CachingCatalog.wrap(catalog, cacheCaseSensitive, cacheExpirationIntervalMs) + : catalog; + if (catalog instanceof SupportsNamespaces) { + this.asNamespaceCatalog = (SupportsNamespaces) catalog; + if (options.containsKey("default-namespace")) { + this.defaultNamespace = + Splitter.on('.').splitToList(options.get("default-namespace")).toArray(new String[0]); + } + } + + if (catalog instanceof ViewCatalog) { + this.asViewCatalog = (ViewCatalog) catalog; + } + + EnvironmentContext.put(EnvironmentContext.ENGINE_NAME, "spark"); + EnvironmentContext.put( + EnvironmentContext.ENGINE_VERSION, sparkSession.sparkContext().version()); + EnvironmentContext.put(CatalogProperties.APP_ID, sparkSession.sparkContext().applicationId()); + } + + @Override + public String name() { + return catalogName; + } + + private static void commitChanges( + org.apache.iceberg.Table table, + SetProperty setLocation, + SetProperty setSnapshotId, + SetProperty pickSnapshotId, + List propertyChanges, + List schemaChanges) { + // don't allow setting the snapshot and picking a commit at the same time because order is + // ambiguous and choosing one order leads to different results + Preconditions.checkArgument( + setSnapshotId == null || pickSnapshotId == null, + "Cannot set the current the current snapshot ID and cherry-pick snapshot changes"); + + if (setSnapshotId != null) { + long newSnapshotId = Long.parseLong(setSnapshotId.value()); + table.manageSnapshots().setCurrentSnapshot(newSnapshotId).commit(); + } + + // if updating the table snapshot, perform that update first in case it fails + if (pickSnapshotId != null) { + long newSnapshotId = Long.parseLong(pickSnapshotId.value()); + table.manageSnapshots().cherrypick(newSnapshotId).commit(); + } + + Transaction transaction = table.newTransaction(); + + if (setLocation != null) { + transaction.updateLocation().setLocation(setLocation.value()).commit(); + } + + if (!propertyChanges.isEmpty()) { + Spark3Util.applyPropertyChanges(transaction.updateProperties(), propertyChanges).commit(); + } + + if (!schemaChanges.isEmpty()) { + Spark3Util.applySchemaChanges(transaction.updateSchema(), schemaChanges).commit(); + } + + transaction.commitTransaction(); + } + + private static boolean isPathIdentifier(Identifier ident) { + return ident instanceof PathIdentifier; + } + + private static void checkNotPathIdentifier(Identifier identifier, String method) { + if (identifier instanceof PathIdentifier) { + throw new IllegalArgumentException( + String.format( + "Cannot pass path based identifier to %s method. %s is a path.", method, identifier)); + } + } + + private Table load(Identifier ident) { + if (isPathIdentifier(ident)) { + return loadFromPathIdentifier((PathIdentifier) ident); + } + + try { + org.apache.iceberg.Table table = icebergCatalog.loadTable(buildIdentifier(ident)); + return new SparkTable(table, !cacheEnabled); + + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + if (ident.namespace().length == 0) { + throw e; + } + + // if the original load didn't work, try using the namespace as an identifier because + // the original identifier may include a snapshot selector or may point to the changelog + TableIdentifier namespaceAsIdent = buildIdentifier(namespaceToIdentifier(ident.namespace())); + org.apache.iceberg.Table table; + try { + 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; + } + + // 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); + } + + 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); + } + } + + if (ident.name().equalsIgnoreCase(REWRITE)) { + return new SparkTable(table, null, !cacheEnabled, true); + } + + // the name wasn't a valid snapshot selector and did not point to the changelog + // throw the original exception + throw e; + } + } + + private Pair> parseLocationString(String location) { + int hashIndex = location.lastIndexOf('#'); + if (hashIndex != -1 && !location.endsWith("#")) { + String baseLocation = location.substring(0, hashIndex); + List metadata = COMMA.splitToList(location.substring(hashIndex + 1)); + return Pair.of(baseLocation, metadata); + } else { + return Pair.of(location, ImmutableList.of()); + } + } + + @SuppressWarnings("CyclomaticComplexity") + private Table loadFromPathIdentifier(PathIdentifier ident) { + Pair> parsed = parseLocationString(ident.location()); + + String metadataTableName = null; + Long asOfTimestamp = null; + Long snapshotId = null; + String branch = null; + String tag = null; + boolean isChangelog = false; + boolean isRewrite = false; + + for (String meta : parsed.second()) { + if (meta.equalsIgnoreCase(SparkChangelogTable.TABLE_NAME)) { + isChangelog = true; + 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); + continue; + } + + Matcher tagRef = TAG.matcher(meta); + if (tagRef.matches()) { + tag = tagRef.group(1); + } + } + + 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 : "")); + + if (isChangelog) { + return new SparkChangelogTable(table, !cacheEnabled); + + } else if (asOfTimestamp != null) { + long snapshotIdAsOfTime = SnapshotUtil.snapshotIdAsOfTime(table, asOfTimestamp); + return new SparkTable(table, snapshotIdAsOfTime, !cacheEnabled); + + } else if (branch != null) { + return new SparkTable(table, branch, !cacheEnabled); + + } 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); + + } else if (isRewrite) { + return new SparkTable(table, null, !cacheEnabled, true); + + } else { + return new SparkTable(table, snapshotId, !cacheEnabled); + } + } + + private Identifier namespaceToIdentifier(String[] namespace) { + Preconditions.checkArgument( + namespace.length > 0, "Cannot convert empty namespace to identifier"); + String[] ns = Arrays.copyOf(namespace, namespace.length - 1); + String name = namespace[ns.length]; + return Identifier.of(ns, name); + } + + private Catalog.TableBuilder newBuilder(Identifier ident, Schema schema) { + return isPathIdentifier(ident) + ? tables.buildTable(((PathIdentifier) ident).location(), schema) + : icebergCatalog.buildTable(buildIdentifier(ident), schema); + } + + @Override + public Catalog icebergCatalog() { + return icebergCatalog; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCompressionUtil.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCompressionUtil.java new file mode 100644 index 000000000000..8f00b7f8301d --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCompressionUtil.java @@ -0,0 +1,148 @@ +/* + * 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.Locale; +import java.util.Map; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.SparkSession; + +class SparkCompressionUtil { + + private static final String LZ4 = "lz4"; + private static final String ZSTD = "zstd"; + private static final String GZIP = "gzip"; + private static final String ZLIB = "zlib"; + private static final String SNAPPY = "snappy"; + private static final String NONE = "none"; + + // an internal Spark config that controls whether shuffle data is compressed + private static final String SHUFFLE_COMPRESSION_ENABLED = "spark.shuffle.compress"; + private static final boolean SHUFFLE_COMPRESSION_ENABLED_DEFAULT = true; + + // an internal Spark config that controls what compression codec is used + private static final String SPARK_COMPRESSION_CODEC = "spark.io.compression.codec"; + private static final String SPARK_COMPRESSION_CODEC_DEFAULT = "lz4"; + + private static final double DEFAULT_COLUMNAR_COMPRESSION = 2; + private static final Map, Double> COLUMNAR_COMPRESSIONS = + initColumnarCompressions(); + + private static final double DEFAULT_ROW_BASED_COMPRESSION = 1; + private static final Map, Double> ROW_BASED_COMPRESSIONS = + initRowBasedCompressions(); + + private SparkCompressionUtil() {} + + /** + * Estimates how much the data in shuffle map files will compress once it is written to disk using + * a particular file format and codec. + */ + public static double shuffleCompressionRatio( + SparkSession spark, FileFormat outputFileFormat, String outputCodec) { + if (outputFileFormat == FileFormat.ORC || outputFileFormat == FileFormat.PARQUET) { + return columnarCompression(shuffleCodec(spark), outputCodec); + } else if (outputFileFormat == FileFormat.AVRO) { + return rowBasedCompression(shuffleCodec(spark), outputCodec); + } else { + return 1.0; + } + } + + private static String shuffleCodec(SparkSession spark) { + SparkConf sparkConf = spark.sparkContext().conf(); + return shuffleCompressionEnabled(sparkConf) ? sparkCodec(sparkConf) : NONE; + } + + private static boolean shuffleCompressionEnabled(SparkConf sparkConf) { + return sparkConf.getBoolean(SHUFFLE_COMPRESSION_ENABLED, SHUFFLE_COMPRESSION_ENABLED_DEFAULT); + } + + private static String sparkCodec(SparkConf sparkConf) { + return sparkConf.get(SPARK_COMPRESSION_CODEC, SPARK_COMPRESSION_CODEC_DEFAULT); + } + + private static double columnarCompression(String shuffleCodec, String outputCodec) { + Pair key = Pair.of(normalize(shuffleCodec), normalize(outputCodec)); + return COLUMNAR_COMPRESSIONS.getOrDefault(key, DEFAULT_COLUMNAR_COMPRESSION); + } + + private static double rowBasedCompression(String shuffleCodec, String outputCodec) { + Pair key = Pair.of(normalize(shuffleCodec), normalize(outputCodec)); + return ROW_BASED_COMPRESSIONS.getOrDefault(key, DEFAULT_ROW_BASED_COMPRESSION); + } + + private static String normalize(String value) { + return value != null ? value.toLowerCase(Locale.ROOT) : null; + } + + private static Map, Double> initColumnarCompressions() { + Map, Double> compressions = Maps.newHashMap(); + + compressions.put(Pair.of(NONE, ZSTD), 4.0); + compressions.put(Pair.of(NONE, GZIP), 4.0); + compressions.put(Pair.of(NONE, ZLIB), 4.0); + compressions.put(Pair.of(NONE, SNAPPY), 3.0); + compressions.put(Pair.of(NONE, LZ4), 3.0); + + compressions.put(Pair.of(ZSTD, ZSTD), 2.0); + compressions.put(Pair.of(ZSTD, GZIP), 2.0); + compressions.put(Pair.of(ZSTD, ZLIB), 2.0); + compressions.put(Pair.of(ZSTD, SNAPPY), 1.5); + compressions.put(Pair.of(ZSTD, LZ4), 1.5); + + compressions.put(Pair.of(SNAPPY, ZSTD), 3.0); + compressions.put(Pair.of(SNAPPY, GZIP), 3.0); + compressions.put(Pair.of(SNAPPY, ZLIB), 3.0); + compressions.put(Pair.of(SNAPPY, SNAPPY), 2.0); + compressions.put(Pair.of(SNAPPY, LZ4), 2.); + + compressions.put(Pair.of(LZ4, ZSTD), 3.0); + compressions.put(Pair.of(LZ4, GZIP), 3.0); + compressions.put(Pair.of(LZ4, ZLIB), 3.0); + compressions.put(Pair.of(LZ4, SNAPPY), 2.0); + compressions.put(Pair.of(LZ4, LZ4), 2.0); + + return compressions; + } + + private static Map, Double> initRowBasedCompressions() { + Map, Double> compressions = Maps.newHashMap(); + + compressions.put(Pair.of(NONE, ZSTD), 2.0); + compressions.put(Pair.of(NONE, GZIP), 2.0); + compressions.put(Pair.of(NONE, ZLIB), 2.0); + + compressions.put(Pair.of(ZSTD, SNAPPY), 0.5); + compressions.put(Pair.of(ZSTD, LZ4), 0.5); + + compressions.put(Pair.of(SNAPPY, ZSTD), 1.5); + compressions.put(Pair.of(SNAPPY, GZIP), 1.5); + compressions.put(Pair.of(SNAPPY, ZLIB), 1.5); + + compressions.put(Pair.of(LZ4, ZSTD), 1.5); + compressions.put(Pair.of(LZ4, GZIP), 1.5); + compressions.put(Pair.of(LZ4, ZLIB), 1.5); + + return compressions; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java new file mode 100644 index 000000000000..923e84a161f7 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java @@ -0,0 +1,320 @@ +/* + * 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.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import org.apache.iceberg.Table; +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.Lists; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.sql.RuntimeConfig; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +class SparkConfParser { + + private final Map properties; + private final RuntimeConfig sessionConf; + private final CaseInsensitiveStringMap options; + + SparkConfParser() { + this.properties = ImmutableMap.of(); + this.sessionConf = new org.apache.spark.sql.classic.RuntimeConfig(SQLConf.get()); + this.options = CaseInsensitiveStringMap.empty(); + } + + SparkConfParser(SparkSession spark, Table table, Map options) { + this.properties = table.properties(); + this.sessionConf = spark.conf(); + this.options = asCaseInsensitiveStringMap(options); + } + + public BooleanConfParser booleanConf() { + return new BooleanConfParser(); + } + + public IntConfParser intConf() { + return new IntConfParser(); + } + + public LongConfParser longConf() { + return new LongConfParser(); + } + + public StringConfParser stringConf() { + return new StringConfParser(); + } + + public DurationConfParser durationConf() { + return new DurationConfParser(); + } + + public > EnumConfParser enumConf(Function toEnum) { + return new EnumConfParser<>(toEnum); + } + + private static CaseInsensitiveStringMap asCaseInsensitiveStringMap(Map map) { + if (map instanceof CaseInsensitiveStringMap) { + return (CaseInsensitiveStringMap) map; + } else { + return new CaseInsensitiveStringMap(map); + } + } + + class BooleanConfParser extends ConfParser { + private Boolean defaultValue; + private boolean negate = false; + + @Override + protected BooleanConfParser self() { + return this; + } + + public BooleanConfParser defaultValue(boolean value) { + this.defaultValue = value; + return self(); + } + + public BooleanConfParser defaultValue(String value) { + this.defaultValue = Boolean.parseBoolean(value); + return self(); + } + + public BooleanConfParser negate() { + this.negate = true; + return self(); + } + + public boolean parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + boolean value = parse(Boolean::parseBoolean, defaultValue); + return negate ? !value : value; + } + } + + class IntConfParser extends ConfParser { + private Integer defaultValue; + + @Override + protected IntConfParser self() { + return this; + } + + public IntConfParser defaultValue(int value) { + this.defaultValue = value; + return self(); + } + + public int parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Integer::parseInt, defaultValue); + } + + public Integer parseOptional() { + return parse(Integer::parseInt, defaultValue); + } + } + + class LongConfParser extends ConfParser { + private Long defaultValue; + + @Override + protected LongConfParser self() { + return this; + } + + public LongConfParser defaultValue(long value) { + this.defaultValue = value; + return self(); + } + + public long parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Long::parseLong, defaultValue); + } + + public Long parseOptional() { + return parse(Long::parseLong, defaultValue); + } + } + + class StringConfParser extends ConfParser { + private String defaultValue; + + @Override + protected StringConfParser self() { + return this; + } + + public StringConfParser defaultValue(String value) { + this.defaultValue = value; + return self(); + } + + public String parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Function.identity(), defaultValue); + } + + public String parseOptional() { + return parse(Function.identity(), defaultValue); + } + } + + class DurationConfParser extends ConfParser { + private Duration defaultValue; + + @Override + protected DurationConfParser self() { + return this; + } + + public DurationConfParser defaultValue(Duration value) { + this.defaultValue = value; + return self(); + } + + public Duration parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(this::toDuration, defaultValue); + } + + public Duration parseOptional() { + return parse(this::toDuration, defaultValue); + } + + private Duration toDuration(String time) { + return Duration.ofSeconds(JavaUtils.timeStringAsSec(time)); + } + } + + class EnumConfParser> extends ConfParser, T> { + private final Function toEnum; + private T defaultValue; + + EnumConfParser(Function toEnum) { + this.toEnum = toEnum; + } + + @Override + protected EnumConfParser self() { + return this; + } + + public EnumConfParser defaultValue(T value) { + this.defaultValue = value; + return self(); + } + + public EnumConfParser defaultValue(String value) { + this.defaultValue = toEnum.apply(value); + return self(); + } + + public T parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(toEnum, defaultValue); + } + + public T parseOptional() { + return parse(toEnum, defaultValue); + } + } + + abstract class ConfParser { + private final List optionNames = Lists.newArrayList(); + private String sessionConfName; + private String tablePropertyName; + + protected abstract ThisT self(); + + public ThisT option(String name) { + this.optionNames.add(name); + return self(); + } + + public ThisT sessionConf(String name) { + this.sessionConfName = name; + return self(); + } + + public ThisT tableProperty(String name) { + this.tablePropertyName = name; + return self(); + } + + protected T parse(Function conversion, T defaultValue) { + for (String optionName : optionNames) { + String optionValue = options.get(optionName); + if (optionValue != null) { + return conversion.apply(optionValue); + } + + String sparkOptionValue = options.get(toCamelCase(optionName)); + if (sparkOptionValue != null) { + return conversion.apply(sparkOptionValue); + } + } + + if (sessionConfName != null) { + String sessionConfValue = sessionConf.get(sessionConfName, null); + if (sessionConfValue != null) { + return conversion.apply(sessionConfValue); + } + + String sparkSessionConfValue = sessionConf.get(toCamelCase(sessionConfName), null); + if (sparkSessionConfValue != null) { + return conversion.apply(sparkSessionConfValue); + } + } + + if (tablePropertyName != null) { + String propertyValue = properties.get(tablePropertyName); + if (propertyValue != null) { + return conversion.apply(propertyValue); + } + } + + return defaultValue; + } + + private String toCamelCase(String key) { + StringBuilder transformedKey = new StringBuilder(); + boolean capitalizeNext = false; + + for (char character : key.toCharArray()) { + if (character == '-') { + capitalizeNext = true; + } else if (capitalizeNext) { + transformedKey.append(Character.toUpperCase(character)); + capitalizeNext = false; + } else { + transformedKey.append(character); + } + } + + return transformedKey.toString(); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java new file mode 100644 index 000000000000..bad31d8d85f4 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java @@ -0,0 +1,277 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.StructProjection; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; + +public abstract class SparkContentFile implements ContentFile { + + private static final FileContent[] FILE_CONTENT_VALUES = FileContent.values(); + + private final int fileContentPosition; + private final int filePathPosition; + private final int fileFormatPosition; + private final int partitionPosition; + private final int recordCountPosition; + private final int fileSizeInBytesPosition; + private final int columnSizesPosition; + private final int valueCountsPosition; + private final int nullValueCountsPosition; + private final int nanValueCountsPosition; + private final int lowerBoundsPosition; + private final int upperBoundsPosition; + private final int keyMetadataPosition; + private final int splitOffsetsPosition; + private final int sortOrderIdPosition; + private final int fileSpecIdPosition; + private final int equalityIdsPosition; + private final int referencedDataFilePosition; + private final int contentOffsetPosition; + private final int contentSizePosition; + private final Type lowerBoundsType; + private final Type upperBoundsType; + private final Type keyMetadataType; + + private final SparkStructLike wrappedPartition; + private final StructLike projectedPartition; + private Row wrapped; + + SparkContentFile(Types.StructType type, Types.StructType projectedType, StructType sparkType) { + this.lowerBoundsType = type.fieldType(DataFile.LOWER_BOUNDS.name()); + this.upperBoundsType = type.fieldType(DataFile.UPPER_BOUNDS.name()); + this.keyMetadataType = type.fieldType(DataFile.KEY_METADATA.name()); + + Types.StructType partitionType = type.fieldType(DataFile.PARTITION_NAME).asStructType(); + this.wrappedPartition = new SparkStructLike(partitionType); + + if (projectedType != null) { + Types.StructType projectedPartitionType = + projectedType.fieldType(DataFile.PARTITION_NAME).asStructType(); + StructProjection partitionProjection = + StructProjection.create(partitionType, projectedPartitionType); + this.projectedPartition = partitionProjection.wrap(wrappedPartition); + } else { + this.projectedPartition = wrappedPartition; + } + + Map positions = Maps.newHashMap(); + for (Types.NestedField field : type.fields()) { + String fieldName = field.name(); + positions.put(fieldName, fieldPosition(fieldName, sparkType)); + } + + this.fileContentPosition = positions.get(DataFile.CONTENT.name()); + this.filePathPosition = positions.get(DataFile.FILE_PATH.name()); + this.fileFormatPosition = positions.get(DataFile.FILE_FORMAT.name()); + this.partitionPosition = positions.get(DataFile.PARTITION_NAME); + this.recordCountPosition = positions.get(DataFile.RECORD_COUNT.name()); + this.fileSizeInBytesPosition = positions.get(DataFile.FILE_SIZE.name()); + this.columnSizesPosition = positions.get(DataFile.COLUMN_SIZES.name()); + this.valueCountsPosition = positions.get(DataFile.VALUE_COUNTS.name()); + this.nullValueCountsPosition = positions.get(DataFile.NULL_VALUE_COUNTS.name()); + this.nanValueCountsPosition = positions.get(DataFile.NAN_VALUE_COUNTS.name()); + this.lowerBoundsPosition = positions.get(DataFile.LOWER_BOUNDS.name()); + this.upperBoundsPosition = positions.get(DataFile.UPPER_BOUNDS.name()); + this.keyMetadataPosition = positions.get(DataFile.KEY_METADATA.name()); + this.splitOffsetsPosition = positions.get(DataFile.SPLIT_OFFSETS.name()); + this.sortOrderIdPosition = positions.get(DataFile.SORT_ORDER_ID.name()); + this.fileSpecIdPosition = positions.get(DataFile.SPEC_ID.name()); + this.equalityIdsPosition = positions.get(DataFile.EQUALITY_IDS.name()); + this.referencedDataFilePosition = positions.get(DataFile.REFERENCED_DATA_FILE.name()); + this.contentOffsetPosition = positions.get(DataFile.CONTENT_OFFSET.name()); + this.contentSizePosition = positions.get(DataFile.CONTENT_SIZE.name()); + } + + public F wrap(Row row) { + this.wrapped = row; + if (wrappedPartition.size() > 0) { + wrappedPartition.wrap(row.getAs(partitionPosition)); + } + return asFile(); + } + + protected abstract F asFile(); + + @Override + public Long pos() { + return null; + } + + @Override + public int specId() { + if (wrapped.isNullAt(fileSpecIdPosition)) { + return -1; + } + return wrapped.getAs(fileSpecIdPosition); + } + + @Override + public FileContent content() { + if (wrapped.isNullAt(fileContentPosition)) { + return null; + } + return FILE_CONTENT_VALUES[wrapped.getInt(fileContentPosition)]; + } + + @Override + public CharSequence path() { + return wrapped.getAs(filePathPosition); + } + + @Override + public FileFormat format() { + return FileFormat.fromString(wrapped.getString(fileFormatPosition)); + } + + @Override + public StructLike partition() { + return projectedPartition; + } + + @Override + public long recordCount() { + return wrapped.getAs(recordCountPosition); + } + + @Override + public long fileSizeInBytes() { + return wrapped.getAs(fileSizeInBytesPosition); + } + + @Override + public Map columnSizes() { + return wrapped.isNullAt(columnSizesPosition) ? null : wrapped.getJavaMap(columnSizesPosition); + } + + @Override + public Map valueCounts() { + return wrapped.isNullAt(valueCountsPosition) ? null : wrapped.getJavaMap(valueCountsPosition); + } + + @Override + public Map nullValueCounts() { + if (wrapped.isNullAt(nullValueCountsPosition)) { + return null; + } + return wrapped.getJavaMap(nullValueCountsPosition); + } + + @Override + public Map nanValueCounts() { + if (wrapped.isNullAt(nanValueCountsPosition)) { + return null; + } + return wrapped.getJavaMap(nanValueCountsPosition); + } + + @Override + public Map lowerBounds() { + Map lowerBounds = + wrapped.isNullAt(lowerBoundsPosition) ? null : wrapped.getJavaMap(lowerBoundsPosition); + return convert(lowerBoundsType, lowerBounds); + } + + @Override + public Map upperBounds() { + Map upperBounds = + wrapped.isNullAt(upperBoundsPosition) ? null : wrapped.getJavaMap(upperBoundsPosition); + return convert(upperBoundsType, upperBounds); + } + + @Override + public ByteBuffer keyMetadata() { + return convert(keyMetadataType, wrapped.get(keyMetadataPosition)); + } + + @Override + public F copy() { + throw new UnsupportedOperationException("Not implemented: copy"); + } + + @Override + public F copyWithoutStats() { + throw new UnsupportedOperationException("Not implemented: copyWithoutStats"); + } + + @Override + public List splitOffsets() { + return wrapped.isNullAt(splitOffsetsPosition) ? null : wrapped.getList(splitOffsetsPosition); + } + + @Override + public Integer sortOrderId() { + return wrapped.getAs(sortOrderIdPosition); + } + + @Override + public List equalityFieldIds() { + return wrapped.isNullAt(equalityIdsPosition) ? null : wrapped.getList(equalityIdsPosition); + } + + public String referencedDataFile() { + if (wrapped.isNullAt(referencedDataFilePosition)) { + return null; + } + return wrapped.getString(referencedDataFilePosition); + } + + public Long contentOffset() { + if (wrapped.isNullAt(contentOffsetPosition)) { + return null; + } + return wrapped.getLong(contentOffsetPosition); + } + + public Long contentSizeInBytes() { + if (wrapped.isNullAt(contentSizePosition)) { + return null; + } + return wrapped.getLong(contentSizePosition); + } + + private int fieldPosition(String name, StructType sparkType) { + try { + return sparkType.fieldIndex(name); + } catch (IllegalArgumentException e) { + // the partition field is absent for unpartitioned tables + if (name.equals(DataFile.PARTITION_NAME) && wrappedPartition.size() == 0) { + return -1; + } + throw e; + } + } + + @SuppressWarnings("unchecked") + private T convert(Type valueType, Object value) { + return (T) SparkValueConverter.convert(valueType, value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java new file mode 100644 index 000000000000..543ebf3f9ea7 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java @@ -0,0 +1,46 @@ +/* + * 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.List; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.types.StructType; + +public class SparkDataFile extends SparkContentFile implements DataFile { + + public SparkDataFile(Types.StructType type, StructType sparkType) { + super(type, null, sparkType); + } + + public SparkDataFile( + Types.StructType type, Types.StructType projectedType, StructType sparkType) { + super(type, projectedType, sparkType); + } + + @Override + protected DataFile asFile() { + return this; + } + + @Override + public List equalityFieldIds() { + return null; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkDeleteFile.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkDeleteFile.java new file mode 100644 index 000000000000..6250a1630683 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkDeleteFile.java @@ -0,0 +1,40 @@ +/* + * 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 org.apache.iceberg.DeleteFile; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.types.StructType; + +public class SparkDeleteFile extends SparkContentFile implements DeleteFile { + + public SparkDeleteFile(Types.StructType type, StructType sparkType) { + super(type, null, sparkType); + } + + public SparkDeleteFile( + Types.StructType type, Types.StructType projectedType, StructType sparkType) { + super(type, projectedType, sparkType); + } + + @Override + protected DeleteFile asFile() { + return this; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkExceptionUtil.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkExceptionUtil.java new file mode 100644 index 000000000000..5c6fe3e0ff96 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkExceptionUtil.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 com.google.errorprone.annotations.FormatMethod; +import java.io.IOException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.spark.sql.AnalysisException; + +public class SparkExceptionUtil { + + private SparkExceptionUtil() {} + + /** + * Converts checked exceptions to unchecked exceptions. + * + * @param cause a checked exception object which is to be converted to its unchecked equivalent. + * @param message exception message as a format string + * @param args format specifiers + * @return unchecked exception. + */ + @FormatMethod + public static RuntimeException toUncheckedException( + final Throwable cause, final String message, final Object... args) { + // Parameters are required to be final to help @FormatMethod do static analysis + if (cause instanceof RuntimeException) { + return (RuntimeException) cause; + + } else if (cause instanceof org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException) { + return new NoSuchNamespaceException(cause, message, args); + + } else if (cause instanceof org.apache.spark.sql.catalyst.analysis.NoSuchTableException) { + return new NoSuchTableException(cause, message, args); + + } else if (cause instanceof AnalysisException) { + return new ValidationException(cause, message, args); + + } else if (cause instanceof IOException) { + return new RuntimeIOException((IOException) cause, message, args); + + } else { + return new RuntimeException(String.format(message, args), cause); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkExecutorCache.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkExecutorCache.java new file mode 100644 index 000000000000..6490d6678b46 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkExecutorCache.java @@ -0,0 +1,228 @@ +/* + * 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 com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import java.time.Duration; +import java.util.List; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An executor cache for reducing the computation and IO overhead in tasks. + * + *

    The cache is configured and controlled through Spark SQL properties. It supports both limits + * on the total cache size and maximum size for individual entries. Additionally, it implements + * automatic eviction of entries after a specified duration of inactivity. The cache will respect + * the SQL configuration valid at the time of initialization. All subsequent changes to the + * configuration will have no effect. + * + *

    The cache is accessed and populated via {@link #getOrLoad(String, String, Supplier, long)}. If + * the value is not present in the cache, it is computed using the provided supplier and stored in + * the cache, subject to the defined size constraints. When a key is added, it must be associated + * with a particular group ID. Once the group is no longer needed, it is recommended to explicitly + * invalidate its state by calling {@link #invalidate(String)} instead of relying on automatic + * eviction. + * + *

    Note that this class employs the singleton pattern to ensure only one cache exists per JVM. + */ +public class SparkExecutorCache { + + private static final Logger LOG = LoggerFactory.getLogger(SparkExecutorCache.class); + + private static volatile SparkExecutorCache instance = null; + + private final Duration timeout; + private final long maxEntrySize; + private final long maxTotalSize; + private volatile Cache state; + + private SparkExecutorCache(Conf conf) { + this.timeout = conf.timeout(); + this.maxEntrySize = conf.maxEntrySize(); + this.maxTotalSize = conf.maxTotalSize(); + } + + /** + * Returns the cache if created or creates and returns it. + * + *

    Note this method returns null if caching is disabled. + */ + public static SparkExecutorCache getOrCreate() { + if (instance == null) { + Conf conf = new Conf(); + if (conf.cacheEnabled()) { + synchronized (SparkExecutorCache.class) { + if (instance == null) { + SparkExecutorCache.instance = new SparkExecutorCache(conf); + } + } + } + } + + return instance; + } + + /** Returns the cache if already created or null otherwise. */ + public static SparkExecutorCache get() { + return instance; + } + + /** Returns the max entry size in bytes that will be considered for caching. */ + public long maxEntrySize() { + return maxEntrySize; + } + + /** + * Gets the cached value for the key or populates the cache with a new mapping. + * + * @param group a group ID + * @param key a cache key + * @param valueSupplier a supplier to compute the value + * @param valueSize an estimated memory size of the value in bytes + * @return the cached or computed value + */ + public V getOrLoad(String group, String key, Supplier valueSupplier, long valueSize) { + if (valueSize > maxEntrySize) { + LOG.debug("{} exceeds max entry size: {} > {}", key, valueSize, maxEntrySize); + return valueSupplier.get(); + } + + String internalKey = group + "_" + key; + CacheValue value = state().get(internalKey, loadFunc(valueSupplier, valueSize)); + Preconditions.checkNotNull(value, "Loaded value must not be null"); + return value.get(); + } + + private Function loadFunc(Supplier valueSupplier, long valueSize) { + return key -> { + long start = System.currentTimeMillis(); + V value = valueSupplier.get(); + long end = System.currentTimeMillis(); + LOG.debug("Loaded {} with size {} in {} ms", key, valueSize, (end - start)); + return new CacheValue(value, valueSize); + }; + } + + /** + * Invalidates all keys associated with the given group ID. + * + * @param group a group ID + */ + public void invalidate(String group) { + if (state != null) { + List internalKeys = findInternalKeys(group); + LOG.info("Invalidating {} keys associated with {}", internalKeys.size(), group); + internalKeys.forEach(internalKey -> state.invalidate(internalKey)); + LOG.info("Current cache stats {}", state.stats()); + } + } + + private List findInternalKeys(String group) { + return state.asMap().keySet().stream() + .filter(internalKey -> internalKey.startsWith(group)) + .collect(Collectors.toList()); + } + + private Cache state() { + if (state == null) { + synchronized (this) { + if (state == null) { + LOG.info("Initializing cache state"); + this.state = initState(); + } + } + } + + return state; + } + + private Cache initState() { + return Caffeine.newBuilder() + .expireAfterAccess(timeout) + .maximumWeight(maxTotalSize) + .weigher((key, value) -> ((CacheValue) value).weight()) + .recordStats() + .removalListener((key, value, cause) -> LOG.debug("Evicted {} ({})", key, cause)) + .build(); + } + + @VisibleForTesting + static class CacheValue { + private final Object value; + private final long size; + + CacheValue(Object value, long size) { + this.value = value; + this.size = size; + } + + @SuppressWarnings("unchecked") + public V get() { + return (V) value; + } + + public int weight() { + return (int) Math.min(size, Integer.MAX_VALUE); + } + } + + @VisibleForTesting + static class Conf { + private final SparkConfParser confParser = new SparkConfParser(); + + public boolean cacheEnabled() { + return confParser + .booleanConf() + .sessionConf(SparkSQLProperties.EXECUTOR_CACHE_ENABLED) + .defaultValue(SparkSQLProperties.EXECUTOR_CACHE_ENABLED_DEFAULT) + .parse(); + } + + public Duration timeout() { + return confParser + .durationConf() + .sessionConf(SparkSQLProperties.EXECUTOR_CACHE_TIMEOUT) + .defaultValue(SparkSQLProperties.EXECUTOR_CACHE_TIMEOUT_DEFAULT) + .parse(); + } + + public long maxEntrySize() { + return confParser + .longConf() + .sessionConf(SparkSQLProperties.EXECUTOR_CACHE_MAX_ENTRY_SIZE) + .defaultValue(SparkSQLProperties.EXECUTOR_CACHE_MAX_ENTRY_SIZE_DEFAULT) + .parse(); + } + + public long maxTotalSize() { + return confParser + .longConf() + .sessionConf(SparkSQLProperties.EXECUTOR_CACHE_MAX_TOTAL_SIZE) + .defaultValue(SparkSQLProperties.EXECUTOR_CACHE_MAX_TOTAL_SIZE_DEFAULT) + .parse(); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java new file mode 100644 index 000000000000..49b73c7b01af --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFilters.java @@ -0,0 +1,280 @@ +/* + * 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 static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.in; +import static org.apache.iceberg.expressions.Expressions.isNaN; +import static org.apache.iceberg.expressions.Expressions.isNull; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.notIn; +import static org.apache.iceberg.expressions.Expressions.notNull; +import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.expressions.Expressions.startsWith; + +import java.sql.Date; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.util.Map; +import java.util.Objects; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.util.NaNUtil; +import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.apache.spark.sql.sources.AlwaysFalse; +import org.apache.spark.sql.sources.AlwaysFalse$; +import org.apache.spark.sql.sources.AlwaysTrue; +import org.apache.spark.sql.sources.AlwaysTrue$; +import org.apache.spark.sql.sources.And; +import org.apache.spark.sql.sources.EqualNullSafe; +import org.apache.spark.sql.sources.EqualTo; +import org.apache.spark.sql.sources.Filter; +import org.apache.spark.sql.sources.GreaterThan; +import org.apache.spark.sql.sources.GreaterThanOrEqual; +import org.apache.spark.sql.sources.In; +import org.apache.spark.sql.sources.IsNotNull; +import org.apache.spark.sql.sources.IsNull; +import org.apache.spark.sql.sources.LessThan; +import org.apache.spark.sql.sources.LessThanOrEqual; +import org.apache.spark.sql.sources.Not; +import org.apache.spark.sql.sources.Or; +import org.apache.spark.sql.sources.StringStartsWith; + +public class SparkFilters { + + private static final Pattern BACKTICKS_PATTERN = Pattern.compile("([`])(.|$)"); + + private SparkFilters() {} + + private static final Map, Operation> FILTERS = + ImmutableMap., Operation>builder() + .put(AlwaysTrue.class, Operation.TRUE) + .put(AlwaysTrue$.class, Operation.TRUE) + .put(AlwaysFalse$.class, Operation.FALSE) + .put(AlwaysFalse.class, Operation.FALSE) + .put(EqualTo.class, Operation.EQ) + .put(EqualNullSafe.class, Operation.EQ) + .put(GreaterThan.class, Operation.GT) + .put(GreaterThanOrEqual.class, Operation.GT_EQ) + .put(LessThan.class, Operation.LT) + .put(LessThanOrEqual.class, Operation.LT_EQ) + .put(In.class, Operation.IN) + .put(IsNull.class, Operation.IS_NULL) + .put(IsNotNull.class, Operation.NOT_NULL) + .put(And.class, Operation.AND) + .put(Or.class, Operation.OR) + .put(Not.class, Operation.NOT) + .put(StringStartsWith.class, Operation.STARTS_WITH) + .buildOrThrow(); + + public static Expression convert(Filter[] filters) { + Expression expression = Expressions.alwaysTrue(); + for (Filter filter : filters) { + Expression converted = convert(filter); + Preconditions.checkArgument( + converted != null, "Cannot convert filter to Iceberg: %s", filter); + expression = Expressions.and(expression, converted); + } + return expression; + } + + public static Expression convert(Filter filter) { + // avoid using a chain of if instanceof statements by mapping to the expression enum. + Operation op = FILTERS.get(filter.getClass()); + if (op != null) { + switch (op) { + case TRUE: + return Expressions.alwaysTrue(); + + case FALSE: + return Expressions.alwaysFalse(); + + case IS_NULL: + IsNull isNullFilter = (IsNull) filter; + return isNull(unquote(isNullFilter.attribute())); + + case NOT_NULL: + IsNotNull notNullFilter = (IsNotNull) filter; + return notNull(unquote(notNullFilter.attribute())); + + case LT: + LessThan lt = (LessThan) filter; + return lessThan(unquote(lt.attribute()), convertLiteral(lt.value())); + + case LT_EQ: + LessThanOrEqual ltEq = (LessThanOrEqual) filter; + return lessThanOrEqual(unquote(ltEq.attribute()), convertLiteral(ltEq.value())); + + case GT: + GreaterThan gt = (GreaterThan) filter; + return greaterThan(unquote(gt.attribute()), convertLiteral(gt.value())); + + case GT_EQ: + GreaterThanOrEqual gtEq = (GreaterThanOrEqual) filter; + return greaterThanOrEqual(unquote(gtEq.attribute()), convertLiteral(gtEq.value())); + + case EQ: // used for both eq and null-safe-eq + if (filter instanceof EqualTo) { + EqualTo eq = (EqualTo) filter; + // comparison with null in normal equality is always null. this is probably a mistake. + Preconditions.checkNotNull( + eq.value(), "Expression is always false (eq is not null-safe): %s", filter); + return handleEqual(unquote(eq.attribute()), eq.value()); + } else { + EqualNullSafe eq = (EqualNullSafe) filter; + if (eq.value() == null) { + return isNull(unquote(eq.attribute())); + } else { + return handleEqual(unquote(eq.attribute()), eq.value()); + } + } + + case IN: + In inFilter = (In) filter; + return in( + unquote(inFilter.attribute()), + Stream.of(inFilter.values()) + .filter(Objects::nonNull) + .map(SparkFilters::convertLiteral) + .collect(Collectors.toList())); + + case NOT: + Not notFilter = (Not) filter; + Filter childFilter = notFilter.child(); + Operation childOp = FILTERS.get(childFilter.getClass()); + if (childOp == Operation.IN) { + // infer an extra notNull predicate for Spark NOT IN filters + // as Iceberg expressions don't follow the 3-value SQL boolean logic + // col NOT IN (1, 2) in Spark is equivalent to notNull(col) && notIn(col, 1, 2) in + // Iceberg + In childInFilter = (In) childFilter; + Expression notIn = + notIn( + unquote(childInFilter.attribute()), + Stream.of(childInFilter.values()) + .map(SparkFilters::convertLiteral) + .collect(Collectors.toList())); + return and(notNull(childInFilter.attribute()), notIn); + } else if (hasNoInFilter(childFilter)) { + Expression child = convert(childFilter); + if (child != null) { + return not(child); + } + } + return null; + + case AND: + { + And andFilter = (And) filter; + Expression left = convert(andFilter.left()); + Expression right = convert(andFilter.right()); + if (left != null && right != null) { + return and(left, right); + } + return null; + } + + case OR: + { + Or orFilter = (Or) filter; + Expression left = convert(orFilter.left()); + Expression right = convert(orFilter.right()); + if (left != null && right != null) { + return or(left, right); + } + return null; + } + + case STARTS_WITH: + { + StringStartsWith stringStartsWith = (StringStartsWith) filter; + return startsWith(unquote(stringStartsWith.attribute()), stringStartsWith.value()); + } + } + } + + return null; + } + + private static Object convertLiteral(Object value) { + if (value instanceof Timestamp) { + return DateTimeUtils.fromJavaTimestamp((Timestamp) value); + } else if (value instanceof Date) { + return DateTimeUtils.fromJavaDate((Date) value); + } else if (value instanceof Instant) { + return DateTimeUtils.instantToMicros((Instant) value); + } else if (value instanceof LocalDateTime) { + return DateTimeUtils.localDateTimeToMicros((LocalDateTime) value); + } else if (value instanceof LocalDate) { + return DateTimeUtils.localDateToDays((LocalDate) value); + } + return value; + } + + private static Expression handleEqual(String attribute, Object value) { + if (NaNUtil.isNaN(value)) { + return isNaN(attribute); + } else { + return equal(attribute, convertLiteral(value)); + } + } + + private static String unquote(String attributeName) { + Matcher matcher = BACKTICKS_PATTERN.matcher(attributeName); + return matcher.replaceAll("$2"); + } + + private static boolean hasNoInFilter(Filter filter) { + Operation op = FILTERS.get(filter.getClass()); + + if (op != null) { + switch (op) { + case AND: + And andFilter = (And) filter; + return hasNoInFilter(andFilter.left()) && hasNoInFilter(andFilter.right()); + case OR: + Or orFilter = (Or) filter; + return hasNoInFilter(orFilter.left()) && hasNoInFilter(orFilter.right()); + case NOT: + Not notFilter = (Not) filter; + return hasNoInFilter(notFilter.child()); + case IN: + return false; + default: + return true; + } + } + + return false; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTypes.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTypes.java new file mode 100644 index 000000000000..6c4ec39b20f1 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFixupTypes.java @@ -0,0 +1,63 @@ +/* + * 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 org.apache.iceberg.Schema; +import org.apache.iceberg.types.FixupTypes; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; + +/** + * Some types, like binary and fixed, are converted to the same Spark type. Conversion back can + * produce only one, which may not be correct. + */ +class SparkFixupTypes extends FixupTypes { + + private SparkFixupTypes(Schema referenceSchema) { + super(referenceSchema); + } + + static Schema fixup(Schema schema, Schema referenceSchema) { + return new Schema( + TypeUtil.visit(schema, new SparkFixupTypes(referenceSchema)).asStructType().fields()); + } + + @Override + protected boolean fixupPrimitive(Type.PrimitiveType type, Type source) { + switch (type.typeId()) { + case STRING: + if (source.typeId() == Type.TypeID.UUID) { + return true; + } + break; + case BINARY: + if (source.typeId() == Type.TypeID.FIXED) { + return true; + } + break; + case TIMESTAMP: + if (source.typeId() == Type.TypeID.TIMESTAMP) { + return true; + } + break; + default: + } + return false; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFunctionCatalog.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFunctionCatalog.java new file mode 100644 index 000000000000..2183b9e5df4d --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkFunctionCatalog.java @@ -0,0 +1,45 @@ +/* + * 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 org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * A function catalog that can be used to resolve Iceberg functions without a metastore connection. + */ +public class SparkFunctionCatalog implements SupportsFunctions { + + private static final SparkFunctionCatalog INSTANCE = new SparkFunctionCatalog(); + + private String name = "iceberg-function-catalog"; + + public static SparkFunctionCatalog get() { + return INSTANCE; + } + + @Override + public void initialize(String catalogName, CaseInsensitiveStringMap options) { + this.name = catalogName; + } + + @Override + public String name() { + return name; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java new file mode 100644 index 000000000000..2788e160d526 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -0,0 +1,378 @@ +/* + * 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 static org.apache.iceberg.PlanningMode.LOCAL; + +import java.util.Map; +import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.SparkSession; + +/** + * A class for common Iceberg configs for Spark reads. + * + *

    If a config is set at multiple levels, the following order of precedence is used (top to + * bottom): + * + *

      + *
    1. Read options + *
    2. Session configuration + *
    3. Table metadata + *
    + * + * The most specific value is set in read options and takes precedence over all other configs. If no + * read option is provided, this class checks the session configuration for any overrides. If no + * applicable value is found in the session configuration, this class uses the table metadata. + * + *

    Note this class is NOT meant to be serialized and sent to executors. + */ +public class SparkReadConf { + + private static final String DRIVER_MAX_RESULT_SIZE = "spark.driver.maxResultSize"; + private static final String DRIVER_MAX_RESULT_SIZE_DEFAULT = "1G"; + private static final long DISTRIBUTED_PLANNING_MIN_RESULT_SIZE = 256L * 1024 * 1024; // 256 MB + + private final SparkSession spark; + private final Table table; + private final String branch; + private final SparkConfParser confParser; + + public SparkReadConf(SparkSession spark, Table table, Map readOptions) { + this(spark, table, null, readOptions); + } + + public SparkReadConf( + SparkSession spark, Table table, String branch, Map readOptions) { + this.spark = spark; + this.table = table; + this.branch = branch; + this.confParser = new SparkConfParser(spark, table, readOptions); + } + + public boolean caseSensitive() { + return SparkUtil.caseSensitive(spark); + } + + public boolean localityEnabled() { + boolean defaultValue = Util.mayHaveBlockLocations(table.io(), table.location()); + return confParser + .booleanConf() + .option(SparkReadOptions.LOCALITY) + .sessionConf(SparkSQLProperties.LOCALITY) + .defaultValue(defaultValue) + .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(); + } + + public Long endSnapshotId() { + return confParser.longConf().option(SparkReadOptions.END_SNAPSHOT_ID).parseOptional(); + } + + public String branch() { + String optionBranch = confParser.stringConf().option(SparkReadOptions.BRANCH).parseOptional(); + ValidationException.check( + branch == null || optionBranch == null || optionBranch.equals(branch), + "Must not specify different branches in both table identifier and read option, " + + "got [%s] in identifier and [%s] in options", + branch, + optionBranch); + String inputBranch = branch != null ? branch : optionBranch; + if (inputBranch != null) { + return inputBranch; + } + + boolean wapEnabled = + PropertyUtil.propertyAsBoolean( + table.properties(), TableProperties.WRITE_AUDIT_PUBLISH_ENABLED, false); + if (wapEnabled) { + String wapBranch = spark.conf().get(SparkSQLProperties.WAP_BRANCH, null); + if (wapBranch != null && table.refs().containsKey(wapBranch)) { + return wapBranch; + } + } + + return null; + } + + public String tag() { + return confParser.stringConf().option(SparkReadOptions.TAG).parseOptional(); + } + + public String scanTaskSetId() { + return confParser.stringConf().option(SparkReadOptions.SCAN_TASK_SET_ID).parseOptional(); + } + + public boolean streamingSkipDeleteSnapshots() { + return confParser + .booleanConf() + .option(SparkReadOptions.STREAMING_SKIP_DELETE_SNAPSHOTS) + .defaultValue(SparkReadOptions.STREAMING_SKIP_DELETE_SNAPSHOTS_DEFAULT) + .parse(); + } + + public boolean streamingSkipOverwriteSnapshots() { + return confParser + .booleanConf() + .option(SparkReadOptions.STREAMING_SKIP_OVERWRITE_SNAPSHOTS) + .defaultValue(SparkReadOptions.STREAMING_SKIP_OVERWRITE_SNAPSHOTS_DEFAULT) + .parse(); + } + + public boolean parquetVectorizationEnabled() { + return confParser + .booleanConf() + .option(SparkReadOptions.VECTORIZATION_ENABLED) + .sessionConf(SparkSQLProperties.VECTORIZATION_ENABLED) + .tableProperty(TableProperties.PARQUET_VECTORIZATION_ENABLED) + .defaultValue(TableProperties.PARQUET_VECTORIZATION_ENABLED_DEFAULT) + .parse(); + } + + public int parquetBatchSize() { + return confParser + .intConf() + .option(SparkReadOptions.VECTORIZATION_BATCH_SIZE) + .tableProperty(TableProperties.PARQUET_BATCH_SIZE) + .defaultValue(TableProperties.PARQUET_BATCH_SIZE_DEFAULT) + .parse(); + } + + public boolean orcVectorizationEnabled() { + return confParser + .booleanConf() + .option(SparkReadOptions.VECTORIZATION_ENABLED) + .sessionConf(SparkSQLProperties.VECTORIZATION_ENABLED) + .tableProperty(TableProperties.ORC_VECTORIZATION_ENABLED) + .defaultValue(TableProperties.ORC_VECTORIZATION_ENABLED_DEFAULT) + .parse(); + } + + public int orcBatchSize() { + return confParser + .intConf() + .option(SparkReadOptions.VECTORIZATION_BATCH_SIZE) + .tableProperty(TableProperties.ORC_BATCH_SIZE) + .defaultValue(TableProperties.ORC_BATCH_SIZE_DEFAULT) + .parse(); + } + + public Long splitSizeOption() { + return confParser.longConf().option(SparkReadOptions.SPLIT_SIZE).parseOptional(); + } + + public long splitSize() { + return confParser + .longConf() + .option(SparkReadOptions.SPLIT_SIZE) + .tableProperty(TableProperties.SPLIT_SIZE) + .defaultValue(TableProperties.SPLIT_SIZE_DEFAULT) + .parse(); + } + + public Integer splitLookbackOption() { + return confParser.intConf().option(SparkReadOptions.LOOKBACK).parseOptional(); + } + + public int splitLookback() { + return confParser + .intConf() + .option(SparkReadOptions.LOOKBACK) + .tableProperty(TableProperties.SPLIT_LOOKBACK) + .defaultValue(TableProperties.SPLIT_LOOKBACK_DEFAULT) + .parse(); + } + + public Long splitOpenFileCostOption() { + return confParser.longConf().option(SparkReadOptions.FILE_OPEN_COST).parseOptional(); + } + + public long splitOpenFileCost() { + return confParser + .longConf() + .option(SparkReadOptions.FILE_OPEN_COST) + .tableProperty(TableProperties.SPLIT_OPEN_FILE_COST) + .defaultValue(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT) + .parse(); + } + + public long streamFromTimestamp() { + return confParser + .longConf() + .option(SparkReadOptions.STREAM_FROM_TIMESTAMP) + .defaultValue(Long.MIN_VALUE) + .parse(); + } + + public Long startTimestamp() { + return confParser.longConf().option(SparkReadOptions.START_TIMESTAMP).parseOptional(); + } + + public Long endTimestamp() { + return confParser.longConf().option(SparkReadOptions.END_TIMESTAMP).parseOptional(); + } + + public int maxFilesPerMicroBatch() { + return confParser + .intConf() + .option(SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH) + .defaultValue(Integer.MAX_VALUE) + .parse(); + } + + public int maxRecordsPerMicroBatch() { + return confParser + .intConf() + .option(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH) + .defaultValue(Integer.MAX_VALUE) + .parse(); + } + + public boolean preserveDataGrouping() { + return confParser + .booleanConf() + .sessionConf(SparkSQLProperties.PRESERVE_DATA_GROUPING) + .defaultValue(SparkSQLProperties.PRESERVE_DATA_GROUPING_DEFAULT) + .parse(); + } + + public boolean aggregatePushDownEnabled() { + return confParser + .booleanConf() + .option(SparkReadOptions.AGGREGATE_PUSH_DOWN_ENABLED) + .sessionConf(SparkSQLProperties.AGGREGATE_PUSH_DOWN_ENABLED) + .defaultValue(SparkSQLProperties.AGGREGATE_PUSH_DOWN_ENABLED_DEFAULT) + .parse(); + } + + public boolean adaptiveSplitSizeEnabled() { + return confParser + .booleanConf() + .tableProperty(TableProperties.ADAPTIVE_SPLIT_SIZE_ENABLED) + .defaultValue(TableProperties.ADAPTIVE_SPLIT_SIZE_ENABLED_DEFAULT) + .parse(); + } + + public int parallelism() { + int defaultParallelism = spark.sparkContext().defaultParallelism(); + int numShufflePartitions = spark.sessionState().conf().numShufflePartitions(); + return Math.max(defaultParallelism, numShufflePartitions); + } + + public boolean distributedPlanningEnabled() { + return dataPlanningMode() != LOCAL || deletePlanningMode() != LOCAL; + } + + public PlanningMode dataPlanningMode() { + if (driverMaxResultSize() < DISTRIBUTED_PLANNING_MIN_RESULT_SIZE) { + return LOCAL; + } + + return confParser + .enumConf(PlanningMode::fromName) + .sessionConf(SparkSQLProperties.DATA_PLANNING_MODE) + .tableProperty(TableProperties.DATA_PLANNING_MODE) + .defaultValue(TableProperties.PLANNING_MODE_DEFAULT) + .parse(); + } + + public PlanningMode deletePlanningMode() { + if (driverMaxResultSize() < DISTRIBUTED_PLANNING_MIN_RESULT_SIZE) { + return LOCAL; + } + + String modeName = + confParser + .stringConf() + .sessionConf(SparkSQLProperties.DELETE_PLANNING_MODE) + .tableProperty(TableProperties.DELETE_PLANNING_MODE) + .defaultValue(TableProperties.PLANNING_MODE_DEFAULT) + .parse(); + return PlanningMode.fromName(modeName); + } + + private long driverMaxResultSize() { + SparkConf sparkConf = spark.sparkContext().conf(); + return sparkConf.getSizeAsBytes(DRIVER_MAX_RESULT_SIZE, DRIVER_MAX_RESULT_SIZE_DEFAULT); + } + + public boolean executorCacheLocalityEnabled() { + return executorCacheEnabled() && executorCacheLocalityEnabledInternal(); + } + + private boolean executorCacheEnabled() { + return confParser + .booleanConf() + .sessionConf(SparkSQLProperties.EXECUTOR_CACHE_ENABLED) + .defaultValue(SparkSQLProperties.EXECUTOR_CACHE_ENABLED_DEFAULT) + .parse(); + } + + public boolean cacheDeleteFilesOnExecutors() { + return executorCacheEnabled() && cacheDeleteFilesOnExecutorsInternal(); + } + + private boolean cacheDeleteFilesOnExecutorsInternal() { + return confParser + .booleanConf() + .sessionConf(SparkSQLProperties.EXECUTOR_CACHE_DELETE_FILES_ENABLED) + .defaultValue(SparkSQLProperties.EXECUTOR_CACHE_DELETE_FILES_ENABLED_DEFAULT) + .parse(); + } + + private boolean executorCacheLocalityEnabledInternal() { + return confParser + .booleanConf() + .sessionConf(SparkSQLProperties.EXECUTOR_CACHE_LOCALITY_ENABLED) + .defaultValue(SparkSQLProperties.EXECUTOR_CACHE_LOCALITY_ENABLED_DEFAULT) + .parse(); + } + + public boolean reportColumnStats() { + return confParser + .booleanConf() + .sessionConf(SparkSQLProperties.REPORT_COLUMN_STATS) + .defaultValue(SparkSQLProperties.REPORT_COLUMN_STATS_DEFAULT) + .parse(); + } + + public ParquetReaderType parquetReaderType() { + return confParser + .enumConf(ParquetReaderType::fromString) + .sessionConf(SparkSQLProperties.PARQUET_READER_TYPE) + .defaultValue(SparkSQLProperties.PARQUET_READER_TYPE_DEFAULT) + .parse(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java new file mode 100644 index 000000000000..17f2bfee69b8 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadOptions.java @@ -0,0 +1,98 @@ +/* + * 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; + +/** Spark DF read options */ +public class SparkReadOptions { + + private SparkReadOptions() {} + + // Snapshot ID of the table snapshot to read + public static final String SNAPSHOT_ID = "snapshot-id"; + + // Start snapshot ID used in incremental scans (exclusive) + public static final String START_SNAPSHOT_ID = "start-snapshot-id"; + + // End snapshot ID used in incremental scans (inclusive) + public static final String END_SNAPSHOT_ID = "end-snapshot-id"; + + // Start timestamp used in multi-snapshot scans (exclusive) + public static final String START_TIMESTAMP = "start-timestamp"; + + // 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"; + + // Overrides the table's read.split.planning-lookback + public static final String LOOKBACK = "lookback"; + + // Overrides the table's read.split.open-file-cost + public static final String FILE_OPEN_COST = "file-open-cost"; + + // Overrides table's vectorization enabled properties + public static final String VECTORIZATION_ENABLED = "vectorization-enabled"; + + // Overrides the table's read.parquet.vectorization.batch-size + public static final String VECTORIZATION_BATCH_SIZE = "batch-size"; + + // Set ID that is used to fetch scan tasks + public static final String SCAN_TASK_SET_ID = "scan-task-set-id"; + + // skip snapshots of type delete while reading stream out of iceberg table + public static final String STREAMING_SKIP_DELETE_SNAPSHOTS = "streaming-skip-delete-snapshots"; + public static final boolean STREAMING_SKIP_DELETE_SNAPSHOTS_DEFAULT = false; + + // skip snapshots of type overwrite while reading stream out of iceberg table + public static final String STREAMING_SKIP_OVERWRITE_SNAPSHOTS = + "streaming-skip-overwrite-snapshots"; + public static final boolean STREAMING_SKIP_OVERWRITE_SNAPSHOTS_DEFAULT = false; + + // Controls whether to report locality information to Spark while allocating input partitions + public static final String LOCALITY = "locality"; + + // Timestamp in milliseconds; start a stream from the snapshot that occurs after this timestamp + public static final String STREAM_FROM_TIMESTAMP = "stream-from-timestamp"; + + // maximum file per micro_batch + public static final String STREAMING_MAX_FILES_PER_MICRO_BATCH = + "streaming-max-files-per-micro-batch"; + // maximum rows per micro_batch + public static final String STREAMING_MAX_ROWS_PER_MICRO_BATCH = + "streaming-max-rows-per-micro-batch"; + + // Table path + public static final String PATH = "path"; + + public static final String VERSION_AS_OF = "versionAsOf"; + + public static final String TIMESTAMP_AS_OF = "timestampAsOf"; + + public static final String AGGREGATE_PUSH_DOWN_ENABLED = "aggregate-push-down-enabled"; +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java new file mode 100644 index 000000000000..81139969f746 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -0,0 +1,112 @@ +/* + * 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.time.Duration; + +public class SparkSQLProperties { + + private SparkSQLProperties() {} + + // Controls whether vectorized reads are enabled + public static final String VECTORIZATION_ENABLED = "spark.sql.iceberg.vectorization.enabled"; + + // Controls which Parquet reader implementation to use + public static final String PARQUET_READER_TYPE = "spark.sql.iceberg.parquet.reader-type"; + public static final ParquetReaderType PARQUET_READER_TYPE_DEFAULT = ParquetReaderType.ICEBERG; + // Controls whether to perform the nullability check during writes + public static final String CHECK_NULLABILITY = "spark.sql.iceberg.check-nullability"; + public static final boolean CHECK_NULLABILITY_DEFAULT = true; + + // Controls whether to check the order of fields during writes + public static final String CHECK_ORDERING = "spark.sql.iceberg.check-ordering"; + public static final boolean CHECK_ORDERING_DEFAULT = true; + + // Controls whether to preserve the existing grouping of data while planning splits + public static final String PRESERVE_DATA_GROUPING = + "spark.sql.iceberg.planning.preserve-data-grouping"; + public static final boolean PRESERVE_DATA_GROUPING_DEFAULT = false; + + // Controls whether to push down aggregate (MAX/MIN/COUNT) to Iceberg + public static final String AGGREGATE_PUSH_DOWN_ENABLED = + "spark.sql.iceberg.aggregate-push-down.enabled"; + public static final boolean AGGREGATE_PUSH_DOWN_ENABLED_DEFAULT = true; + + // Controls write distribution mode + public static final String DISTRIBUTION_MODE = "spark.sql.iceberg.distribution-mode"; + + // Controls the WAP ID used for write-audit-publish workflow. + // When set, new snapshots will be staged with this ID in snapshot summary. + public static final String WAP_ID = "spark.wap.id"; + + // Controls the WAP branch used for write-audit-publish workflow. + // When set, new snapshots will be committed to this branch. + public static final String WAP_BRANCH = "spark.wap.branch"; + + // Controls write compress options + public static final String COMPRESSION_CODEC = "spark.sql.iceberg.compression-codec"; + public static final String COMPRESSION_LEVEL = "spark.sql.iceberg.compression-level"; + public static final String COMPRESSION_STRATEGY = "spark.sql.iceberg.compression-strategy"; + + // Overrides the data planning mode + public static final String DATA_PLANNING_MODE = "spark.sql.iceberg.data-planning-mode"; + + // Overrides the delete planning mode + public static final String DELETE_PLANNING_MODE = "spark.sql.iceberg.delete-planning-mode"; + + // Overrides the advisory partition size + public static final String ADVISORY_PARTITION_SIZE = "spark.sql.iceberg.advisory-partition-size"; + + // Controls whether to report locality information to Spark while allocating input partitions + public static final String LOCALITY = "spark.sql.iceberg.locality.enabled"; + + public static final String EXECUTOR_CACHE_ENABLED = "spark.sql.iceberg.executor-cache.enabled"; + public static final boolean EXECUTOR_CACHE_ENABLED_DEFAULT = true; + + // Controls whether to enable executor cache for delete files + public static final String EXECUTOR_CACHE_DELETE_FILES_ENABLED = + "spark.sql.iceberg.executor-cache.delete-files.enabled"; + public static final boolean EXECUTOR_CACHE_DELETE_FILES_ENABLED_DEFAULT = true; + + public static final String EXECUTOR_CACHE_TIMEOUT = "spark.sql.iceberg.executor-cache.timeout"; + public static final Duration EXECUTOR_CACHE_TIMEOUT_DEFAULT = Duration.ofMinutes(10); + + public static final String EXECUTOR_CACHE_MAX_ENTRY_SIZE = + "spark.sql.iceberg.executor-cache.max-entry-size"; + public static final long EXECUTOR_CACHE_MAX_ENTRY_SIZE_DEFAULT = 64 * 1024 * 1024; // 64 MB + + public static final String EXECUTOR_CACHE_MAX_TOTAL_SIZE = + "spark.sql.iceberg.executor-cache.max-total-size"; + public static final long EXECUTOR_CACHE_MAX_TOTAL_SIZE_DEFAULT = 128 * 1024 * 1024; // 128 MB + + // Controls whether to merge schema during write operation + public static final String MERGE_SCHEMA = "spark.sql.iceberg.merge-schema"; + public static final boolean MERGE_SCHEMA_DEFAULT = false; + + public static final String EXECUTOR_CACHE_LOCALITY_ENABLED = + "spark.sql.iceberg.executor-cache.locality.enabled"; + public static final boolean EXECUTOR_CACHE_LOCALITY_ENABLED_DEFAULT = false; + + // Controls whether to report available column statistics to Spark for query optimization. + public static final String REPORT_COLUMN_STATS = "spark.sql.iceberg.report-column-stats"; + public static final boolean REPORT_COLUMN_STATS_DEFAULT = true; + + // Prefix for custom snapshot properties + public static final String SNAPSHOT_PROPERTY_PREFIX = "spark.sql.iceberg.snapshot-property."; +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java new file mode 100644 index 000000000000..fcf5fbeb2acb --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java @@ -0,0 +1,372 @@ +/* + * 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.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Binder; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.relocated.com.google.common.base.Splitter; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.math.LongMath; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalog.Column; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.StructType; + +/** Helper methods for working with Spark/Hive metadata. */ +public class SparkSchemaUtil { + private SparkSchemaUtil() {} + + /** + * Returns a {@link Schema} for the given table with fresh field ids. + * + *

    This creates a Schema for an existing table by looking up the table's schema with Spark and + * converting that schema. Spark/Hive partition columns are included in the schema. + * + * @param spark a Spark session + * @param name a table name and (optional) database + * @return a Schema for the table, if found + */ + public static Schema schemaForTable(SparkSession spark, String name) { + return convert(spark.table(name).schema()); + } + + /** + * Returns a {@link PartitionSpec} for the given table. + * + *

    This creates a partition spec for an existing table by looking up the table's schema and + * creating a spec with identity partitions for each partition column. + * + * @param spark a Spark session + * @param name a table name and (optional) database + * @return a PartitionSpec for the table + * @throws AnalysisException if thrown by the Spark catalog + */ + public static PartitionSpec specForTable(SparkSession spark, String name) + throws AnalysisException { + List parts = Lists.newArrayList(Splitter.on('.').limit(2).split(name)); + String db = parts.size() == 1 ? "default" : parts.get(0); + String table = parts.get(parts.size() == 1 ? 0 : 1); + + PartitionSpec spec = + identitySpec( + schemaForTable(spark, name), spark.catalog().listColumns(db, table).collectAsList()); + return spec == null ? PartitionSpec.unpartitioned() : spec; + } + + /** + * Convert a {@link Schema} to a {@link DataType Spark type}. + * + * @param schema a Schema + * @return the equivalent Spark type + * @throws IllegalArgumentException if the type cannot be converted to Spark + */ + public static StructType convert(Schema schema) { + return (StructType) TypeUtil.visit(schema, new TypeToSparkType()); + } + + /** + * Convert a {@link Type} to a {@link DataType Spark type}. + * + * @param type a Type + * @return the equivalent Spark type + * @throws IllegalArgumentException if the type cannot be converted to Spark + */ + public static DataType convert(Type type) { + return TypeUtil.visit(type, new TypeToSparkType()); + } + + /** + * Convert a Spark {@link StructType struct} to a {@link Schema} with new field ids. + * + *

    This conversion assigns fresh ids. + * + *

    Some data types are represented as the same Spark type. These are converted to a default + * type. + * + *

    To convert using a reference schema for field ids and ambiguous types, use {@link + * #convert(Schema, StructType)}. + * + * @param sparkType a Spark StructType + * @return the equivalent Schema + * @throws IllegalArgumentException if the type cannot be converted + */ + public static Schema convert(StructType sparkType) { + Type converted = SparkTypeVisitor.visit(sparkType, new SparkTypeToType(sparkType)); + return new Schema(converted.asNestedType().asStructType().fields()); + } + + /** + * Convert a Spark {@link DataType struct} to a {@link Type} with new field ids. + * + *

    This conversion assigns fresh ids. + * + *

    Some data types are represented as the same Spark type. These are converted to a default + * type. + * + *

    To convert using a reference schema for field ids and ambiguous types, use {@link + * #convert(Schema, StructType)}. + * + * @param sparkType a Spark DataType + * @return the equivalent Type + * @throws IllegalArgumentException if the type cannot be converted + */ + public static Type convert(DataType sparkType) { + return SparkTypeVisitor.visit(sparkType, new SparkTypeToType()); + } + + /** + * Convert a Spark {@link StructType struct} to a {@link Schema} based on the given schema. + * + *

    This conversion does not assign new ids; it uses ids from the base schema. + * + *

    Data types, field order, and nullability will match the spark type. This conversion may + * return a schema that is not compatible with base schema. + * + * @param baseSchema a Schema on which conversion is based + * @param sparkType a Spark StructType + * @return the equivalent Schema + * @throws IllegalArgumentException if the type cannot be converted or there are missing ids + */ + public static Schema convert(Schema baseSchema, StructType sparkType) { + return convert(baseSchema, sparkType, true); + } + + /** + * Convert a Spark {@link StructType struct} to a {@link Schema} based on the given schema. + * + *

    This conversion does not assign new ids; it uses ids from the base schema. + * + *

    Data types, field order, and nullability will match the spark type. This conversion may + * return a schema that is not compatible with base schema. + * + * @param baseSchema a Schema on which conversion is based + * @param sparkType a Spark StructType + * @param caseSensitive when false, the case of schema fields is ignored + * @return the equivalent Schema + * @throws IllegalArgumentException if the type cannot be converted or there are missing ids + */ + public static Schema convert(Schema baseSchema, StructType sparkType, boolean caseSensitive) { + // convert to a type with fresh ids + Types.StructType struct = + SparkTypeVisitor.visit(sparkType, new SparkTypeToType(sparkType)).asStructType(); + // reassign ids to match the base schema + Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema, caseSensitive); + // fix types that can't be represented in Spark (UUID and Fixed) + return SparkFixupTypes.fixup(schema, baseSchema); + } + + /** + * Convert a Spark {@link StructType struct} to a {@link Schema} based on the given schema. + * + *

    This conversion will assign new ids for fields that are not found in the base schema. + * + *

    Data types, field order, and nullability will match the spark type. This conversion may + * return a schema that is not compatible with base schema. + * + * @param baseSchema a Schema on which conversion is based + * @param sparkType a Spark StructType + * @return the equivalent Schema + * @throws IllegalArgumentException if the type cannot be converted or there are missing ids + * @deprecated since 1.11.0, will be removed in 1.12.0 + */ + @Deprecated + public static Schema convertWithFreshIds(Schema baseSchema, StructType sparkType) { + return convertWithFreshIds(baseSchema, sparkType, true); + } + + /** + * Convert a Spark {@link StructType struct} to a {@link Schema} based on the given schema. + * + *

    This conversion will assign new ids for fields that are not found in the base schema. + * + *

    Data types, field order, and nullability will match the spark type. This conversion may + * return a schema that is not compatible with base schema. + * + * @param baseSchema a Schema on which conversion is based + * @param sparkType a Spark StructType + * @param caseSensitive when false, case of field names in schema is ignored + * @return the equivalent Schema + * @throws IllegalArgumentException if the type cannot be converted or there are missing ids + */ + public static Schema convertWithFreshIds( + Schema baseSchema, StructType sparkType, boolean caseSensitive) { + // convert to a type with fresh ids + Types.StructType struct = + SparkTypeVisitor.visit(sparkType, new SparkTypeToType(sparkType)).asStructType(); + // reassign ids to match the base schema + Schema schema = + TypeUtil.reassignOrRefreshIds(new Schema(struct.fields()), baseSchema, caseSensitive); + // fix types that can't be represented in Spark (UUID and Fixed) + return SparkFixupTypes.fixup(schema, baseSchema); + } + + /** + * Prune columns from a {@link Schema} using a {@link StructType Spark type} projection. + * + *

    This requires that the Spark type is a projection of the Schema. Nullability and types must + * match. + * + * @param schema a Schema + * @param requestedType a projection of the Spark representation of the Schema + * @return a Schema corresponding to the Spark projection + * @throws IllegalArgumentException if the Spark type does not match the Schema + */ + public static Schema prune(Schema schema, StructType requestedType) { + return new Schema( + TypeUtil.visit(schema, new PruneColumnsWithoutReordering(requestedType, ImmutableSet.of())) + .asNestedType() + .asStructType() + .fields()); + } + + /** + * Prune columns from a {@link Schema} using a {@link StructType Spark type} projection. + * + *

    This requires that the Spark type is a projection of the Schema. Nullability and types must + * match. + * + *

    The filters list of {@link Expression} is used to ensure that columns referenced by filters + * are projected. + * + * @param schema a Schema + * @param requestedType a projection of the Spark representation of the Schema + * @param filters a list of filters + * @return a Schema corresponding to the Spark projection + * @throws IllegalArgumentException if the Spark type does not match the Schema + * @deprecated since 1.11.0, will be removed in 1.12.0 + */ + @Deprecated + public static Schema prune(Schema schema, StructType requestedType, List filters) { + Set filterRefs = Binder.boundReferences(schema.asStruct(), filters, true); + return new Schema( + TypeUtil.visit(schema, new PruneColumnsWithoutReordering(requestedType, filterRefs)) + .asNestedType() + .asStructType() + .fields()); + } + + /** + * Prune columns from a {@link Schema} using a {@link StructType Spark type} projection. + * + *

    This requires that the Spark type is a projection of the Schema. Nullability and types must + * match. + * + *

    The filters list of {@link Expression} is used to ensure that columns referenced by filters + * are projected. + * + * @param schema a Schema + * @param requestedType a projection of the Spark representation of the Schema + * @param filter a filters + * @return a Schema corresponding to the Spark projection + * @throws IllegalArgumentException if the Spark type does not match the Schema + */ + public static Schema prune( + Schema schema, StructType requestedType, Expression filter, boolean caseSensitive) { + Set filterRefs = + Binder.boundReferences(schema.asStruct(), Collections.singletonList(filter), caseSensitive); + + return new Schema( + TypeUtil.visit(schema, new PruneColumnsWithoutReordering(requestedType, filterRefs)) + .asNestedType() + .asStructType() + .fields()); + } + + private static PartitionSpec identitySpec(Schema schema, Collection columns) { + List names = Lists.newArrayList(); + for (Column column : columns) { + if (column.isPartition()) { + names.add(column.name()); + } + } + + return identitySpec(schema, names); + } + + private static PartitionSpec identitySpec(Schema schema, List partitionNames) { + if (partitionNames == null || partitionNames.isEmpty()) { + return null; + } + + PartitionSpec.Builder builder = PartitionSpec.builderFor(schema); + for (String partitionName : partitionNames) { + builder.identity(partitionName); + } + + return builder.build(); + } + + /** + * Estimate approximate table size based on Spark schema and total records. + * + * @param tableSchema Spark schema + * @param totalRecords total records in the table + * @return approximate size based on table schema + */ + public static long estimateSize(StructType tableSchema, long totalRecords) { + if (totalRecords == Long.MAX_VALUE) { + return totalRecords; + } + + long result; + try { + result = LongMath.checkedMultiply(tableSchema.defaultSize(), totalRecords); + } catch (ArithmeticException e) { + result = Long.MAX_VALUE; + } + return result; + } + + public static void validateMetadataColumnReferences(Schema tableSchema, Schema readSchema) { + List conflictingColumnNames = + readSchema.columns().stream() + .map(Types.NestedField::name) + .filter( + name -> + MetadataColumns.isMetadataColumn(name) && tableSchema.findField(name) != null) + .collect(Collectors.toList()); + + ValidationException.check( + conflictingColumnNames.isEmpty(), + "Table column names conflict with names reserved for Iceberg metadata columns: %s.\n" + + "Please, use ALTER TABLE statements to rename the conflicting table columns.", + conflictingColumnNames); + } + + public static Map indexQuotedNameById(Schema schema) { + Function quotingFunc = name -> String.format("`%s`", name.replace("`", "``")); + return TypeUtil.indexQuotedNameById(schema.asStruct(), quotingFunc); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java new file mode 100644 index 000000000000..f49660a9f27c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSessionCatalog.java @@ -0,0 +1,542 @@ +/* + * 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.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.source.HasIcebergCatalog; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NamespaceAlreadyExistsException; +import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException; +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.analysis.NoSuchViewException; +import org.apache.spark.sql.catalyst.analysis.NonEmptyNamespaceException; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.catalyst.analysis.ViewAlreadyExistsException; +import org.apache.spark.sql.connector.catalog.CatalogExtension; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; +import org.apache.spark.sql.connector.catalog.FunctionCatalog; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.NamespaceChange; +import org.apache.spark.sql.connector.catalog.StagedTable; +import org.apache.spark.sql.connector.catalog.StagingTableCatalog; +import org.apache.spark.sql.connector.catalog.SupportsNamespaces; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.TableChange; +import org.apache.spark.sql.connector.catalog.View; +import org.apache.spark.sql.connector.catalog.ViewCatalog; +import org.apache.spark.sql.connector.catalog.ViewChange; +import org.apache.spark.sql.connector.catalog.ViewInfo; +import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * A Spark catalog that can also load non-Iceberg tables. + * + * @param CatalogPlugin class to avoid casting to TableCatalog, FunctionCatalog and + * SupportsNamespaces. + */ +public class SparkSessionCatalog< + T extends TableCatalog & FunctionCatalog & SupportsNamespaces & ViewCatalog> + extends BaseCatalog implements CatalogExtension { + private static final String[] DEFAULT_NAMESPACE = new String[] {"default"}; + + private String catalogName = null; + private TableCatalog icebergCatalog = null; + private StagingTableCatalog asStagingCatalog = null; + private ViewCatalog asViewCatalog = null; + private T sessionCatalog = null; + private boolean createParquetAsIceberg = false; + private boolean createAvroAsIceberg = false; + private boolean createOrcAsIceberg = false; + + /** + * Build a {@link SparkCatalog} to be used for Iceberg operations. + * + *

    The default implementation creates a new SparkCatalog with the session catalog's name and + * options. + * + * @param name catalog name + * @param options catalog options + * @return a SparkCatalog to be used for Iceberg tables + */ + protected TableCatalog buildSparkCatalog(String name, CaseInsensitiveStringMap options) { + SparkCatalog newCatalog = new SparkCatalog(); + newCatalog.initialize(name, options); + return newCatalog; + } + + @Override + public String[] defaultNamespace() { + return DEFAULT_NAMESPACE; + } + + @Override + public String[][] listNamespaces() throws NoSuchNamespaceException { + return getSessionCatalog().listNamespaces(); + } + + @Override + public String[][] listNamespaces(String[] namespace) throws NoSuchNamespaceException { + return getSessionCatalog().listNamespaces(namespace); + } + + @Override + public boolean namespaceExists(String[] namespace) { + return getSessionCatalog().namespaceExists(namespace); + } + + @Override + public Map loadNamespaceMetadata(String[] namespace) + throws NoSuchNamespaceException { + return getSessionCatalog().loadNamespaceMetadata(namespace); + } + + @Override + public void createNamespace(String[] namespace, Map metadata) + throws NamespaceAlreadyExistsException { + getSessionCatalog().createNamespace(namespace, metadata); + } + + @Override + public void alterNamespace(String[] namespace, NamespaceChange... changes) + throws NoSuchNamespaceException { + getSessionCatalog().alterNamespace(namespace, changes); + } + + @Override + public boolean dropNamespace(String[] namespace, boolean cascade) + throws NoSuchNamespaceException, NonEmptyNamespaceException { + return getSessionCatalog().dropNamespace(namespace, cascade); + } + + @Override + public Identifier[] listTables(String[] namespace) throws NoSuchNamespaceException { + // delegate to the session catalog because all tables share the same namespace + return getSessionCatalog().listTables(namespace); + } + + @Override + public Table loadTable(Identifier ident) throws NoSuchTableException { + try { + return icebergCatalog.loadTable(ident); + } catch (NoSuchTableException e) { + return getSessionCatalog().loadTable(ident); + } + } + + @Override + public Table loadTable(Identifier ident, String version) throws NoSuchTableException { + try { + return icebergCatalog.loadTable(ident, version); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + return getSessionCatalog().loadTable(ident, version); + } + } + + @Override + public Table loadTable(Identifier ident, long timestamp) throws NoSuchTableException { + try { + return icebergCatalog.loadTable(ident, timestamp); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + return getSessionCatalog().loadTable(ident, timestamp); + } + } + + @Override + public void invalidateTable(Identifier ident) { + // We do not need to check whether the table exists and whether + // it is an Iceberg table to reduce remote service requests. + icebergCatalog.invalidateTable(ident); + getSessionCatalog().invalidateTable(ident); + } + + @Override + public boolean tableExists(Identifier ident) { + return icebergCatalog.tableExists(ident) || getSessionCatalog().tableExists(ident); + } + + @Override + public Table createTable( + Identifier ident, StructType schema, Transform[] partitions, Map properties) + throws TableAlreadyExistsException, NoSuchNamespaceException { + String provider = properties.get("provider"); + if (useIceberg(provider)) { + return icebergCatalog.createTable(ident, schema, partitions, properties); + } else { + // delegate to the session catalog + return getSessionCatalog().createTable(ident, schema, partitions, properties); + } + } + + @Override + public StagedTable stageCreate( + Identifier ident, StructType schema, Transform[] partitions, Map properties) + throws TableAlreadyExistsException, NoSuchNamespaceException { + String provider = properties.get("provider"); + TableCatalog catalog; + if (useIceberg(provider)) { + if (asStagingCatalog != null) { + return asStagingCatalog.stageCreate(ident, schema, partitions, properties); + } + catalog = icebergCatalog; + } else { + catalog = getSessionCatalog(); + } + + // create the table with the session catalog, then wrap it in a staged table that will delete to + // roll back + Table table = catalog.createTable(ident, schema, partitions, properties); + return new RollbackStagedTable(catalog, ident, table); + } + + @Override + public StagedTable stageReplace( + Identifier ident, StructType schema, Transform[] partitions, Map properties) + throws NoSuchNamespaceException, NoSuchTableException { + String provider = properties.get("provider"); + TableCatalog catalog; + if (useIceberg(provider)) { + if (asStagingCatalog != null) { + return asStagingCatalog.stageReplace(ident, schema, partitions, properties); + } + catalog = icebergCatalog; + } else { + catalog = getSessionCatalog(); + } + + // attempt to drop the table and fail if it doesn't exist + if (!catalog.dropTable(ident)) { + throw new NoSuchTableException(ident); + } + + try { + // create the table with the session catalog, then wrap it in a staged table that will delete + // to roll back + Table table = catalog.createTable(ident, schema, partitions, properties); + return new RollbackStagedTable(catalog, ident, table); + + } catch (TableAlreadyExistsException e) { + // the table was deleted, but now already exists again. retry the replace. + return stageReplace(ident, schema, partitions, properties); + } + } + + @Override + public StagedTable stageCreateOrReplace( + Identifier ident, StructType schema, Transform[] partitions, Map properties) + throws NoSuchNamespaceException { + String provider = properties.get("provider"); + TableCatalog catalog; + if (useIceberg(provider)) { + if (asStagingCatalog != null) { + return asStagingCatalog.stageCreateOrReplace(ident, schema, partitions, properties); + } + catalog = icebergCatalog; + } else { + catalog = getSessionCatalog(); + } + + // drop the table if it exists + catalog.dropTable(ident); + + try { + // create the table with the session catalog, then wrap it in a staged table that will delete + // to roll back + Table sessionCatalogTable = catalog.createTable(ident, schema, partitions, properties); + return new RollbackStagedTable(catalog, ident, sessionCatalogTable); + + } catch (TableAlreadyExistsException e) { + // the table was deleted, but now already exists again. retry the replace. + return stageCreateOrReplace(ident, schema, partitions, properties); + } + } + + @Override + public Table alterTable(Identifier ident, TableChange... changes) throws NoSuchTableException { + if (icebergCatalog.tableExists(ident)) { + return icebergCatalog.alterTable(ident, changes); + } else { + return getSessionCatalog().alterTable(ident, changes); + } + } + + @Override + public boolean dropTable(Identifier ident) { + // no need to check table existence to determine which catalog to use. if a table doesn't exist + // then both are + // required to return false. + return icebergCatalog.dropTable(ident) || getSessionCatalog().dropTable(ident); + } + + @Override + public boolean purgeTable(Identifier ident) { + // no need to check table existence to determine which catalog to use. if a table doesn't exist + // then both are + // required to return false. + return icebergCatalog.purgeTable(ident) || getSessionCatalog().purgeTable(ident); + } + + @Override + public void renameTable(Identifier from, Identifier to) + throws NoSuchTableException, TableAlreadyExistsException { + // rename is not supported by HadoopCatalog. to avoid UnsupportedOperationException for session + // catalog tables, + // check table existence first to ensure that the table belongs to the Iceberg catalog. + if (icebergCatalog.tableExists(from)) { + icebergCatalog.renameTable(from, to); + } else { + getSessionCatalog().renameTable(from, to); + } + } + + @Override + public final void initialize(String name, CaseInsensitiveStringMap options) { + super.initialize(name, options); + + if (options.containsKey(CatalogUtil.ICEBERG_CATALOG_TYPE) + && options + .get(CatalogUtil.ICEBERG_CATALOG_TYPE) + .equalsIgnoreCase(CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE)) { + validateHmsUri(options.get(CatalogProperties.URI)); + } + + this.catalogName = name; + this.icebergCatalog = buildSparkCatalog(name, options); + if (icebergCatalog instanceof StagingTableCatalog) { + this.asStagingCatalog = (StagingTableCatalog) icebergCatalog; + } + + if (icebergCatalog instanceof ViewCatalog) { + this.asViewCatalog = (ViewCatalog) icebergCatalog; + } + + this.createParquetAsIceberg = options.getBoolean("parquet-enabled", createParquetAsIceberg); + this.createAvroAsIceberg = options.getBoolean("avro-enabled", createAvroAsIceberg); + this.createOrcAsIceberg = options.getBoolean("orc-enabled", createOrcAsIceberg); + } + + private void validateHmsUri(String catalogHmsUri) { + if (catalogHmsUri == null) { + return; + } + + Configuration conf = SparkSession.active().sessionState().newHadoopConf(); + String envHmsUri = conf.get(HiveConf.ConfVars.METASTOREURIS.varname, null); + if (envHmsUri == null) { + return; + } + + Preconditions.checkArgument( + catalogHmsUri.equals(envHmsUri), + "Inconsistent Hive metastore URIs: %s (Spark session) != %s (spark_catalog)", + envHmsUri, + catalogHmsUri); + } + + @Override + @SuppressWarnings("unchecked") + public void setDelegateCatalog(CatalogPlugin sparkSessionCatalog) { + if (sparkSessionCatalog instanceof TableCatalog + && sparkSessionCatalog instanceof FunctionCatalog + && sparkSessionCatalog instanceof SupportsNamespaces) { + this.sessionCatalog = (T) sparkSessionCatalog; + } else { + throw new IllegalArgumentException("Invalid session catalog: " + sparkSessionCatalog); + } + } + + @Override + public String name() { + return catalogName; + } + + private boolean useIceberg(String provider) { + if (provider == null || "iceberg".equalsIgnoreCase(provider)) { + return true; + } else if (createParquetAsIceberg && "parquet".equalsIgnoreCase(provider)) { + return true; + } else if (createAvroAsIceberg && "avro".equalsIgnoreCase(provider)) { + return true; + } else if (createOrcAsIceberg && "orc".equalsIgnoreCase(provider)) { + return true; + } + + return false; + } + + private T getSessionCatalog() { + Preconditions.checkNotNull( + sessionCatalog, + "Delegated SessionCatalog is missing. " + + "Please make sure your are replacing Spark's default catalog, named 'spark_catalog'."); + return sessionCatalog; + } + + @Override + public Catalog icebergCatalog() { + Preconditions.checkArgument( + icebergCatalog instanceof HasIcebergCatalog, + "Cannot return underlying Iceberg Catalog, wrapped catalog does not contain an Iceberg Catalog"); + return ((HasIcebergCatalog) icebergCatalog).icebergCatalog(); + } + + private boolean isViewCatalog() { + return getSessionCatalog() instanceof ViewCatalog; + } + + @Override + public boolean functionExists(Identifier ident) { + return super.functionExists(ident) || getSessionCatalog().functionExists(ident); + } + + @Override + public UnboundFunction loadFunction(Identifier ident) throws NoSuchFunctionException { + try { + return super.loadFunction(ident); + } catch (NoSuchFunctionException e) { + return getSessionCatalog().loadFunction(ident); + } + } + + @Override + public Identifier[] listViews(String... namespace) { + try { + if (null != asViewCatalog) { + return asViewCatalog.listViews(namespace); + } else if (isViewCatalog()) { + getSessionCatalog().listViews(namespace); + } + } catch (NoSuchNamespaceException e) { + throw new RuntimeException(e); + } + + return new Identifier[0]; + } + + @Override + public boolean viewExists(Identifier ident) { + return (asViewCatalog != null && asViewCatalog.viewExists(ident)) + || (isViewCatalog() && getSessionCatalog().viewExists(ident)); + } + + @Override + public View loadView(Identifier ident) throws NoSuchViewException { + if (null != asViewCatalog && asViewCatalog.viewExists(ident)) { + return asViewCatalog.loadView(ident); + } else if (isViewCatalog() && getSessionCatalog().viewExists(ident)) { + return getSessionCatalog().loadView(ident); + } + + throw new NoSuchViewException(ident); + } + + @Override + public View createView(ViewInfo viewInfo) + throws ViewAlreadyExistsException, NoSuchNamespaceException { + if (viewInfo == null) { + return null; + } + + if (null != asViewCatalog) { + return asViewCatalog.createView(viewInfo); + } else if (isViewCatalog()) { + return getSessionCatalog().createView(viewInfo); + } + + throw new UnsupportedOperationException( + "Creating a view is not supported by catalog: " + catalogName); + } + + @Override + public View replaceView( + Identifier ident, + String sql, + String currentCatalog, + String[] currentNamespace, + StructType schema, + String[] queryColumnNames, + String[] columnAliases, + String[] columnComments, + Map properties) + throws NoSuchNamespaceException, NoSuchViewException { + if (asViewCatalog instanceof SupportsReplaceView) { + return ((SupportsReplaceView) asViewCatalog) + .replaceView( + ident, + sql, + currentCatalog, + currentNamespace, + schema, + queryColumnNames, + columnAliases, + columnComments, + properties); + } + + throw new UnsupportedOperationException( + "Replacing a view is not supported by catalog: " + catalogName); + } + + @Override + public View alterView(Identifier ident, ViewChange... changes) + throws NoSuchViewException, IllegalArgumentException { + if (null != asViewCatalog && asViewCatalog.viewExists(ident)) { + return asViewCatalog.alterView(ident, changes); + } else if (isViewCatalog()) { + return getSessionCatalog().alterView(ident, changes); + } + + throw new UnsupportedOperationException( + "Altering a view is not supported by catalog: " + catalogName); + } + + @Override + public boolean dropView(Identifier ident) { + if (null != asViewCatalog && asViewCatalog.viewExists(ident)) { + return asViewCatalog.dropView(ident); + } else if (isViewCatalog()) { + return getSessionCatalog().dropView(ident); + } + + return false; + } + + @Override + public void renameView(Identifier fromIdentifier, Identifier toIdentifier) + throws NoSuchViewException, ViewAlreadyExistsException { + if (null != asViewCatalog && asViewCatalog.viewExists(fromIdentifier)) { + asViewCatalog.renameView(fromIdentifier, toIdentifier); + } else if (isViewCatalog()) { + getSessionCatalog().renameView(fromIdentifier, toIdentifier); + } else { + throw new UnsupportedOperationException( + "Renaming a view is not supported by catalog: " + catalogName); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java new file mode 100644 index 000000000000..77cfa0f34c63 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkStructLike.java @@ -0,0 +1,54 @@ +/* + * 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 org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Row; + +public class SparkStructLike implements StructLike { + + private final Types.StructType type; + private Row wrapped; + + public SparkStructLike(Types.StructType type) { + this.type = type; + } + + public SparkStructLike wrap(Row row) { + this.wrapped = row; + return this; + } + + @Override + public int size() { + return type.fields().size(); + } + + @Override + public T get(int pos, Class javaClass) { + Types.NestedField field = type.fields().get(pos); + return javaClass.cast(SparkValueConverter.convert(field.type(), wrapped.get(pos))); + } + + @Override + public void set(int pos, T value) { + throw new UnsupportedOperationException("Not implemented: set"); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTableCache.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTableCache.java new file mode 100644 index 000000000000..6218423db491 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTableCache.java @@ -0,0 +1,54 @@ +/* + * 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.Map; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public class SparkTableCache { + + private static final SparkTableCache INSTANCE = new SparkTableCache(); + + private final Map cache = Maps.newConcurrentMap(); + + public static SparkTableCache get() { + return INSTANCE; + } + + public int size() { + return cache.size(); + } + + public void add(String key, Table table) { + cache.put(key, table); + } + + public boolean contains(String key) { + return cache.containsKey(key); + } + + public Table get(String key) { + return cache.get(key); + } + + public Table remove(String key) { + return cache.remove(key); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java new file mode 100644 index 000000000000..98d3e41535cf --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTableUtil.java @@ -0,0 +1,1311 @@ +/* + * 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 static org.apache.spark.sql.functions.col; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.Serializable; +import java.net.URI; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.TableMigrationUtil; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.hadoop.SerializableConfiguration; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.SupportsBulkOperations; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Splitter; +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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.SparkTable; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.util.ThreadPools; +import org.apache.spark.TaskContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.MapPartitionsFunction; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.catalog.CatalogTablePartition; +import org.apache.spark.sql.catalyst.catalog.SessionCatalog; +import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.catalyst.expressions.NamedExpression; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Function2; +import scala.Option; +import scala.Some; +import scala.Tuple2; +import scala.collection.JavaConverters; +import scala.collection.immutable.Map$; +import scala.collection.immutable.Seq; +import scala.collection.mutable.Builder; +import scala.runtime.AbstractPartialFunction; + +/** + * Java version of the original SparkTableUtil.scala + * https://github.com/apache/iceberg/blob/apache-iceberg-0.8.0-incubating/spark/src/main/scala/org/apache/iceberg/spark/SparkTableUtil.scala + */ +public class SparkTableUtil { + private static final Logger LOG = LoggerFactory.getLogger(SparkTableUtil.class); + + private static final String DUPLICATE_FILE_MESSAGE = + "Cannot complete import because data files " + + "to be imported already exist within the target table: %s. " + + "This is disabled by default as Iceberg is not designed for multiple references to the same file" + + " within the same table. If you are sure, you may set 'check_duplicate_files' to false to force the import."; + + private SparkTableUtil() {} + + /** + * Returns a DataFrame with a row for each partition in the table. + * + *

    The DataFrame has 3 columns, partition key (a=1/b=2), partition location, and format (avro + * or parquet). + * + * @param spark a Spark session + * @param table a table name and (optional) database + * @return a DataFrame of the table's partitions + * @deprecated since 1.11.0, will be removed in 1.12.0 + */ + @Deprecated + public static Dataset partitionDF(SparkSession spark, String table) { + List partitions = getPartitions(spark, table); + return spark + .createDataFrame(partitions, SparkPartition.class) + .toDF("partition", "uri", "format"); + } + + /** + * Returns a DataFrame with a row for each partition that matches the specified 'expression'. + * + * @param spark a Spark session. + * @param table name of the table. + * @param expression The expression whose matching partitions are returned. + * @return a DataFrame of the table partitions. + * @deprecated since 1.11.0, will be removed in 1.12.0 + */ + @Deprecated + public static Dataset partitionDFByFilter( + SparkSession spark, String table, String expression) { + List partitions = getPartitionsByFilter(spark, table, expression); + return spark + .createDataFrame(partitions, SparkPartition.class) + .toDF("partition", "uri", "format"); + } + + /** + * Returns all partitions in the table. + * + * @param spark a Spark session + * @param table a table name and (optional) database + * @return all table's partitions + * @deprecated since 1.11.0, will be removed in 1.12.0 + */ + @Deprecated + public static List getPartitions(SparkSession spark, String table) { + try { + TableIdentifier tableIdent = spark.sessionState().sqlParser().parseTableIdentifier(table); + return getPartitions(spark, tableIdent, null); + } catch (ParseException e) { + throw SparkExceptionUtil.toUncheckedException( + e, "Unable to parse table identifier: %s", table); + } + } + + /** + * Returns all partitions in the table. + * + * @param spark a Spark session + * @param tableIdent a table identifier + * @param partitionFilter partition filter, or null if no filter + * @return all table's partitions + */ + public static List getPartitions( + SparkSession spark, TableIdentifier tableIdent, Map partitionFilter) { + try { + SessionCatalog catalog = spark.sessionState().catalog(); + CatalogTable catalogTable = catalog.getTableMetadata(tableIdent); + + Option> scalaPartitionFilter; + if (partitionFilter != null && !partitionFilter.isEmpty()) { + Builder, scala.collection.immutable.Map> builder = + Map$.MODULE$.newBuilder(); + partitionFilter.forEach((key, value) -> builder.$plus$eq(Tuple2.apply(key, value))); + scalaPartitionFilter = Option.apply(builder.result()); + } else { + scalaPartitionFilter = Option.empty(); + } + Seq partitions = + catalog.listPartitions(tableIdent, scalaPartitionFilter).toIndexedSeq(); + return JavaConverters.seqAsJavaListConverter(partitions).asJava().stream() + .map(catalogPartition -> toSparkPartition(catalogPartition, catalogTable)) + .collect(Collectors.toList()); + } catch (NoSuchDatabaseException e) { + throw SparkExceptionUtil.toUncheckedException( + e, "Unknown table: %s. Database not found in catalog.", tableIdent); + } catch (NoSuchTableException e) { + throw SparkExceptionUtil.toUncheckedException( + e, "Unknown table: %s. Table not found in catalog.", tableIdent); + } + } + + /** + * Returns partitions that match the specified 'predicate'. + * + * @param spark a Spark session + * @param table a table name and (optional) database + * @param predicate a predicate on partition columns + * @return matching table's partitions + * @deprecated since 1.11.0, will be removed in 1.12.0 + */ + @Deprecated + public static List getPartitionsByFilter( + SparkSession spark, String table, String predicate) { + TableIdentifier tableIdent; + try { + tableIdent = spark.sessionState().sqlParser().parseTableIdentifier(table); + } catch (ParseException e) { + throw SparkExceptionUtil.toUncheckedException( + e, "Unable to parse the table identifier: %s", table); + } + + Expression unresolvedPredicateExpr; + try { + unresolvedPredicateExpr = spark.sessionState().sqlParser().parseExpression(predicate); + } catch (ParseException e) { + throw SparkExceptionUtil.toUncheckedException( + e, "Unable to parse the predicate expression: %s", predicate); + } + + Expression resolvedPredicateExpr = resolveAttrs(spark, table, unresolvedPredicateExpr); + return getPartitionsByFilter(spark, tableIdent, resolvedPredicateExpr); + } + + /** + * Returns partitions that match the specified 'predicate'. + * + * @param spark a Spark session + * @param tableIdent a table identifier + * @param predicateExpr a predicate expression on partition columns + * @return matching table's partitions + * @deprecated since 1.11.0, will be removed in 1.12.0 + */ + @Deprecated + public static List getPartitionsByFilter( + SparkSession spark, TableIdentifier tableIdent, Expression predicateExpr) { + try { + SessionCatalog catalog = spark.sessionState().catalog(); + CatalogTable catalogTable = catalog.getTableMetadata(tableIdent); + + Expression resolvedPredicateExpr; + if (!predicateExpr.resolved()) { + resolvedPredicateExpr = resolveAttrs(spark, tableIdent.quotedString(), predicateExpr); + } else { + resolvedPredicateExpr = predicateExpr; + } + Seq predicates = + JavaConverters.collectionAsScalaIterableConverter(ImmutableList.of(resolvedPredicateExpr)) + .asScala() + .toIndexedSeq(); + + Seq partitions = + catalog.listPartitionsByFilter(tableIdent, predicates).toIndexedSeq(); + + return JavaConverters.seqAsJavaListConverter(partitions).asJava().stream() + .map(catalogPartition -> toSparkPartition(catalogPartition, catalogTable)) + .collect(Collectors.toList()); + } catch (NoSuchDatabaseException e) { + throw SparkExceptionUtil.toUncheckedException( + e, "Unknown table: %s. Database not found in catalog.", tableIdent); + } catch (NoSuchTableException e) { + throw SparkExceptionUtil.toUncheckedException( + e, "Unknown table: %s. Table not found in catalog.", tableIdent); + } + } + + private static List listPartition( + SparkPartition partition, + PartitionSpec spec, + SerializableConfiguration conf, + MetricsConfig metricsConfig, + NameMapping mapping, + boolean ignoreMissingFiles, + ExecutorService service) { + try { + return TableMigrationUtil.listPartition( + partition.values, + partition.uri, + partition.format, + spec, + conf.get(), + metricsConfig, + mapping, + service); + } catch (RuntimeException e) { + if (ignoreMissingFiles && e.getCause() instanceof FileNotFoundException) { + LOG.warn("Ignoring FileNotFoundException when listing partition of {}", partition, e); + return Collections.emptyList(); + } else { + throw e; + } + } + } + + private static SparkPartition toSparkPartition( + CatalogTablePartition partition, CatalogTable table) { + Option locationUri = partition.storage().locationUri(); + Option serde = partition.storage().serde(); + + Preconditions.checkArgument(locationUri.nonEmpty(), "Partition URI should be defined"); + Preconditions.checkArgument( + serde.nonEmpty() || table.provider().nonEmpty(), "Partition format should be defined"); + + String uri = Util.uriToString(locationUri.get()); + String format = serde.nonEmpty() ? serde.get() : table.provider().get(); + + Map partitionSpec = + JavaConverters.mapAsJavaMapConverter(partition.spec()).asJava(); + return new SparkPartition(partitionSpec, uri, format); + } + + private static Expression resolveAttrs(SparkSession spark, String table, Expression expr) { + Function2 resolver = spark.sessionState().analyzer().resolver(); + LogicalPlan plan = spark.table(table).queryExecution().analyzed(); + return expr.transform( + new AbstractPartialFunction() { + @Override + public Expression apply(Expression attr) { + UnresolvedAttribute unresolvedAttribute = (UnresolvedAttribute) attr; + Option namedExpressionOption = + plan.resolve(unresolvedAttribute.nameParts(), resolver); + if (namedExpressionOption.isDefined()) { + return (Expression) namedExpressionOption.get(); + } else { + throw new IllegalArgumentException( + String.format("Could not resolve %s using columns: %s", attr, plan.output())); + } + } + + @Override + public boolean isDefinedAt(Expression attr) { + return attr instanceof UnresolvedAttribute; + } + }); + } + + private static Iterator buildManifest( + int formatVersion, + Long snapshotId, + SerializableConfiguration conf, + PartitionSpec spec, + String basePath, + Iterator> fileTuples) { + if (fileTuples.hasNext()) { + FileIO io = new HadoopFileIO(conf.get()); + TaskContext ctx = TaskContext.get(); + String suffix = + String.format( + Locale.ROOT, + "stage-%d-task-%d-manifest-%s", + ctx.stageId(), + ctx.taskAttemptId(), + UUID.randomUUID()); + Path location = new Path(basePath, suffix); + String outputPath = FileFormat.AVRO.addExtension(location.toString()); + OutputFile outputFile = io.newOutputFile(outputPath); + ManifestWriter writer = + ManifestFiles.write(formatVersion, spec, outputFile, snapshotId); + + try (ManifestWriter writerRef = writer) { + fileTuples.forEachRemaining(fileTuple -> writerRef.add(fileTuple._2)); + } catch (IOException e) { + throw SparkExceptionUtil.toUncheckedException( + e, "Unable to close the manifest writer: %s", outputPath); + } + + ManifestFile manifestFile = writer.toManifestFile(); + return ImmutableList.of(manifestFile).iterator(); + } else { + return Collections.emptyIterator(); + } + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param partitionFilter only import partitions whose values match those in the map, can be + * partially defined + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @deprecated since 1.11.0, will be removed in 1.12.0 + */ + @Deprecated + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + Map partitionFilter, + boolean checkDuplicateFiles) { + importSparkTable( + spark, sourceTableIdent, targetTable, stagingDir, partitionFilter, checkDuplicateFiles, 1); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param parallelism number of threads to use for file reading + * @deprecated since 1.11.0, will be removed in 1.12.0 + */ + @Deprecated + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + int parallelism) { + importSparkTable( + spark, + sourceTableIdent, + targetTable, + stagingDir, + TableMigrationUtil.migrationService(parallelism)); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param service executor service to use for file reading. If null, file reading will be + * performed on the current thread. * If non-null, the provided ExecutorService will be + * shutdown within this method after file reading is complete. + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + ExecutorService service) { + importSparkTable( + spark, sourceTableIdent, targetTable, stagingDir, Collections.emptyMap(), false, service); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param partitionFilter only import partitions whose values match those in the map, can be + * partially defined + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param parallelism number of threads to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + Map partitionFilter, + boolean checkDuplicateFiles, + int parallelism) { + importSparkTable( + spark, + sourceTableIdent, + targetTable, + stagingDir, + partitionFilter, + checkDuplicateFiles, + migrationService(parallelism)); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param partitionFilter only import partitions whose values match those in the map, can be + * partially defined + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param service executor service to use for file reading. If null, file reading will be + * performed on the current thread. If non-null, the provided ExecutorService will be shutdown + * within this method after file reading is complete. + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + Map partitionFilter, + boolean checkDuplicateFiles, + ExecutorService service) { + importSparkTable( + spark, + sourceTableIdent, + targetTable, + stagingDir, + partitionFilter, + checkDuplicateFiles, + false, + service); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param partitionFilter only import partitions whose values match those in the map, can be + * partially defined + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param ignoreMissingFiles if true, ignore {@link FileNotFoundException} when running {@link + * #listPartition} for the Spark partitions + * @param service executor service to use for file reading. If null, file reading will be + * performed on the current thread. If non-null, the provided ExecutorService will be shutdown + * within this method after file reading is complete. + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + Map partitionFilter, + boolean checkDuplicateFiles, + boolean ignoreMissingFiles, + ExecutorService service) { + SessionCatalog catalog = spark.sessionState().catalog(); + + String db = + sourceTableIdent.database().nonEmpty() + ? sourceTableIdent.database().get() + : catalog.getCurrentDatabase(); + TableIdentifier sourceTableIdentWithDB = + new TableIdentifier(sourceTableIdent.table(), Some.apply(db)); + + if (!catalog.tableExists(sourceTableIdentWithDB)) { + throw new org.apache.iceberg.exceptions.NoSuchTableException( + "Table %s does not exist", sourceTableIdentWithDB); + } + + try { + PartitionSpec spec = + findCompatibleSpec(targetTable, spark, sourceTableIdentWithDB.unquotedString()); + + validatePartitionFilter(spec, partitionFilter, targetTable.name()); + + if (Objects.equal(spec, PartitionSpec.unpartitioned())) { + importUnpartitionedSparkTable( + spark, sourceTableIdentWithDB, targetTable, checkDuplicateFiles, service); + } else { + List sourceTablePartitions = + getPartitions(spark, sourceTableIdent, partitionFilter); + if (sourceTablePartitions.isEmpty()) { + targetTable.newAppend().commit(); + } else { + importSparkPartitions( + spark, + sourceTablePartitions, + targetTable, + spec, + stagingDir, + checkDuplicateFiles, + ignoreMissingFiles, + service); + } + } + } catch (AnalysisException e) { + throw SparkExceptionUtil.toUncheckedException( + e, "Unable to get partition spec for table: %s", sourceTableIdentWithDB); + } + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @deprecated since 1.11.0, will be removed in 1.12.0 + */ + @Deprecated + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + boolean checkDuplicateFiles) { + importSparkTable( + spark, + sourceTableIdent, + targetTable, + stagingDir, + Collections.emptyMap(), + checkDuplicateFiles, + 1); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + */ + public static void importSparkTable( + SparkSession spark, TableIdentifier sourceTableIdent, Table targetTable, String stagingDir) { + importSparkTable( + spark, sourceTableIdent, targetTable, stagingDir, Collections.emptyMap(), false, 1); + } + + private static void importUnpartitionedSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + boolean checkDuplicateFiles, + ExecutorService service) { + try { + CatalogTable sourceTable = spark.sessionState().catalog().getTableMetadata(sourceTableIdent); + Option format = + sourceTable.storage().serde().nonEmpty() + ? sourceTable.storage().serde() + : sourceTable.provider(); + Preconditions.checkArgument(format.nonEmpty(), "Could not determine table format"); + + Map partition = Collections.emptyMap(); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Configuration conf = spark.sessionState().newHadoopConf(); + MetricsConfig metricsConfig = MetricsConfig.forTable(targetTable); + String nameMappingString = targetTable.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + NameMapping nameMapping = + nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; + + List files = + TableMigrationUtil.listPartition( + partition, + Util.uriToString(sourceTable.location()), + format.get(), + spec, + conf, + metricsConfig, + nameMapping, + service); + + if (checkDuplicateFiles) { + Dataset importedFiles = + spark + .createDataset(Lists.transform(files, ContentFile::location), Encoders.STRING()) + .toDF("file_path"); + Dataset existingFiles = + loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); + Column joinCond = + existingFiles.col("data_file.file_path").equalTo(importedFiles.col("file_path")); + Dataset duplicates = + importedFiles.join(existingFiles, joinCond).select("file_path").as(Encoders.STRING()); + Preconditions.checkState( + duplicates.isEmpty(), + String.format( + DUPLICATE_FILE_MESSAGE, Joiner.on(",").join((String[]) duplicates.take(10)))); + } + + AppendFiles append = targetTable.newAppend(); + files.forEach(append::appendFile); + append.commit(); + } catch (NoSuchDatabaseException e) { + throw SparkExceptionUtil.toUncheckedException( + e, "Unknown table: %s. Database not found in catalog.", sourceTableIdent); + } catch (NoSuchTableException e) { + throw SparkExceptionUtil.toUncheckedException( + e, "Unknown table: %s. Table not found in catalog.", sourceTableIdent); + } + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @deprecated since 1.11.0, will be removed in 1.12.0 + */ + @Deprecated + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir, + boolean checkDuplicateFiles) { + importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, checkDuplicateFiles, 1); + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param parallelism number of threads to use for file reading + */ + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir, + boolean checkDuplicateFiles, + int parallelism) { + importSparkPartitions( + spark, + partitions, + targetTable, + spec, + stagingDir, + checkDuplicateFiles, + migrationService(parallelism)); + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param service executor service to use for file reading. If null, file reading will be + * performed on the current thread. If non-null, the provided ExecutorService will be shutdown + * within this method after file reading is complete. + */ + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir, + boolean checkDuplicateFiles, + ExecutorService service) { + importSparkPartitions( + spark, partitions, targetTable, spec, stagingDir, checkDuplicateFiles, false, service); + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param ignoreMissingFiles if true, ignore {@link FileNotFoundException} when running {@link + * #listPartition} for the Spark partitions + * @param service executor service to use for file reading. If null, file reading will be + * performed on the current thread. If non-null, the provided ExecutorService will be shutdown + * within this method after file reading is complete. + */ + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir, + boolean checkDuplicateFiles, + boolean ignoreMissingFiles, + ExecutorService service) { + Configuration conf = spark.sessionState().newHadoopConf(); + SerializableConfiguration serializableConf = new SerializableConfiguration(conf); + int listingParallelism = + Math.min( + partitions.size(), spark.sessionState().conf().parallelPartitionDiscoveryParallelism()); + int numShufflePartitions = spark.sessionState().conf().numShufflePartitions(); + MetricsConfig metricsConfig = MetricsConfig.forTable(targetTable); + String nameMappingString = targetTable.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + NameMapping nameMapping = + nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; + + JavaSparkContext sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + JavaRDD partitionRDD = sparkContext.parallelize(partitions, listingParallelism); + + Dataset partitionDS = + spark.createDataset(partitionRDD.rdd(), Encoders.javaSerialization(SparkPartition.class)); + + Dataset filesToImport = + partitionDS.flatMap( + (FlatMapFunction) + sparkPartition -> + listPartition( + sparkPartition, + spec, + serializableConf, + metricsConfig, + nameMapping, + ignoreMissingFiles, + service) + .iterator(), + Encoders.javaSerialization(DataFile.class)); + + if (checkDuplicateFiles) { + Dataset importedFiles = + filesToImport + .map((MapFunction) ContentFile::location, Encoders.STRING()) + .toDF("file_path"); + Dataset existingFiles = + loadMetadataTable(spark, targetTable, MetadataTableType.ENTRIES).filter("status != 2"); + Column joinCond = + existingFiles.col("data_file.file_path").equalTo(importedFiles.col("file_path")); + Dataset duplicates = + importedFiles.join(existingFiles, joinCond).select("file_path").as(Encoders.STRING()); + Preconditions.checkState( + duplicates.isEmpty(), + String.format( + DUPLICATE_FILE_MESSAGE, Joiner.on(",").join((String[]) duplicates.take(10)))); + } + + TableOperations ops = ((HasTableOperations) targetTable).operations(); + int formatVersion = ops.current().formatVersion(); + boolean snapshotIdInheritanceEnabled = + PropertyUtil.propertyAsBoolean( + targetTable.properties(), + TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, + TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT); + + final Long snapshotId; + if (formatVersion == 1 && !snapshotIdInheritanceEnabled) { + snapshotId = -1L; + } else { + snapshotId = null; + } + + List manifests = + filesToImport + .repartition(numShufflePartitions) + .map( + (MapFunction>) + file -> Tuple2.apply(file.location(), file), + Encoders.tuple(Encoders.STRING(), Encoders.javaSerialization(DataFile.class))) + .orderBy(col("_1")) + .mapPartitions( + (MapPartitionsFunction, ManifestFile>) + fileTuple -> + buildManifest( + formatVersion, + snapshotId, + serializableConf, + spec, + stagingDir, + fileTuple), + Encoders.javaSerialization(ManifestFile.class)) + .collectAsList(); + + try { + + AppendFiles append = targetTable.newAppend(); + manifests.forEach(append::appendManifest); + append.commit(); + + if (formatVersion == 1 && !snapshotIdInheritanceEnabled) { + // delete original manifests as they were rewritten before the commit + deleteManifests(targetTable.io(), manifests); + } + } catch (Throwable e) { + deleteManifests(targetTable.io(), manifests); + throw e; + } + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @deprecated since 1.11.0, will be removed in 1.12.0 + */ + @Deprecated + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir) { + importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, false, 1); + } + + /** + * @deprecated since 1.11.0, will be removed in 1.12.0 + */ + @Deprecated + public static List filterPartitions( + List partitions, Map partitionFilter) { + if (partitionFilter.isEmpty()) { + return partitions; + } else { + return partitions.stream() + .filter(p -> p.getValues().entrySet().containsAll(partitionFilter.entrySet())) + .collect(Collectors.toList()); + } + } + + private static void deleteManifests(FileIO io, List manifests) { + if (io instanceof SupportsBulkOperations) { + ((SupportsBulkOperations) io).deleteFiles(Lists.transform(manifests, ManifestFile::path)); + } else { + Tasks.foreach(manifests) + .executeWith(ThreadPools.getWorkerPool()) + .noRetry() + .suppressFailureWhenFinished() + .run(item -> io.deleteFile(item.path())); + } + } + + public static Dataset loadTable(SparkSession spark, Table table, long snapshotId) { + SparkTable sparkTable = new SparkTable(table, snapshotId, false); + DataSourceV2Relation relation = createRelation(sparkTable, ImmutableMap.of()); + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + + return org.apache.spark.sql.classic.Dataset.ofRows( + (org.apache.spark.sql.classic.SparkSession) spark, relation); + } + + public static Dataset loadMetadataTable( + SparkSession spark, Table table, MetadataTableType type) { + return loadMetadataTable(spark, table, type, ImmutableMap.of()); + } + + public static Dataset loadMetadataTable( + SparkSession spark, Table table, MetadataTableType type, Map extraOptions) { + Table metadataTable = MetadataTableUtils.createMetadataTableInstance(table, type); + SparkTable sparkMetadataTable = new SparkTable(metadataTable, false); + DataSourceV2Relation relation = createRelation(sparkMetadataTable, extraOptions); + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + + return org.apache.spark.sql.classic.Dataset.ofRows( + (org.apache.spark.sql.classic.SparkSession) spark, relation); + } + + private static DataSourceV2Relation createRelation( + SparkTable sparkTable, Map extraOptions) { + CaseInsensitiveStringMap options = new CaseInsensitiveStringMap(extraOptions); + return DataSourceV2Relation.create(sparkTable, Option.empty(), Option.empty(), options); + } + + /** + * Determine the write branch. + * + *

    Validate wap config and determine the write branch. + * + * @param spark a Spark Session + * @param branch write branch if there is no WAP branch configured + * @return branch for write operation + */ + public static String determineWriteBranch(SparkSession spark, String branch) { + String wapId = spark.conf().get(SparkSQLProperties.WAP_ID, null); + String wapBranch = spark.conf().get(SparkSQLProperties.WAP_BRANCH, null); + ValidationException.check( + wapId == null || wapBranch == null, + "Cannot set both WAP ID and branch, but got ID [%s] and branch [%s]", + wapId, + wapBranch); + + if (wapBranch != null) { + ValidationException.check( + branch == null, + "Cannot write to both branch and WAP branch, but got branch [%s] and WAP branch [%s]", + branch, + wapBranch); + + return wapBranch; + } + return branch; + } + + public static boolean wapEnabled(Table table) { + return PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.WRITE_AUDIT_PUBLISH_ENABLED, + Boolean.parseBoolean(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED_DEFAULT)); + } + + /** Class representing a table partition. */ + public static class SparkPartition implements Serializable { + private final Map values; + private final String uri; + private final String format; + + public SparkPartition(Map values, String uri, String format) { + this.values = Maps.newHashMap(values); + this.uri = uri; + this.format = format; + } + + public Map getValues() { + return values; + } + + public String getUri() { + return uri; + } + + public String getFormat() { + return format; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("values", values) + .add("uri", uri) + .add("format", format) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SparkPartition that = (SparkPartition) o; + return Objects.equal(values, that.values) + && Objects.equal(uri, that.uri) + && Objects.equal(format, that.format); + } + + @Override + public int hashCode() { + return Objects.hashCode(values, uri, format); + } + } + + @Nullable + public static ExecutorService migrationService(int parallelism) { + return parallelism == 1 ? null : new LazyExecutorService(parallelism); + } + + private static class LazyExecutorService implements ExecutorService, Serializable { + + private final int parallelism; + private volatile ExecutorService service; + + LazyExecutorService(int parallelism) { + this.parallelism = parallelism; + } + + @Override + public void shutdown() { + getService().shutdown(); + } + + @Nonnull + @Override + public List shutdownNow() { + return getService().shutdownNow(); + } + + @Override + public boolean isShutdown() { + return getService().isShutdown(); + } + + @Override + public boolean isTerminated() { + return getService().isTerminated(); + } + + @Override + public boolean awaitTermination(long timeout, @Nonnull TimeUnit unit) + throws InterruptedException { + return getService().awaitTermination(timeout, unit); + } + + @Nonnull + @Override + public Future submit(@Nonnull Callable task) { + return getService().submit(task); + } + + @Nonnull + @Override + public Future submit(@Nonnull Runnable task, T result) { + return getService().submit(task, result); + } + + @Nonnull + @Override + public Future submit(@Nonnull Runnable task) { + return getService().submit(task); + } + + @Nonnull + @Override + public List> invokeAll(@Nonnull Collection> tasks) + throws InterruptedException { + return getService().invokeAll(tasks); + } + + @Nonnull + @Override + public List> invokeAll( + @Nonnull Collection> tasks, long timeout, @Nonnull TimeUnit unit) + throws InterruptedException { + return getService().invokeAll(tasks, timeout, unit); + } + + @Nonnull + @Override + public T invokeAny(@Nonnull Collection> tasks) + throws InterruptedException, ExecutionException { + return getService().invokeAny(tasks); + } + + @Override + public T invokeAny( + @Nonnull Collection> tasks, long timeout, @Nonnull TimeUnit unit) + throws InterruptedException, ExecutionException, TimeoutException { + return getService().invokeAny(tasks, timeout, unit); + } + + @Override + public void execute(@Nonnull Runnable command) { + getService().execute(command); + } + + private ExecutorService getService() { + if (service == null) { + synchronized (this) { + if (service == null) { + service = TableMigrationUtil.migrationService(parallelism); + } + } + } + return service; + } + } + + /** + * Returns the first partition spec in an IcebergTable that shares the same names and ordering as + * the partition columns provided. Throws an error if not found + */ + public static PartitionSpec findCompatibleSpec(List partitionNames, Table icebergTable) { + List partitionNamesLower = + partitionNames.stream() + .map(name -> name.toLowerCase(Locale.ROOT)) + .collect(Collectors.toList()); + for (PartitionSpec icebergSpec : icebergTable.specs().values()) { + boolean allIdentity = + icebergSpec.fields().stream().allMatch(field -> field.transform().isIdentity()); + if (allIdentity) { + List icebergPartNames = + icebergSpec.fields().stream() + .map(PartitionField::name) + .map(name -> name.toLowerCase(Locale.ROOT)) + .collect(Collectors.toList()); + if (icebergPartNames.equals(partitionNamesLower)) { + return icebergSpec; + } + } + } + + throw new IllegalArgumentException( + String.format( + "Cannot find a partition spec in Iceberg table %s that matches the partition" + + " columns (%s) in input table", + icebergTable, partitionNames)); + } + + /** + * Returns the first partition spec in an IcebergTable that shares the same names and ordering as + * the partition columns in a given Spark Table. Throws an error if not found + */ + private static PartitionSpec findCompatibleSpec( + Table icebergTable, SparkSession spark, String sparkTable) throws AnalysisException { + List parts = Lists.newArrayList(Splitter.on('.').limit(2).split(sparkTable)); + String db = parts.size() == 1 ? "default" : parts.get(0); + String table = parts.get(parts.size() == 1 ? 0 : 1); + + List sparkPartNames = + spark.catalog().listColumns(db, table).collectAsList().stream() + .filter(org.apache.spark.sql.catalog.Column::isPartition) + .map(org.apache.spark.sql.catalog.Column::name) + .collect(Collectors.toList()); + return findCompatibleSpec(sparkPartNames, icebergTable); + } + + public static void validatePartitionFilter( + PartitionSpec spec, Map partitionFilter, String tableName) { + List partitionFields = spec.fields(); + Set partitionNames = + spec.fields().stream().map(PartitionField::name).collect(Collectors.toSet()); + + boolean tablePartitioned = !partitionFields.isEmpty(); + boolean partitionFilterPassed = !partitionFilter.isEmpty(); + + if (tablePartitioned && partitionFilterPassed) { + // Check to see there are sufficient partition columns to satisfy the filter + Preconditions.checkArgument( + partitionFields.size() >= partitionFilter.size(), + "Cannot add data files to target table %s because that table is partitioned, " + + "but the number of columns in the provided partition filter (%s) " + + "is greater than the number of partitioned columns in table (%s)", + tableName, + partitionFilter.size(), + partitionFields.size()); + + // Check for any filters of non-existent columns + List unMatchedFilters = + partitionFilter.keySet().stream() + .filter(filterName -> !partitionNames.contains(filterName)) + .collect(Collectors.toList()); + Preconditions.checkArgument( + unMatchedFilters.isEmpty(), + "Cannot add files to target table %s. %s is partitioned but the specified partition filter " + + "refers to columns that are not partitioned: %s . Valid partition columns: [%s]", + tableName, + tableName, + unMatchedFilters, + String.join(",", partitionNames)); + } else { + Preconditions.checkArgument( + !partitionFilterPassed, + "Cannot use partition filter with an unpartitioned table %s", + tableName); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java new file mode 100644 index 000000000000..54ad899ade77 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTypeToType.java @@ -0,0 +1,171 @@ +/* + * 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.List; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.BinaryType; +import org.apache.spark.sql.types.BooleanType; +import org.apache.spark.sql.types.ByteType; +import org.apache.spark.sql.types.CharType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DateType; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.DoubleType; +import org.apache.spark.sql.types.FloatType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.types.LongType; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.NullType; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.TimestampNTZType; +import org.apache.spark.sql.types.TimestampType; +import org.apache.spark.sql.types.VarcharType; +import org.apache.spark.sql.types.VariantType; + +class SparkTypeToType extends SparkTypeVisitor { + private final StructType root; + private int nextId = 0; + + SparkTypeToType() { + this.root = null; + } + + SparkTypeToType(StructType root) { + this.root = root; + // the root struct's fields use the first ids + this.nextId = root.fields().length; + } + + private int getNextId() { + int next = nextId; + nextId += 1; + return next; + } + + @Override + @SuppressWarnings("ReferenceEquality") + public Type struct(StructType struct, List types) { + StructField[] fields = struct.fields(); + List newFields = Lists.newArrayListWithExpectedSize(fields.length); + boolean isRoot = root == struct; + for (int i = 0; i < fields.length; i += 1) { + StructField field = fields[i]; + Type type = types.get(i); + + int id; + if (isRoot) { + // for new conversions, use ordinals for ids in the root struct + id = i; + } else { + id = getNextId(); + } + + String doc = field.getComment().isDefined() ? field.getComment().get() : null; + + if (field.nullable()) { + newFields.add(Types.NestedField.optional(id, field.name(), type, doc)); + } else { + newFields.add(Types.NestedField.required(id, field.name(), type, doc)); + } + } + + return Types.StructType.of(newFields); + } + + @Override + public Type field(StructField field, Type typeResult) { + return typeResult; + } + + @Override + public Type array(ArrayType array, Type elementType) { + if (array.containsNull()) { + return Types.ListType.ofOptional(getNextId(), elementType); + } else { + return Types.ListType.ofRequired(getNextId(), elementType); + } + } + + @Override + public Type map(MapType map, Type keyType, Type valueType) { + if (map.valueContainsNull()) { + return Types.MapType.ofOptional(getNextId(), getNextId(), keyType, valueType); + } else { + return Types.MapType.ofRequired(getNextId(), getNextId(), keyType, valueType); + } + } + + @Override + public Type variant(VariantType variant) { + return Types.VariantType.get(); + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + @Override + public Type atomic(DataType atomic) { + if (atomic instanceof BooleanType) { + return Types.BooleanType.get(); + + } else if (atomic instanceof IntegerType + || atomic instanceof ShortType + || atomic instanceof ByteType) { + return Types.IntegerType.get(); + + } else if (atomic instanceof LongType) { + return Types.LongType.get(); + + } else if (atomic instanceof FloatType) { + return Types.FloatType.get(); + + } else if (atomic instanceof DoubleType) { + return Types.DoubleType.get(); + + } else if (atomic instanceof StringType + || atomic instanceof CharType + || atomic instanceof VarcharType) { + return Types.StringType.get(); + + } else if (atomic instanceof DateType) { + return Types.DateType.get(); + + } else if (atomic instanceof TimestampType) { + return Types.TimestampType.withZone(); + + } else if (atomic instanceof TimestampNTZType) { + return Types.TimestampType.withoutZone(); + + } else if (atomic instanceof DecimalType) { + return Types.DecimalType.of( + ((DecimalType) atomic).precision(), ((DecimalType) atomic).scale()); + } else if (atomic instanceof BinaryType) { + return Types.BinaryType.get(); + } else if (atomic instanceof NullType) { + return Types.UnknownType.get(); + } + + throw new UnsupportedOperationException("Not a supported type: " + atomic.catalogString()); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java new file mode 100644 index 000000000000..7b35f6183559 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkTypeVisitor.java @@ -0,0 +1,86 @@ +/* + * 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.List; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.UserDefinedType; +import org.apache.spark.sql.types.VariantType; + +class SparkTypeVisitor { + static T visit(DataType type, SparkTypeVisitor visitor) { + if (type instanceof StructType) { + StructField[] fields = ((StructType) type).fields(); + List fieldResults = Lists.newArrayListWithExpectedSize(fields.length); + + for (StructField field : fields) { + fieldResults.add(visitor.field(field, visit(field.dataType(), visitor))); + } + + return visitor.struct((StructType) type, fieldResults); + + } else if (type instanceof MapType) { + return visitor.map( + (MapType) type, + visit(((MapType) type).keyType(), visitor), + visit(((MapType) type).valueType(), visitor)); + + } else if (type instanceof ArrayType) { + return visitor.array((ArrayType) type, visit(((ArrayType) type).elementType(), visitor)); + + } else if (type instanceof VariantType) { + return visitor.variant((VariantType) type); + + } else if (type instanceof UserDefinedType) { + throw new UnsupportedOperationException("User-defined types are not supported"); + + } else { + return visitor.atomic(type); + } + } + + public T variant(VariantType variant) { + throw new UnsupportedOperationException("Not implemented for variant"); + } + + public T struct(StructType struct, List fieldResults) { + return null; + } + + public T field(StructField field, T typeResult) { + return null; + } + + public T array(ArrayType array, T elementResult) { + return null; + } + + public T map(MapType map, T keyResult, T valueResult) { + return null; + } + + public T atomic(DataType atomic) { + return null; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java new file mode 100644 index 000000000000..ef9990c7bd58 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkUtil.java @@ -0,0 +1,339 @@ +/* + * 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.math.BigDecimal; +import java.nio.ByteBuffer; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.stream.Collectors; +import org.apache.avro.generic.GenericData; +import org.apache.avro.util.Utf8; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.transforms.Transform; +import org.apache.iceberg.transforms.UnknownTransform; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.util.Pair; +import org.apache.spark.SparkEnv; +import org.apache.spark.scheduler.ExecutorCacheTaskLocation; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.expressions.BoundReference; +import org.apache.spark.sql.catalyst.expressions.EqualTo; +import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.expressions.Literal; +import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.storage.BlockManager; +import org.apache.spark.storage.BlockManagerId; +import org.apache.spark.storage.BlockManagerMaster; +import org.apache.spark.unsafe.types.UTF8String; +import org.joda.time.DateTime; +import scala.collection.JavaConverters; +import scala.collection.Seq; + +public class SparkUtil { + private static final String SPARK_CATALOG_CONF_PREFIX = "spark.sql.catalog"; + // Format string used as the prefix for Spark configuration keys to override Hadoop configuration + // values for Iceberg tables from a given catalog. These keys can be specified as + // `spark.sql.catalog.$catalogName.hadoop.*`, similar to using `spark.hadoop.*` to override + // Hadoop configurations globally for a given Spark session. + private static final String SPARK_CATALOG_HADOOP_CONF_OVERRIDE_FMT_STR = + SPARK_CATALOG_CONF_PREFIX + ".%s.hadoop."; + + private static final Joiner DOT = Joiner.on("."); + + private SparkUtil() {} + + /** + * Check whether the partition transforms in a spec can be used to write data. + * + * @param spec a PartitionSpec + * @throws UnsupportedOperationException if the spec contains unknown partition transforms + */ + public static void validatePartitionTransforms(PartitionSpec spec) { + if (spec.fields().stream().anyMatch(field -> field.transform() instanceof UnknownTransform)) { + String unsupported = + spec.fields().stream() + .map(PartitionField::transform) + .filter(transform -> transform instanceof UnknownTransform) + .map(Transform::toString) + .collect(Collectors.joining(", ")); + + throw new UnsupportedOperationException( + String.format("Cannot write using unsupported transforms: %s", unsupported)); + } + } + + /** + * A modified version of Spark's LookupCatalog.CatalogAndIdentifier.unapply Attempts to find the + * catalog and identifier a multipart identifier represents + * + * @param nameParts Multipart identifier representing a table + * @return The CatalogPlugin and Identifier for the table + */ + public static Pair catalogAndIdentifier( + List nameParts, + Function catalogProvider, + BiFunction identiferProvider, + C currentCatalog, + String[] currentNamespace) { + Preconditions.checkArgument( + !nameParts.isEmpty(), "Cannot determine catalog and identifier from empty name"); + + int lastElementIndex = nameParts.size() - 1; + String name = nameParts.get(lastElementIndex); + + if (nameParts.size() == 1) { + // Only a single element, use current catalog and namespace + return Pair.of(currentCatalog, identiferProvider.apply(currentNamespace, name)); + } else { + C catalog = catalogProvider.apply(nameParts.get(0)); + if (catalog == null) { + // The first element was not a valid catalog, treat it like part of the namespace + String[] namespace = nameParts.subList(0, lastElementIndex).toArray(new String[0]); + return Pair.of(currentCatalog, identiferProvider.apply(namespace, name)); + } else { + // Assume the first element is a valid catalog + String[] namespace = nameParts.subList(1, lastElementIndex).toArray(new String[0]); + return Pair.of(catalog, identiferProvider.apply(namespace, name)); + } + } + } + + /** + * Pulls any Catalog specific overrides for the Hadoop conf from the current SparkSession, which + * can be set via `spark.sql.catalog.$catalogName.hadoop.*` + * + *

    Mirrors the override of hadoop configurations for a given spark session using + * `spark.hadoop.*`. + * + *

    The SparkCatalog allows for hadoop configurations to be overridden per catalog, by setting + * them on the SQLConf, where the following will add the property "fs.default.name" with value + * "hdfs://hanksnamenode:8020" to the catalog's hadoop configuration. SparkSession.builder() + * .config(s"spark.sql.catalog.$catalogName.hadoop.fs.default.name", "hdfs://hanksnamenode:8020") + * .getOrCreate() + * + * @param spark The current Spark session + * @param catalogName Name of the catalog to find overrides for. + * @return the Hadoop Configuration that should be used for this catalog, with catalog specific + * overrides applied. + */ + public static Configuration hadoopConfCatalogOverrides(SparkSession spark, String catalogName) { + // Find keys for the catalog intended to be hadoop configurations + final String hadoopConfCatalogPrefix = hadoopConfPrefixForCatalog(catalogName); + final Configuration conf = spark.sessionState().newHadoopConf(); + + spark + .sessionState() + .conf() + .settings() + .forEach( + (k, v) -> { + // these checks are copied from `spark.sessionState().newHadoopConfWithOptions()` + // to avoid converting back and forth between Scala / Java map types + if (v != null && k != null && k.startsWith(hadoopConfCatalogPrefix)) { + conf.set(k.substring(hadoopConfCatalogPrefix.length()), v); + } + }); + return conf; + } + + private static String hadoopConfPrefixForCatalog(String catalogName) { + return String.format(SPARK_CATALOG_HADOOP_CONF_OVERRIDE_FMT_STR, catalogName); + } + + /** + * Get a List of Spark filter Expression. + * + * @param schema table schema + * @param filters filters in the format of a Map, where key is one of the table column name, and + * value is the specific value to be filtered on the column. + * @return a List of filters in the format of Spark Expression. + */ + public static List partitionMapToExpression( + StructType schema, Map filters) { + List filterExpressions = Lists.newArrayList(); + for (Map.Entry entry : filters.entrySet()) { + try { + int index = schema.fieldIndex(entry.getKey()); + DataType dataType = schema.fields()[index].dataType(); + BoundReference ref = new BoundReference(index, dataType, true); + switch (dataType.typeName()) { + case "integer": + filterExpressions.add( + new EqualTo( + ref, + Literal.create(Integer.parseInt(entry.getValue()), DataTypes.IntegerType))); + break; + case "string": + filterExpressions.add( + new EqualTo(ref, Literal.create(entry.getValue(), DataTypes.StringType))); + break; + case "short": + filterExpressions.add( + new EqualTo( + ref, Literal.create(Short.parseShort(entry.getValue()), DataTypes.ShortType))); + break; + case "long": + filterExpressions.add( + new EqualTo( + ref, Literal.create(Long.parseLong(entry.getValue()), DataTypes.LongType))); + break; + case "float": + filterExpressions.add( + new EqualTo( + ref, Literal.create(Float.parseFloat(entry.getValue()), DataTypes.FloatType))); + break; + case "double": + filterExpressions.add( + new EqualTo( + ref, + Literal.create(Double.parseDouble(entry.getValue()), DataTypes.DoubleType))); + break; + case "date": + filterExpressions.add( + new EqualTo( + ref, + Literal.create( + new Date(DateTime.parse(entry.getValue()).getMillis()), + DataTypes.DateType))); + break; + case "timestamp": + filterExpressions.add( + new EqualTo( + ref, + Literal.create( + new Timestamp(DateTime.parse(entry.getValue()).getMillis()), + DataTypes.TimestampType))); + break; + default: + throw new IllegalStateException( + "Unexpected data type in partition filters: " + dataType); + } + } catch (IllegalArgumentException e) { + // ignore if filter is not on table columns + } + } + + return filterExpressions; + } + + public static String toColumnName(NamedReference ref) { + return DOT.join(ref.fieldNames()); + } + + public static boolean caseSensitive(SparkSession spark) { + return Boolean.parseBoolean(spark.conf().get("spark.sql.caseSensitive")); + } + + public static List executorLocations() { + BlockManager driverBlockManager = SparkEnv.get().blockManager(); + List executorBlockManagerIds = fetchPeers(driverBlockManager); + return executorBlockManagerIds.stream() + .map(SparkUtil::toExecutorLocation) + .sorted() + .collect(Collectors.toList()); + } + + private static List fetchPeers(BlockManager blockManager) { + BlockManagerMaster master = blockManager.master(); + BlockManagerId id = blockManager.blockManagerId(); + return toJavaList(master.getPeers(id)); + } + + private static List toJavaList(Seq seq) { + return JavaConverters.seqAsJavaListConverter(seq).asJava(); + } + + private static String toExecutorLocation(BlockManagerId id) { + return ExecutorCacheTaskLocation.apply(id.host(), id.executorId()).toString(); + } + + /** + * Converts a value to pass into Spark from Iceberg's internal object model. + * + * @param type an Iceberg type + * @param value a value that is an instance of {@link Type.TypeID#javaClass()} + * @return the value converted for Spark + */ + public static Object internalToSpark(Type type, Object value) { + if (value == null) { + return null; + } + + switch (type.typeId()) { + case DECIMAL: + return Decimal.apply((BigDecimal) value); + case UUID: + case STRING: + if (value instanceof Utf8) { + Utf8 utf8 = (Utf8) value; + return UTF8String.fromBytes(utf8.getBytes(), 0, utf8.getByteLength()); + } + return UTF8String.fromString(value.toString()); + case FIXED: + if (value instanceof byte[]) { + return value; + } else if (value instanceof GenericData.Fixed) { + return ((GenericData.Fixed) value).bytes(); + } + return ByteBuffers.toByteArray((ByteBuffer) value); + case BINARY: + return ByteBuffers.toByteArray((ByteBuffer) value); + case STRUCT: + Types.StructType structType = (Types.StructType) type; + + if (structType.fields().isEmpty()) { + return new GenericInternalRow(); + } + + List fields = structType.fields(); + Object[] values = new Object[fields.size()]; + StructLike struct = (StructLike) value; + + for (int index = 0; index < fields.size(); index++) { + Types.NestedField field = fields.get(index); + Type fieldType = field.type(); + values[index] = + internalToSpark(fieldType, struct.get(index, fieldType.typeId().javaClass())); + } + + return new GenericInternalRow(values); + default: + } + + return value; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java new file mode 100644 index 000000000000..57b9d61e38bd --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkV2Filters.java @@ -0,0 +1,484 @@ +/* + * 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 static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.bucket; +import static org.apache.iceberg.expressions.Expressions.day; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.hour; +import static org.apache.iceberg.expressions.Expressions.in; +import static org.apache.iceberg.expressions.Expressions.isNaN; +import static org.apache.iceberg.expressions.Expressions.isNull; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.month; +import static org.apache.iceberg.expressions.Expressions.not; +import static org.apache.iceberg.expressions.Expressions.notEqual; +import static org.apache.iceberg.expressions.Expressions.notIn; +import static org.apache.iceberg.expressions.Expressions.notNaN; +import static org.apache.iceberg.expressions.Expressions.notNull; +import static org.apache.iceberg.expressions.Expressions.or; +import static org.apache.iceberg.expressions.Expressions.startsWith; +import static org.apache.iceberg.expressions.Expressions.truncate; +import static org.apache.iceberg.expressions.Expressions.year; + +import java.util.Arrays; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.expressions.UnboundTerm; +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.util.NaNUtil; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.connector.expressions.Literal; +import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.expressions.UserDefinedScalarFunc; +import org.apache.spark.sql.connector.expressions.filter.And; +import org.apache.spark.sql.connector.expressions.filter.Not; +import org.apache.spark.sql.connector.expressions.filter.Or; +import org.apache.spark.sql.connector.expressions.filter.Predicate; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.types.UTF8String; + +public class SparkV2Filters { + + public static final Set SUPPORTED_FUNCTIONS = + ImmutableSet.of("years", "months", "days", "hours", "bucket", "truncate"); + + private static final String TRUE = "ALWAYS_TRUE"; + private static final String FALSE = "ALWAYS_FALSE"; + private static final String EQ = "="; + private static final String EQ_NULL_SAFE = "<=>"; + private static final String NOT_EQ = "<>"; + private static final String GT = ">"; + private static final String GT_EQ = ">="; + private static final String LT = "<"; + private static final String LT_EQ = "<="; + private static final String IN = "IN"; + private static final String IS_NULL = "IS_NULL"; + private static final String NOT_NULL = "IS_NOT_NULL"; + private static final String AND = "AND"; + private static final String OR = "OR"; + private static final String NOT = "NOT"; + private static final String STARTS_WITH = "STARTS_WITH"; + + private static final Map FILTERS = + ImmutableMap.builder() + .put(TRUE, Operation.TRUE) + .put(FALSE, Operation.FALSE) + .put(EQ, Operation.EQ) + .put(EQ_NULL_SAFE, Operation.EQ) + .put(NOT_EQ, Operation.NOT_EQ) + .put(GT, Operation.GT) + .put(GT_EQ, Operation.GT_EQ) + .put(LT, Operation.LT) + .put(LT_EQ, Operation.LT_EQ) + .put(IN, Operation.IN) + .put(IS_NULL, Operation.IS_NULL) + .put(NOT_NULL, Operation.NOT_NULL) + .put(AND, Operation.AND) + .put(OR, Operation.OR) + .put(NOT, Operation.NOT) + .put(STARTS_WITH, Operation.STARTS_WITH) + .buildOrThrow(); + + private SparkV2Filters() {} + + public static Expression convert(Predicate[] predicates) { + Expression expression = Expressions.alwaysTrue(); + for (Predicate predicate : predicates) { + Expression converted = convert(predicate); + Preconditions.checkArgument( + converted != null, "Cannot convert Spark predicate to Iceberg expression: %s", predicate); + expression = Expressions.and(expression, converted); + } + + return expression; + } + + @SuppressWarnings({"checkstyle:CyclomaticComplexity", "checkstyle:MethodLength"}) + public static Expression convert(Predicate predicate) { + Operation op = FILTERS.get(predicate.name()); + if (op != null) { + switch (op) { + case TRUE: + return Expressions.alwaysTrue(); + + case FALSE: + return Expressions.alwaysFalse(); + + case IS_NULL: + if (canConvertToTerm(child(predicate))) { + UnboundTerm term = toTerm(child(predicate)); + return term != null ? isNull(term) : null; + } + + return null; + + case NOT_NULL: + if (canConvertToTerm(child(predicate))) { + UnboundTerm term = toTerm(child(predicate)); + return term != null ? notNull(term) : null; + } + + return null; + + case LT: + if (canConvertToTerm(leftChild(predicate)) && isLiteral(rightChild(predicate))) { + UnboundTerm term = toTerm(leftChild(predicate)); + return term != null ? lessThan(term, convertLiteral(rightChild(predicate))) : null; + } else if (canConvertToTerm(rightChild(predicate)) && isLiteral(leftChild(predicate))) { + UnboundTerm term = toTerm(rightChild(predicate)); + return term != null ? greaterThan(term, convertLiteral(leftChild(predicate))) : null; + } else { + return null; + } + + case LT_EQ: + if (canConvertToTerm(leftChild(predicate)) && isLiteral(rightChild(predicate))) { + UnboundTerm term = toTerm(leftChild(predicate)); + return term != null + ? lessThanOrEqual(term, convertLiteral(rightChild(predicate))) + : null; + } else if (canConvertToTerm(rightChild(predicate)) && isLiteral(leftChild(predicate))) { + UnboundTerm term = toTerm(rightChild(predicate)); + return term != null + ? greaterThanOrEqual(term, convertLiteral(leftChild(predicate))) + : null; + } else { + return null; + } + + case GT: + if (canConvertToTerm(leftChild(predicate)) && isLiteral(rightChild(predicate))) { + UnboundTerm term = toTerm(leftChild(predicate)); + return term != null ? greaterThan(term, convertLiteral(rightChild(predicate))) : null; + } else if (canConvertToTerm(rightChild(predicate)) && isLiteral(leftChild(predicate))) { + UnboundTerm term = toTerm(rightChild(predicate)); + return term != null ? lessThan(term, convertLiteral(leftChild(predicate))) : null; + } else { + return null; + } + + case GT_EQ: + if (canConvertToTerm(leftChild(predicate)) && isLiteral(rightChild(predicate))) { + UnboundTerm term = toTerm(leftChild(predicate)); + return term != null + ? greaterThanOrEqual(term, convertLiteral(rightChild(predicate))) + : null; + } else if (canConvertToTerm(rightChild(predicate)) && isLiteral(leftChild(predicate))) { + UnboundTerm term = toTerm(rightChild(predicate)); + return term != null + ? lessThanOrEqual(term, convertLiteral(leftChild(predicate))) + : null; + } else { + return null; + } + + case EQ: // used for both eq and null-safe-eq + Pair, Object> eqChildren = predicateChildren(predicate); + if (eqChildren == null) { + return null; + } + + if (predicate.name().equals(EQ)) { + // comparison with null in normal equality is always null. this is probably a mistake. + Preconditions.checkNotNull( + eqChildren.second(), + "Expression is always false (eq is not null-safe): %s", + predicate); + } + + return handleEqual(eqChildren.first(), eqChildren.second()); + + case NOT_EQ: + Pair, Object> notEqChildren = predicateChildren(predicate); + if (notEqChildren == null) { + return null; + } + + // comparison with null in normal equality is always null. this is probably a mistake. + Preconditions.checkNotNull( + notEqChildren.second(), + "Expression is always false (notEq is not null-safe): %s", + predicate); + + return handleNotEqual(notEqChildren.first(), notEqChildren.second()); + + case IN: + if (isSupportedInPredicate(predicate)) { + UnboundTerm term = toTerm(childAtIndex(predicate, 0)); + + return term != null + ? in( + term, + Arrays.stream(predicate.children()) + .skip(1) + .map(val -> convertLiteral(((Literal) val))) + .filter(Objects::nonNull) + .collect(Collectors.toList())) + : null; + } else { + return null; + } + + case NOT: + Not notPredicate = (Not) predicate; + Predicate childPredicate = notPredicate.child(); + if (childPredicate.name().equals(IN) && isSupportedInPredicate(childPredicate)) { + UnboundTerm term = toTerm(childAtIndex(childPredicate, 0)); + if (term == null) { + return null; + } + + // infer an extra notNull predicate for Spark NOT IN filters + // as Iceberg expressions don't follow the 3-value SQL boolean logic + // col NOT IN (1, 2) in Spark is equal to notNull(col) && notIn(col, 1, 2) in Iceberg + Expression notIn = + notIn( + term, + Arrays.stream(childPredicate.children()) + .skip(1) + .map(val -> convertLiteral(((Literal) val))) + .filter(Objects::nonNull) + .collect(Collectors.toList())); + return and(notNull(term), notIn); + } else if (hasNoInFilter(childPredicate)) { + Expression child = convert(childPredicate); + if (child != null) { + return not(child); + } + } + return null; + + case AND: + { + And andPredicate = (And) predicate; + Expression left = convert(andPredicate.left()); + Expression right = convert(andPredicate.right()); + if (left != null && right != null) { + return and(left, right); + } + return null; + } + + case OR: + { + Or orPredicate = (Or) predicate; + Expression left = convert(orPredicate.left()); + Expression right = convert(orPredicate.right()); + if (left != null && right != null) { + return or(left, right); + } + return null; + } + + case STARTS_WITH: + String colName = SparkUtil.toColumnName(leftChild(predicate)); + return startsWith(colName, convertLiteral(rightChild(predicate)).toString()); + } + } + + return null; + } + + private static Pair, Object> predicateChildren(Predicate predicate) { + if (canConvertToTerm(leftChild(predicate)) && isLiteral(rightChild(predicate))) { + UnboundTerm term = toTerm(leftChild(predicate)); + return term != null ? Pair.of(term, convertLiteral(rightChild(predicate))) : null; + + } else if (canConvertToTerm(rightChild(predicate)) && isLiteral(leftChild(predicate))) { + UnboundTerm term = toTerm(rightChild(predicate)); + return term != null ? Pair.of(term, convertLiteral(leftChild(predicate))) : null; + + } else { + return null; + } + } + + @SuppressWarnings("unchecked") + private static T child(Predicate predicate) { + org.apache.spark.sql.connector.expressions.Expression[] children = predicate.children(); + Preconditions.checkArgument( + children.length == 1, "Predicate should have one child: %s", predicate); + return (T) children[0]; + } + + @SuppressWarnings("unchecked") + private static T leftChild(Predicate predicate) { + org.apache.spark.sql.connector.expressions.Expression[] children = predicate.children(); + Preconditions.checkArgument( + children.length == 2, "Predicate should have two children: %s", predicate); + return (T) children[0]; + } + + @SuppressWarnings("unchecked") + private static T rightChild(Predicate predicate) { + org.apache.spark.sql.connector.expressions.Expression[] children = predicate.children(); + Preconditions.checkArgument( + children.length == 2, "Predicate should have two children: %s", predicate); + return (T) children[1]; + } + + @SuppressWarnings("unchecked") + private static T childAtIndex(Predicate predicate, int index) { + return (T) predicate.children()[index]; + } + + private static boolean canConvertToTerm( + org.apache.spark.sql.connector.expressions.Expression expr) { + return isRef(expr) || isSystemFunc(expr); + } + + private static boolean isRef(org.apache.spark.sql.connector.expressions.Expression expr) { + return expr instanceof NamedReference; + } + + private static boolean isSystemFunc(org.apache.spark.sql.connector.expressions.Expression expr) { + if (expr instanceof UserDefinedScalarFunc) { + UserDefinedScalarFunc udf = (UserDefinedScalarFunc) expr; + return udf.canonicalName().startsWith("iceberg") + && SUPPORTED_FUNCTIONS.contains(udf.name()) + && Arrays.stream(udf.children()).allMatch(child -> isLiteral(child) || isRef(child)); + } + + return false; + } + + private static boolean isLiteral(org.apache.spark.sql.connector.expressions.Expression expr) { + return expr instanceof Literal; + } + + private static Object convertLiteral(Literal literal) { + if (literal.value() instanceof UTF8String) { + return ((UTF8String) literal.value()).toString(); + } else if (literal.value() instanceof Decimal) { + return ((Decimal) literal.value()).toJavaBigDecimal(); + } + return literal.value(); + } + + private static UnboundPredicate handleEqual(UnboundTerm term, Object value) { + if (value == null) { + return isNull(term); + } else if (NaNUtil.isNaN(value)) { + return isNaN(term); + } else { + return equal(term, value); + } + } + + private static UnboundPredicate handleNotEqual(UnboundTerm term, Object value) { + if (NaNUtil.isNaN(value)) { + return notNaN(term); + } else { + return notEqual(term, value); + } + } + + private static boolean hasNoInFilter(Predicate predicate) { + Operation op = FILTERS.get(predicate.name()); + + if (op != null) { + switch (op) { + case AND: + And andPredicate = (And) predicate; + return hasNoInFilter(andPredicate.left()) && hasNoInFilter(andPredicate.right()); + case OR: + Or orPredicate = (Or) predicate; + return hasNoInFilter(orPredicate.left()) && hasNoInFilter(orPredicate.right()); + case NOT: + Not notPredicate = (Not) predicate; + return hasNoInFilter(notPredicate.child()); + case IN: + return false; + default: + return true; + } + } + + return false; + } + + private static boolean isSupportedInPredicate(Predicate predicate) { + if (!canConvertToTerm(childAtIndex(predicate, 0))) { + return false; + } else { + return Arrays.stream(predicate.children()).skip(1).allMatch(SparkV2Filters::isLiteral); + } + } + + /** Should be called after {@link #canConvertToTerm} passed */ + private static UnboundTerm toTerm(T input) { + if (input instanceof NamedReference) { + return Expressions.ref(SparkUtil.toColumnName((NamedReference) input)); + } else if (input instanceof UserDefinedScalarFunc) { + return udfToTerm((UserDefinedScalarFunc) input); + } else { + return null; + } + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + private static UnboundTerm udfToTerm(UserDefinedScalarFunc udf) { + org.apache.spark.sql.connector.expressions.Expression[] children = udf.children(); + String udfName = udf.name().toLowerCase(Locale.ROOT); + if (children.length == 1) { + org.apache.spark.sql.connector.expressions.Expression child = children[0]; + if (isRef(child)) { + String column = SparkUtil.toColumnName((NamedReference) child); + switch (udfName) { + case "years": + return year(column); + case "months": + return month(column); + case "days": + return day(column); + case "hours": + return hour(column); + } + } + } else if (children.length == 2) { + if (isLiteral(children[0]) && isRef(children[1])) { + String column = SparkUtil.toColumnName((NamedReference) children[1]); + switch (udfName) { + case "bucket": + int numBuckets = (Integer) convertLiteral((Literal) children[0]); + return bucket(column, numBuckets); + case "truncate": + int width = (Integer) convertLiteral((Literal) children[0]); + return truncate(column, width); + } + } + } + + return null; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java new file mode 100644 index 000000000000..28b717ac090e --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkValueConverter.java @@ -0,0 +1,157 @@ +/* + * 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.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.util.DateTimeUtils; + +/** A utility class that converts Spark values to Iceberg's internal representation. */ +public class SparkValueConverter { + + private SparkValueConverter() {} + + public static Record convert(Schema schema, Row row) { + return convert(schema.asStruct(), row); + } + + public static Object convert(Type type, Object object) { + if (object == null) { + return null; + } + + switch (type.typeId()) { + case STRUCT: + return convert(type.asStructType(), (Row) object); + + case LIST: + List convertedList = Lists.newArrayList(); + List list = (List) object; + for (Object element : list) { + convertedList.add(convert(type.asListType().elementType(), element)); + } + return convertedList; + + case MAP: + Map convertedMap = Maps.newLinkedHashMap(); + Map map = (Map) object; + for (Map.Entry entry : map.entrySet()) { + convertedMap.put( + convert(type.asMapType().keyType(), entry.getKey()), + convert(type.asMapType().valueType(), entry.getValue())); + } + return convertedMap; + + case DATE: + // if spark.sql.datetime.java8API.enabled is set to true, java.time.LocalDate + // for Spark SQL DATE type otherwise java.sql.Date is returned. + return DateTimeUtils.anyToDays(object); + case TIMESTAMP: + return DateTimeUtils.anyToMicros(object); + case BINARY: + return ByteBuffer.wrap((byte[]) object); + case INTEGER: + return ((Number) object).intValue(); + case BOOLEAN: + case LONG: + case FLOAT: + case DOUBLE: + case DECIMAL: + case STRING: + case FIXED: + return object; + default: + throw new UnsupportedOperationException("Not a supported type: " + type); + } + } + + private static Record convert(Types.StructType struct, Row row) { + if (row == null) { + return null; + } + + Record record = GenericRecord.create(struct); + List fields = struct.fields(); + for (int i = 0; i < fields.size(); i += 1) { + Types.NestedField field = fields.get(i); + + Type fieldType = field.type(); + + switch (fieldType.typeId()) { + case STRUCT: + record.set(i, convert(fieldType.asStructType(), row.getStruct(i))); + break; + case LIST: + record.set(i, convert(fieldType.asListType(), row.getList(i))); + break; + case MAP: + record.set(i, convert(fieldType.asMapType(), row.getJavaMap(i))); + break; + default: + record.set(i, convert(fieldType, row.get(i))); + } + } + return record; + } + + public static Object convertToSpark(Type type, Object object) { + if (object == null) { + return null; + } + + switch (type.typeId()) { + case STRUCT: + case LIST: + case MAP: + return new UnsupportedOperationException("Complex types currently not supported"); + case DATE: + return DateTimeUtils.daysToLocalDate((int) object); + case TIMESTAMP: + Types.TimestampType ts = (Types.TimestampType) type.asPrimitiveType(); + if (ts.shouldAdjustToUTC()) { + return DateTimeUtils.microsToInstant((long) object); + } else { + return DateTimeUtils.microsToLocalDateTime((long) object); + } + case BINARY: + return ByteBuffers.toByteArray((ByteBuffer) object); + case INTEGER: + case BOOLEAN: + case LONG: + case FLOAT: + case DOUBLE: + case DECIMAL: + case STRING: + case FIXED: + return object; + default: + throw new UnsupportedOperationException("Not a supported type: " + type); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java new file mode 100644 index 000000000000..96131e0e56dd --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java @@ -0,0 +1,732 @@ +/* + * 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 static org.apache.iceberg.DistributionMode.HASH; +import static org.apache.iceberg.DistributionMode.NONE; +import static org.apache.iceberg.DistributionMode.RANGE; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.DELETE_AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.DELETE_AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.DELETE_ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.DELETE_ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.DELETE_PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.DELETE_PARQUET_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; +import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE; + +import java.util.Locale; +import java.util.Map; +import org.apache.iceberg.BaseMetadataTable; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.IsolationLevel; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableUtil; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.spark.sql.RuntimeConfig; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.write.RowLevelOperation.Command; +import org.apache.spark.sql.internal.SQLConf; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.collection.JavaConverters; + +/** + * A class for common Iceberg configs for Spark writes. + * + *

    If a config is set at multiple levels, the following order of precedence is used (top to + * bottom): + * + *

      + *
    1. Write options + *
    2. Session configuration + *
    3. Table metadata + *
    + * + * The most specific value is set in write options and takes precedence over all other configs. If + * no write option is provided, this class checks the session configuration for any overrides. If no + * applicable value is found in the session configuration, this class uses the table metadata. + * + *

    Note this class is NOT meant to be serialized and sent to executors. + */ +public class SparkWriteConf { + + private static final Logger LOG = LoggerFactory.getLogger(SparkWriteConf.class); + + private static final long DATA_FILE_SIZE = 128 * 1024 * 1024; // 128 MB + private static final long DELETE_FILE_SIZE = 32 * 1024 * 1024; // 32 MB + + private final SparkSession spark; + private final Table table; + private final String branch; + private final RuntimeConfig sessionConf; + private final Map writeOptions; + private final SparkConfParser confParser; + + public SparkWriteConf(SparkSession spark, Table table, Map writeOptions) { + this(spark, table, null, writeOptions); + } + + public SparkWriteConf( + SparkSession spark, Table table, String branch, Map writeOptions) { + this.spark = spark; + this.table = table; + this.branch = branch; + this.sessionConf = spark.conf(); + this.writeOptions = writeOptions; + this.confParser = new SparkConfParser(spark, table, writeOptions); + } + + public boolean checkNullability() { + return confParser + .booleanConf() + .option(SparkWriteOptions.CHECK_NULLABILITY) + .sessionConf(SparkSQLProperties.CHECK_NULLABILITY) + .defaultValue(SparkSQLProperties.CHECK_NULLABILITY_DEFAULT) + .parse(); + } + + public boolean checkOrdering() { + return confParser + .booleanConf() + .option(SparkWriteOptions.CHECK_ORDERING) + .sessionConf(SparkSQLProperties.CHECK_ORDERING) + .defaultValue(SparkSQLProperties.CHECK_ORDERING_DEFAULT) + .parse(); + } + + public String overwriteMode() { + String overwriteMode = writeOptions.get(SparkWriteOptions.OVERWRITE_MODE); + return overwriteMode != null ? overwriteMode.toLowerCase(Locale.ROOT) : null; + } + + public boolean wapEnabled() { + return confParser + .booleanConf() + .tableProperty(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED) + .defaultValue(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED_DEFAULT) + .parse(); + } + + public String wapId() { + return sessionConf.get(SparkSQLProperties.WAP_ID, null); + } + + public boolean mergeSchema() { + return confParser + .booleanConf() + .option(SparkWriteOptions.MERGE_SCHEMA) + .option(SparkWriteOptions.SPARK_MERGE_SCHEMA) + .sessionConf(SparkSQLProperties.MERGE_SCHEMA) + .defaultValue(SparkSQLProperties.MERGE_SCHEMA_DEFAULT) + .parse(); + } + + public int outputSpecId() { + int outputSpecId = + confParser + .intConf() + .option(SparkWriteOptions.OUTPUT_SPEC_ID) + .defaultValue(table.spec().specId()) + .parse(); + Preconditions.checkArgument( + table.specs().containsKey(outputSpecId), + "Output spec id %s is not a valid spec id for table", + outputSpecId); + return outputSpecId; + } + + public FileFormat dataFileFormat() { + String valueAsString = + confParser + .stringConf() + .option(SparkWriteOptions.WRITE_FORMAT) + .tableProperty(TableProperties.DEFAULT_FILE_FORMAT) + .defaultValue(TableProperties.DEFAULT_FILE_FORMAT_DEFAULT) + .parse(); + return FileFormat.fromString(valueAsString); + } + + private String dataCompressionCodec() { + switch (dataFileFormat()) { + case PARQUET: + return parquetCompressionCodec(); + case AVRO: + return avroCompressionCodec(); + case ORC: + return orcCompressionCodec(); + default: + return null; + } + } + + public long targetDataFileSize() { + return confParser + .longConf() + .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES) + .tableProperty(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES) + .defaultValue(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT) + .parse(); + } + + public boolean useFanoutWriter(SparkWriteRequirements writeRequirements) { + boolean defaultValue = !writeRequirements.hasOrdering(); + return fanoutWriterEnabled(defaultValue); + } + + private boolean fanoutWriterEnabled() { + return fanoutWriterEnabled(true /* enabled by default */); + } + + private boolean fanoutWriterEnabled(boolean defaultValue) { + return confParser + .booleanConf() + .option(SparkWriteOptions.FANOUT_ENABLED) + .tableProperty(TableProperties.SPARK_WRITE_PARTITIONED_FANOUT_ENABLED) + .defaultValue(defaultValue) + .parse(); + } + + public FileFormat deleteFileFormat() { + if (!(table instanceof BaseMetadataTable) && TableUtil.formatVersion(table) >= 3) { + return FileFormat.PUFFIN; + } + + String valueAsString = + confParser + .stringConf() + .option(SparkWriteOptions.DELETE_FORMAT) + .tableProperty(TableProperties.DELETE_DEFAULT_FILE_FORMAT) + .parseOptional(); + return valueAsString != null ? FileFormat.fromString(valueAsString) : dataFileFormat(); + } + + private String deleteCompressionCodec() { + switch (deleteFileFormat()) { + case PARQUET: + return deleteParquetCompressionCodec(); + case AVRO: + return deleteAvroCompressionCodec(); + case ORC: + return deleteOrcCompressionCodec(); + default: + return null; + } + } + + public long targetDeleteFileSize() { + return confParser + .longConf() + .option(SparkWriteOptions.TARGET_DELETE_FILE_SIZE_BYTES) + .tableProperty(TableProperties.DELETE_TARGET_FILE_SIZE_BYTES) + .defaultValue(TableProperties.DELETE_TARGET_FILE_SIZE_BYTES_DEFAULT) + .parse(); + } + + public Map extraSnapshotMetadata() { + Map extraSnapshotMetadata = Maps.newHashMap(); + + // Add session configuration properties with SNAPSHOT_PROPERTY_PREFIX if necessary + extraSnapshotMetadata.putAll( + PropertyUtil.propertiesWithPrefix( + JavaConverters.mapAsJavaMap(sessionConf.getAll()), + SparkSQLProperties.SNAPSHOT_PROPERTY_PREFIX)); + + // Add write options, overriding session configuration if necessary + extraSnapshotMetadata.putAll( + PropertyUtil.propertiesWithPrefix(writeOptions, SnapshotSummary.EXTRA_METADATA_PREFIX)); + + return extraSnapshotMetadata; + } + + public String rewrittenFileSetId() { + return confParser + .stringConf() + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID) + .parseOptional(); + } + + public SparkWriteRequirements writeRequirements() { + if (ignoreTableDistributionAndOrdering()) { + LOG.info("Skipping distribution/ordering: disabled per job configuration"); + return SparkWriteRequirements.EMPTY; + } + + return SparkWriteUtil.writeRequirements( + table, distributionMode(), fanoutWriterEnabled(), dataAdvisoryPartitionSize()); + } + + @VisibleForTesting + DistributionMode distributionMode() { + String modeName = + confParser + .stringConf() + .option(SparkWriteOptions.DISTRIBUTION_MODE) + .sessionConf(SparkSQLProperties.DISTRIBUTION_MODE) + .tableProperty(TableProperties.WRITE_DISTRIBUTION_MODE) + .parseOptional(); + + if (modeName != null) { + DistributionMode mode = DistributionMode.fromName(modeName); + return adjustWriteDistributionMode(mode); + } else { + return defaultWriteDistributionMode(); + } + } + + private DistributionMode adjustWriteDistributionMode(DistributionMode mode) { + if (mode == RANGE && table.spec().isUnpartitioned() && table.sortOrder().isUnsorted()) { + return NONE; + } else if (mode == HASH && table.spec().isUnpartitioned()) { + return NONE; + } else { + return mode; + } + } + + private DistributionMode defaultWriteDistributionMode() { + if (table.sortOrder().isSorted()) { + return RANGE; + } else if (table.spec().isPartitioned()) { + return HASH; + } else { + return NONE; + } + } + + public SparkWriteRequirements copyOnWriteRequirements(Command command) { + if (ignoreTableDistributionAndOrdering()) { + LOG.info("Skipping distribution/ordering: disabled per job configuration"); + return SparkWriteRequirements.EMPTY; + } + + return SparkWriteUtil.copyOnWriteRequirements( + table, + command, + copyOnWriteDistributionMode(command), + fanoutWriterEnabled(), + dataAdvisoryPartitionSize()); + } + + @VisibleForTesting + DistributionMode copyOnWriteDistributionMode(Command command) { + switch (command) { + case DELETE: + return deleteDistributionMode(); + case UPDATE: + return updateDistributionMode(); + case MERGE: + return copyOnWriteMergeDistributionMode(); + default: + throw new IllegalArgumentException("Unexpected command: " + command); + } + } + + public SparkWriteRequirements positionDeltaRequirements(Command command) { + if (ignoreTableDistributionAndOrdering()) { + LOG.info("Skipping distribution/ordering: disabled per job configuration"); + return SparkWriteRequirements.EMPTY; + } + + return SparkWriteUtil.positionDeltaRequirements( + table, + command, + positionDeltaDistributionMode(command), + fanoutWriterEnabled(), + command == DELETE ? deleteAdvisoryPartitionSize() : dataAdvisoryPartitionSize()); + } + + @VisibleForTesting + DistributionMode positionDeltaDistributionMode(Command command) { + switch (command) { + case DELETE: + return deleteDistributionMode(); + case UPDATE: + return updateDistributionMode(); + case MERGE: + return positionDeltaMergeDistributionMode(); + default: + throw new IllegalArgumentException("Unexpected command: " + command); + } + } + + private DistributionMode deleteDistributionMode() { + String deleteModeName = + confParser + .stringConf() + .option(SparkWriteOptions.DISTRIBUTION_MODE) + .sessionConf(SparkSQLProperties.DISTRIBUTION_MODE) + .tableProperty(TableProperties.DELETE_DISTRIBUTION_MODE) + .defaultValue(TableProperties.WRITE_DISTRIBUTION_MODE_HASH) + .parse(); + return DistributionMode.fromName(deleteModeName); + } + + private DistributionMode updateDistributionMode() { + String updateModeName = + confParser + .stringConf() + .option(SparkWriteOptions.DISTRIBUTION_MODE) + .sessionConf(SparkSQLProperties.DISTRIBUTION_MODE) + .tableProperty(TableProperties.UPDATE_DISTRIBUTION_MODE) + .defaultValue(TableProperties.WRITE_DISTRIBUTION_MODE_HASH) + .parse(); + return DistributionMode.fromName(updateModeName); + } + + private DistributionMode copyOnWriteMergeDistributionMode() { + String mergeModeName = + confParser + .stringConf() + .option(SparkWriteOptions.DISTRIBUTION_MODE) + .sessionConf(SparkSQLProperties.DISTRIBUTION_MODE) + .tableProperty(TableProperties.MERGE_DISTRIBUTION_MODE) + .parseOptional(); + + if (mergeModeName != null) { + DistributionMode mergeMode = DistributionMode.fromName(mergeModeName); + return adjustWriteDistributionMode(mergeMode); + + } else if (table.spec().isPartitioned()) { + return HASH; + + } else { + return distributionMode(); + } + } + + private DistributionMode positionDeltaMergeDistributionMode() { + String mergeModeName = + confParser + .stringConf() + .option(SparkWriteOptions.DISTRIBUTION_MODE) + .sessionConf(SparkSQLProperties.DISTRIBUTION_MODE) + .tableProperty(TableProperties.MERGE_DISTRIBUTION_MODE) + .defaultValue(TableProperties.WRITE_DISTRIBUTION_MODE_HASH) + .parse(); + return DistributionMode.fromName(mergeModeName); + } + + private boolean ignoreTableDistributionAndOrdering() { + return confParser + .booleanConf() + .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING) + .defaultValue(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING_DEFAULT) + .negate() + .parse(); + } + + public Long validateFromSnapshotId() { + return confParser + .longConf() + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID) + .parseOptional(); + } + + public IsolationLevel isolationLevel() { + String isolationLevelName = + confParser.stringConf().option(SparkWriteOptions.ISOLATION_LEVEL).parseOptional(); + return isolationLevelName != null ? IsolationLevel.fromName(isolationLevelName) : null; + } + + public boolean caseSensitive() { + return confParser + .booleanConf() + .sessionConf(SQLConf.CASE_SENSITIVE().key()) + .defaultValue(SQLConf.CASE_SENSITIVE().defaultValueString()) + .parse(); + } + + public String branch() { + if (wapEnabled()) { + String wapId = wapId(); + String wapBranch = + confParser.stringConf().sessionConf(SparkSQLProperties.WAP_BRANCH).parseOptional(); + + ValidationException.check( + wapId == null || wapBranch == null, + "Cannot set both WAP ID and branch, but got ID [%s] and branch [%s]", + wapId, + wapBranch); + + if (wapBranch != null) { + ValidationException.check( + branch == null, + "Cannot write to both branch and WAP branch, but got branch [%s] and WAP branch [%s]", + branch, + wapBranch); + + return wapBranch; + } + } + + return branch; + } + + public Map writeProperties() { + Map writeProperties = Maps.newHashMap(); + writeProperties.putAll(dataWriteProperties()); + writeProperties.putAll(deleteWriteProperties()); + return writeProperties; + } + + private Map dataWriteProperties() { + Map writeProperties = Maps.newHashMap(); + FileFormat dataFormat = dataFileFormat(); + + switch (dataFormat) { + case PARQUET: + writeProperties.put(PARQUET_COMPRESSION, parquetCompressionCodec()); + String parquetCompressionLevel = parquetCompressionLevel(); + if (parquetCompressionLevel != null) { + writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); + } + break; + + case AVRO: + writeProperties.put(AVRO_COMPRESSION, avroCompressionCodec()); + String avroCompressionLevel = avroCompressionLevel(); + if (avroCompressionLevel != null) { + writeProperties.put(AVRO_COMPRESSION_LEVEL, avroCompressionLevel); + } + break; + + case ORC: + writeProperties.put(ORC_COMPRESSION, orcCompressionCodec()); + writeProperties.put(ORC_COMPRESSION_STRATEGY, orcCompressionStrategy()); + break; + + default: + // skip + } + + return writeProperties; + } + + private Map deleteWriteProperties() { + Map writeProperties = Maps.newHashMap(); + FileFormat deleteFormat = deleteFileFormat(); + + switch (deleteFormat) { + case PARQUET: + writeProperties.put(DELETE_PARQUET_COMPRESSION, deleteParquetCompressionCodec()); + String deleteParquetCompressionLevel = deleteParquetCompressionLevel(); + if (deleteParquetCompressionLevel != null) { + writeProperties.put(DELETE_PARQUET_COMPRESSION_LEVEL, deleteParquetCompressionLevel); + } + break; + + case AVRO: + writeProperties.put(DELETE_AVRO_COMPRESSION, deleteAvroCompressionCodec()); + String deleteAvroCompressionLevel = deleteAvroCompressionLevel(); + if (deleteAvroCompressionLevel != null) { + writeProperties.put(DELETE_AVRO_COMPRESSION_LEVEL, deleteAvroCompressionLevel); + } + break; + + case ORC: + writeProperties.put(DELETE_ORC_COMPRESSION, deleteOrcCompressionCodec()); + writeProperties.put(DELETE_ORC_COMPRESSION_STRATEGY, deleteOrcCompressionStrategy()); + break; + + default: + // skip + } + + return writeProperties; + } + + private String parquetCompressionCodec() { + return confParser + .stringConf() + .option(SparkWriteOptions.COMPRESSION_CODEC) + .sessionConf(SparkSQLProperties.COMPRESSION_CODEC) + .tableProperty(TableProperties.PARQUET_COMPRESSION) + .defaultValue(TableProperties.PARQUET_COMPRESSION_DEFAULT) + .parse(); + } + + private String deleteParquetCompressionCodec() { + return confParser + .stringConf() + .option(SparkWriteOptions.COMPRESSION_CODEC) + .sessionConf(SparkSQLProperties.COMPRESSION_CODEC) + .tableProperty(DELETE_PARQUET_COMPRESSION) + .defaultValue(parquetCompressionCodec()) + .parse(); + } + + private String parquetCompressionLevel() { + return confParser + .stringConf() + .option(SparkWriteOptions.COMPRESSION_LEVEL) + .sessionConf(SparkSQLProperties.COMPRESSION_LEVEL) + .tableProperty(TableProperties.PARQUET_COMPRESSION_LEVEL) + .defaultValue(TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT) + .parseOptional(); + } + + private String deleteParquetCompressionLevel() { + return confParser + .stringConf() + .option(SparkWriteOptions.COMPRESSION_LEVEL) + .sessionConf(SparkSQLProperties.COMPRESSION_LEVEL) + .tableProperty(DELETE_PARQUET_COMPRESSION_LEVEL) + .defaultValue(parquetCompressionLevel()) + .parseOptional(); + } + + private String avroCompressionCodec() { + return confParser + .stringConf() + .option(SparkWriteOptions.COMPRESSION_CODEC) + .sessionConf(SparkSQLProperties.COMPRESSION_CODEC) + .tableProperty(TableProperties.AVRO_COMPRESSION) + .defaultValue(TableProperties.AVRO_COMPRESSION_DEFAULT) + .parse(); + } + + private String deleteAvroCompressionCodec() { + return confParser + .stringConf() + .option(SparkWriteOptions.COMPRESSION_CODEC) + .sessionConf(SparkSQLProperties.COMPRESSION_CODEC) + .tableProperty(DELETE_AVRO_COMPRESSION) + .defaultValue(avroCompressionCodec()) + .parse(); + } + + private String avroCompressionLevel() { + return confParser + .stringConf() + .option(SparkWriteOptions.COMPRESSION_LEVEL) + .sessionConf(SparkSQLProperties.COMPRESSION_LEVEL) + .tableProperty(TableProperties.AVRO_COMPRESSION_LEVEL) + .defaultValue(TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT) + .parseOptional(); + } + + private String deleteAvroCompressionLevel() { + return confParser + .stringConf() + .option(SparkWriteOptions.COMPRESSION_LEVEL) + .sessionConf(SparkSQLProperties.COMPRESSION_LEVEL) + .tableProperty(DELETE_AVRO_COMPRESSION_LEVEL) + .defaultValue(avroCompressionLevel()) + .parseOptional(); + } + + private String orcCompressionCodec() { + return confParser + .stringConf() + .option(SparkWriteOptions.COMPRESSION_CODEC) + .sessionConf(SparkSQLProperties.COMPRESSION_CODEC) + .tableProperty(TableProperties.ORC_COMPRESSION) + .defaultValue(TableProperties.ORC_COMPRESSION_DEFAULT) + .parse(); + } + + private String deleteOrcCompressionCodec() { + return confParser + .stringConf() + .option(SparkWriteOptions.COMPRESSION_CODEC) + .sessionConf(SparkSQLProperties.COMPRESSION_CODEC) + .tableProperty(DELETE_ORC_COMPRESSION) + .defaultValue(orcCompressionCodec()) + .parse(); + } + + private String orcCompressionStrategy() { + return confParser + .stringConf() + .option(SparkWriteOptions.COMPRESSION_STRATEGY) + .sessionConf(SparkSQLProperties.COMPRESSION_STRATEGY) + .tableProperty(TableProperties.ORC_COMPRESSION_STRATEGY) + .defaultValue(TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT) + .parse(); + } + + private String deleteOrcCompressionStrategy() { + return confParser + .stringConf() + .option(SparkWriteOptions.COMPRESSION_STRATEGY) + .sessionConf(SparkSQLProperties.COMPRESSION_STRATEGY) + .tableProperty(DELETE_ORC_COMPRESSION_STRATEGY) + .defaultValue(orcCompressionStrategy()) + .parse(); + } + + private long dataAdvisoryPartitionSize() { + long defaultValue = + advisoryPartitionSize(DATA_FILE_SIZE, dataFileFormat(), dataCompressionCodec()); + return advisoryPartitionSize(defaultValue); + } + + private long deleteAdvisoryPartitionSize() { + long defaultValue = + advisoryPartitionSize(DELETE_FILE_SIZE, deleteFileFormat(), deleteCompressionCodec()); + return advisoryPartitionSize(defaultValue); + } + + private long advisoryPartitionSize(long defaultValue) { + return confParser + .longConf() + .option(SparkWriteOptions.ADVISORY_PARTITION_SIZE) + .sessionConf(SparkSQLProperties.ADVISORY_PARTITION_SIZE) + .tableProperty(TableProperties.SPARK_WRITE_ADVISORY_PARTITION_SIZE_BYTES) + .defaultValue(defaultValue) + .parse(); + } + + private long advisoryPartitionSize( + long expectedFileSize, FileFormat outputFileFormat, String outputCodec) { + double shuffleCompressionRatio = shuffleCompressionRatio(outputFileFormat, outputCodec); + long suggestedAdvisoryPartitionSize = (long) (expectedFileSize * shuffleCompressionRatio); + return Math.max(suggestedAdvisoryPartitionSize, sparkAdvisoryPartitionSize()); + } + + private long sparkAdvisoryPartitionSize() { + return (long) spark.sessionState().conf().getConf(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES()); + } + + private double shuffleCompressionRatio(FileFormat outputFileFormat, String outputCodec) { + return SparkCompressionUtil.shuffleCompressionRatio(spark, outputFileFormat, outputCodec); + } + + public DeleteGranularity deleteGranularity() { + return confParser + .enumConf(DeleteGranularity::fromString) + .option(SparkWriteOptions.DELETE_GRANULARITY) + .tableProperty(TableProperties.DELETE_GRANULARITY) + .defaultValue(DeleteGranularity.FILE) + .parse(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java new file mode 100644 index 000000000000..33db70bae587 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java @@ -0,0 +1,88 @@ +/* + * 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; + +/** Spark DF write options */ +public class SparkWriteOptions { + + private SparkWriteOptions() {} + + // Fileformat for write operations(default: Table write.format.default ) + public static final String WRITE_FORMAT = "write-format"; + + // Overrides this table's write.target-file-size-bytes + public static final String TARGET_FILE_SIZE_BYTES = "target-file-size-bytes"; + + // Overrides the default file format for delete files + public static final String DELETE_FORMAT = "delete-format"; + + // Overrides the default size for delete files + public static final String TARGET_DELETE_FILE_SIZE_BYTES = "target-delete-file-size-bytes"; + + // Sets the nullable check on fields(default: true) + public static final String CHECK_NULLABILITY = "check-nullability"; + + // Adds an entry with custom-key and corresponding value in the snapshot summary + // ex: df.write().format(iceberg) + // .option(SparkWriteOptions.SNAPSHOT_PROPERTY_PREFIX."key1", "value1") + // .save(location) + public static final String SNAPSHOT_PROPERTY_PREFIX = "snapshot-property"; + + // Overrides table property write.spark.fanout.enabled(default: false) + public static final String FANOUT_ENABLED = "fanout-enabled"; + + // Checks if input schema and table schema are same(default: true) + public static final String CHECK_ORDERING = "check-ordering"; + + // File scan task set ID that indicates which files must be replaced + public static final String REWRITTEN_FILE_SCAN_TASK_SET_ID = "rewritten-file-scan-task-set-id"; + + public static final String OUTPUT_SPEC_ID = "output-spec-id"; + + public static final String OVERWRITE_MODE = "overwrite-mode"; + + // Overrides the default distribution mode for a write operation + public static final String DISTRIBUTION_MODE = "distribution-mode"; + + // Controls whether to take into account the table distribution and sort order during a write + // operation + public static final String USE_TABLE_DISTRIBUTION_AND_ORDERING = + "use-table-distribution-and-ordering"; + public static final boolean USE_TABLE_DISTRIBUTION_AND_ORDERING_DEFAULT = true; + + public static final String MERGE_SCHEMA = "merge-schema"; + public static final String SPARK_MERGE_SCHEMA = "mergeSchema"; + + // Identifies snapshot from which to start validating conflicting changes + public static final String VALIDATE_FROM_SNAPSHOT_ID = "validate-from-snapshot-id"; + + // Isolation Level for DataFrame calls. Currently supported by overwritePartitions + public static final String ISOLATION_LEVEL = "isolation-level"; + + // Controls write compress options + public static final String COMPRESSION_CODEC = "compression-codec"; + public static final String COMPRESSION_LEVEL = "compression-level"; + public static final String COMPRESSION_STRATEGY = "compression-strategy"; + + // Overrides the advisory partition size + public static final String ADVISORY_PARTITION_SIZE = "advisory-partition-size"; + + // Overrides the delete granularity + public static final String DELETE_GRANULARITY = "delete-granularity"; +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java new file mode 100644 index 000000000000..833e0e44e391 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java @@ -0,0 +1,59 @@ +/* + * 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 org.apache.spark.sql.connector.distributions.Distribution; +import org.apache.spark.sql.connector.distributions.Distributions; +import org.apache.spark.sql.connector.distributions.UnspecifiedDistribution; +import org.apache.spark.sql.connector.expressions.SortOrder; + +/** A set of requirements such as distribution and ordering reported to Spark during writes. */ +public class SparkWriteRequirements { + + public static final SparkWriteRequirements EMPTY = + new SparkWriteRequirements(Distributions.unspecified(), new SortOrder[0], 0); + + private final Distribution distribution; + private final SortOrder[] ordering; + private final long advisoryPartitionSize; + + SparkWriteRequirements( + Distribution distribution, SortOrder[] ordering, long advisoryPartitionSize) { + this.distribution = distribution; + this.ordering = ordering; + this.advisoryPartitionSize = advisoryPartitionSize; + } + + public Distribution distribution() { + return distribution; + } + + public SortOrder[] ordering() { + return ordering; + } + + public boolean hasOrdering() { + return ordering.length != 0; + } + + public long advisoryPartitionSize() { + // Spark prohibits requesting a particular advisory partition size without distribution + return distribution instanceof UnspecifiedDistribution ? 0 : advisoryPartitionSize; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java new file mode 100644 index 000000000000..0d68a0d8cdd0 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java @@ -0,0 +1,259 @@ +/* + * 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 static org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE; +import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.MERGE; +import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.UPDATE; + +import java.util.Arrays; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ObjectArrays; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SortOrderUtil; +import org.apache.spark.sql.connector.distributions.Distribution; +import org.apache.spark.sql.connector.distributions.Distributions; +import org.apache.spark.sql.connector.expressions.Expression; +import org.apache.spark.sql.connector.expressions.Expressions; +import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.expressions.SortDirection; +import org.apache.spark.sql.connector.expressions.SortOrder; +import org.apache.spark.sql.connector.write.RowLevelOperation.Command; + +/** + * A utility that contains helper methods for working with Spark writes. + * + *

    Note it is an evolving internal API that is subject to change even in minor releases. + */ +public class SparkWriteUtil { + + private static final NamedReference SPEC_ID = ref(MetadataColumns.SPEC_ID); + private static final NamedReference PARTITION = ref(MetadataColumns.PARTITION_COLUMN_NAME); + private static final NamedReference FILE_PATH = ref(MetadataColumns.FILE_PATH); + private static final NamedReference ROW_POSITION = ref(MetadataColumns.ROW_POSITION); + + private static final Expression[] FILE_CLUSTERING = clusterBy(FILE_PATH); + private static final Expression[] PARTITION_CLUSTERING = clusterBy(SPEC_ID, PARTITION); + private static final Expression[] PARTITION_FILE_CLUSTERING = + clusterBy(SPEC_ID, PARTITION, FILE_PATH); + + private static final SortOrder[] EMPTY_ORDERING = new SortOrder[0]; + private static final SortOrder[] EXISTING_ROW_ORDERING = orderBy(FILE_PATH, ROW_POSITION); + private static final SortOrder[] PARTITION_ORDERING = orderBy(SPEC_ID, PARTITION); + private static final SortOrder[] PARTITION_FILE_ORDERING = orderBy(SPEC_ID, PARTITION, FILE_PATH); + private static final SortOrder[] POSITION_DELETE_ORDERING = + orderBy(SPEC_ID, PARTITION, FILE_PATH, ROW_POSITION); + + private SparkWriteUtil() {} + + /** Builds requirements for batch and micro-batch writes such as append or overwrite. */ + public static SparkWriteRequirements writeRequirements( + Table table, DistributionMode mode, boolean fanoutEnabled, long advisoryPartitionSize) { + + Distribution distribution = writeDistribution(table, mode); + SortOrder[] ordering = writeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + } + + private static Distribution writeDistribution(Table table, DistributionMode mode) { + switch (mode) { + case NONE: + return Distributions.unspecified(); + + case HASH: + return Distributions.clustered(clustering(table)); + + case RANGE: + return Distributions.ordered(ordering(table)); + + default: + throw new IllegalArgumentException("Unsupported distribution mode: " + mode); + } + } + + /** Builds requirements for copy-on-write DELETE, UPDATE, MERGE operations. */ + public static SparkWriteRequirements copyOnWriteRequirements( + Table table, + Command command, + DistributionMode mode, + boolean fanoutEnabled, + long advisoryPartitionSize) { + + if (command == DELETE || command == UPDATE) { + Distribution distribution = copyOnWriteDeleteUpdateDistribution(table, mode); + SortOrder[] ordering = writeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + } else { + return writeRequirements(table, mode, fanoutEnabled, advisoryPartitionSize); + } + } + + private static Distribution copyOnWriteDeleteUpdateDistribution( + Table table, DistributionMode mode) { + + switch (mode) { + case NONE: + return Distributions.unspecified(); + + case HASH: + if (table.spec().isPartitioned()) { + return Distributions.clustered(clustering(table)); + } else { + return Distributions.clustered(FILE_CLUSTERING); + } + + case RANGE: + if (table.spec().isPartitioned() || table.sortOrder().isSorted()) { + return Distributions.ordered(ordering(table)); + } else { + return Distributions.ordered(EXISTING_ROW_ORDERING); + } + + default: + throw new IllegalArgumentException("Unexpected distribution mode: " + mode); + } + } + + /** Builds requirements for merge-on-read DELETE, UPDATE, MERGE operations. */ + public static SparkWriteRequirements positionDeltaRequirements( + Table table, + Command command, + DistributionMode mode, + boolean fanoutEnabled, + long advisoryPartitionSize) { + + if (command == UPDATE || command == MERGE) { + Distribution distribution = positionDeltaUpdateMergeDistribution(table, mode); + SortOrder[] ordering = positionDeltaUpdateMergeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + } else { + Distribution distribution = positionDeltaDeleteDistribution(table, mode); + SortOrder[] ordering = fanoutEnabled ? EMPTY_ORDERING : POSITION_DELETE_ORDERING; + return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + } + } + + private static Distribution positionDeltaUpdateMergeDistribution( + Table table, DistributionMode mode) { + + switch (mode) { + case NONE: + return Distributions.unspecified(); + + case HASH: + if (table.spec().isUnpartitioned()) { + return Distributions.clustered(concat(PARTITION_FILE_CLUSTERING, clustering(table))); + } else { + return Distributions.clustered(concat(PARTITION_CLUSTERING, clustering(table))); + } + + case RANGE: + if (table.spec().isUnpartitioned()) { + return Distributions.ordered(concat(PARTITION_FILE_ORDERING, ordering(table))); + } else { + return Distributions.ordered(concat(PARTITION_ORDERING, ordering(table))); + } + + default: + throw new IllegalArgumentException("Unsupported distribution mode: " + mode); + } + } + + private static SortOrder[] positionDeltaUpdateMergeOrdering(Table table, boolean fanoutEnabled) { + if (fanoutEnabled && table.sortOrder().isUnsorted()) { + return EMPTY_ORDERING; + } else { + return concat(POSITION_DELETE_ORDERING, ordering(table)); + } + } + + private static Distribution positionDeltaDeleteDistribution(Table table, DistributionMode mode) { + switch (mode) { + case NONE: + return Distributions.unspecified(); + + case HASH: + if (table.spec().isUnpartitioned()) { + return Distributions.clustered(PARTITION_FILE_CLUSTERING); + } else { + return Distributions.clustered(PARTITION_CLUSTERING); + } + + case RANGE: + if (table.spec().isUnpartitioned()) { + return Distributions.ordered(PARTITION_FILE_ORDERING); + } else { + return Distributions.ordered(PARTITION_ORDERING); + } + + default: + throw new IllegalArgumentException("Unsupported distribution mode: " + mode); + } + } + + // a local ordering within a task is beneficial in two cases: + // - there is a defined table sort order, so it is clear how the data should be ordered + // - the table is partitioned and fanout writers are disabled, + // so records for one partition must be co-located within a task + private static SortOrder[] writeOrdering(Table table, boolean fanoutEnabled) { + if (fanoutEnabled && table.sortOrder().isUnsorted()) { + return EMPTY_ORDERING; + } else { + return ordering(table); + } + } + + private static Expression[] clustering(Table table) { + return Spark3Util.toTransforms(table.spec()); + } + + private static SortOrder[] ordering(Table table) { + return Spark3Util.toOrdering(SortOrderUtil.buildSortOrder(table)); + } + + private static Expression[] concat(Expression[] clustering, Expression... otherClustering) { + return ObjectArrays.concat(clustering, otherClustering, Expression.class); + } + + private static SortOrder[] concat(SortOrder[] ordering, SortOrder... otherOrdering) { + return ObjectArrays.concat(ordering, otherOrdering, SortOrder.class); + } + + private static NamedReference ref(Types.NestedField field) { + return Expressions.column(field.name()); + } + + private static NamedReference ref(String name) { + return Expressions.column(name); + } + + private static Expression[] clusterBy(Expression... exprs) { + return exprs; + } + + private static SortOrder[] orderBy(Expression... exprs) { + return Arrays.stream(exprs).map(SparkWriteUtil::sort).toArray(SortOrder[]::new); + } + + private static SortOrder sort(Expression expr) { + return Expressions.sort(expr, SortDirection.ASCENDING); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java new file mode 100644 index 000000000000..83e56d1bd9b5 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java @@ -0,0 +1,65 @@ +/* + * 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 org.apache.iceberg.spark.functions.SparkFunctions; +import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException; +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.connector.catalog.FunctionCatalog; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; + +interface SupportsFunctions extends FunctionCatalog { + + default boolean isFunctionNamespace(String[] namespace) { + return namespace.length == 0; + } + + default boolean isExistingNamespace(String[] namespace) { + return namespace.length == 0; + } + + @Override + default Identifier[] listFunctions(String[] namespace) throws NoSuchNamespaceException { + if (isFunctionNamespace(namespace)) { + return SparkFunctions.list().stream() + .map(name -> Identifier.of(namespace, name)) + .toArray(Identifier[]::new); + } else if (isExistingNamespace(namespace)) { + return new Identifier[0]; + } + + throw new NoSuchNamespaceException(namespace); + } + + @Override + default UnboundFunction loadFunction(Identifier ident) throws NoSuchFunctionException { + String[] namespace = ident.namespace(); + String name = ident.name(); + + if (isFunctionNamespace(namespace)) { + UnboundFunction func = SparkFunctions.load(name); + if (func != null) { + return func; + } + } + + throw new NoSuchFunctionException(ident); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SupportsReplaceView.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SupportsReplaceView.java new file mode 100644 index 000000000000..8bdb7b13861c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SupportsReplaceView.java @@ -0,0 +1,56 @@ +/* + * 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.Map; +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.catalyst.analysis.NoSuchViewException; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.View; +import org.apache.spark.sql.connector.catalog.ViewCatalog; +import org.apache.spark.sql.types.StructType; + +public interface SupportsReplaceView extends ViewCatalog { + /** + * Replace a view in the catalog + * + * @param ident a view identifier + * @param sql the SQL text that defines the view + * @param currentCatalog the current catalog + * @param currentNamespace the current namespace + * @param schema the view query output schema + * @param queryColumnNames the query column names + * @param columnAliases the column aliases + * @param columnComments the column comments + * @param properties the view properties + * @throws NoSuchViewException If the view doesn't exist or is a table + * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) + */ + View replaceView( + Identifier ident, + String sql, + String currentCatalog, + String[] currentNamespace, + StructType schema, + String[] queryColumnNames, + String[] columnAliases, + String[] columnComments, + Map properties) + throws NoSuchViewException, NoSuchNamespaceException; +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java new file mode 100644 index 000000000000..09c89bbba813 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/TypeToSparkType.java @@ -0,0 +1,166 @@ +/* + * 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.List; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.expressions.Literal$; +import org.apache.spark.sql.types.ArrayType$; +import org.apache.spark.sql.types.BinaryType$; +import org.apache.spark.sql.types.BooleanType$; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DateType$; +import org.apache.spark.sql.types.DecimalType$; +import org.apache.spark.sql.types.DoubleType$; +import org.apache.spark.sql.types.FloatType$; +import org.apache.spark.sql.types.IntegerType$; +import org.apache.spark.sql.types.LongType$; +import org.apache.spark.sql.types.MapType$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.MetadataBuilder; +import org.apache.spark.sql.types.NullType$; +import org.apache.spark.sql.types.StringType$; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType$; +import org.apache.spark.sql.types.TimestampNTZType$; +import org.apache.spark.sql.types.TimestampType$; +import org.apache.spark.sql.types.VariantType$; + +class TypeToSparkType extends TypeUtil.SchemaVisitor { + TypeToSparkType() {} + + public static final String METADATA_COL_ATTR_KEY = "__metadata_col"; + + @Override + public DataType schema(Schema schema, DataType structType) { + return structType; + } + + @Override + public DataType struct(Types.StructType struct, List fieldResults) { + List fields = struct.fields(); + + List sparkFields = Lists.newArrayListWithExpectedSize(fieldResults.size()); + for (int i = 0; i < fields.size(); i += 1) { + Types.NestedField field = fields.get(i); + DataType type = fieldResults.get(i); + Metadata metadata = fieldMetadata(field.fieldId()); + StructField sparkField = StructField.apply(field.name(), type, field.isOptional(), metadata); + if (field.doc() != null) { + sparkField = sparkField.withComment(field.doc()); + } + + // Convert both write and initial default values to Spark SQL string literal representations + // on the StructField metadata + if (field.writeDefault() != null) { + Object writeDefault = SparkUtil.internalToSpark(field.type(), field.writeDefault()); + sparkField = + sparkField.withCurrentDefaultValue(Literal$.MODULE$.create(writeDefault, type).sql()); + } + + if (field.initialDefault() != null) { + Object initialDefault = SparkUtil.internalToSpark(field.type(), field.initialDefault()); + sparkField = + sparkField.withExistenceDefaultValue( + Literal$.MODULE$.create(initialDefault, type).sql()); + } + + sparkFields.add(sparkField); + } + + return StructType$.MODULE$.apply(sparkFields); + } + + @Override + public DataType field(Types.NestedField field, DataType fieldResult) { + return fieldResult; + } + + @Override + public DataType list(Types.ListType list, DataType elementResult) { + return ArrayType$.MODULE$.apply(elementResult, list.isElementOptional()); + } + + @Override + public DataType map(Types.MapType map, DataType keyResult, DataType valueResult) { + return MapType$.MODULE$.apply(keyResult, valueResult, map.isValueOptional()); + } + + @Override + public DataType variant(Types.VariantType variant) { + return VariantType$.MODULE$; + } + + @Override + public DataType primitive(Type.PrimitiveType primitive) { + switch (primitive.typeId()) { + case BOOLEAN: + return BooleanType$.MODULE$; + case INTEGER: + return IntegerType$.MODULE$; + case LONG: + return LongType$.MODULE$; + case FLOAT: + return FloatType$.MODULE$; + case DOUBLE: + return DoubleType$.MODULE$; + case DATE: + return DateType$.MODULE$; + case TIME: + throw new UnsupportedOperationException("Spark does not support time fields"); + case TIMESTAMP: + Types.TimestampType ts = (Types.TimestampType) primitive; + if (ts.shouldAdjustToUTC()) { + return TimestampType$.MODULE$; + } else { + return TimestampNTZType$.MODULE$; + } + case STRING: + return StringType$.MODULE$; + case UUID: + // use String + return StringType$.MODULE$; + case FIXED: + return BinaryType$.MODULE$; + case BINARY: + return BinaryType$.MODULE$; + case DECIMAL: + Types.DecimalType decimal = (Types.DecimalType) primitive; + return DecimalType$.MODULE$.apply(decimal.precision(), decimal.scale()); + case UNKNOWN: + return NullType$.MODULE$; + default: + throw new UnsupportedOperationException( + "Cannot convert unsupported type to Spark: " + primitive); + } + } + + private Metadata fieldMetadata(int fieldId) { + if (MetadataColumns.metadataFieldIds().contains(fieldId)) { + return new MetadataBuilder().putBoolean(METADATA_COL_ATTR_KEY, true).build(); + } + + return Metadata.empty(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSnapshotUpdateSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSnapshotUpdateSparkAction.java new file mode 100644 index 000000000000..b69b80a8d3a6 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSnapshotUpdateSparkAction.java @@ -0,0 +1,47 @@ +/* + * 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.actions; + +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.spark.sql.SparkSession; + +abstract class BaseSnapshotUpdateSparkAction extends BaseSparkAction { + + private final Map summary = Maps.newHashMap(); + + protected BaseSnapshotUpdateSparkAction(SparkSession spark) { + super(spark); + } + + public ThisT snapshotProperty(String property, String value) { + summary.put(property, value); + return self(); + } + + protected void commit(org.apache.iceberg.SnapshotUpdate update) { + summary.forEach(update::set); + update.commit(); + } + + protected Map commitSummary() { + return ImmutableMap.copyOf(summary); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java new file mode 100644 index 000000000000..a8e82d101fbf --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -0,0 +1,443 @@ +/* + * 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.actions; + +import static org.apache.iceberg.MetadataTableType.ALL_MANIFESTS; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.lit; + +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; +import java.util.function.Supplier; +import org.apache.iceberg.AllManifestsTable; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.ReachableFileUtil; +import org.apache.iceberg.StaticTableOperations; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.BulkDeletionFailureException; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.ClosingIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.SupportsBulkOperations; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.base.Splitter; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Iterators; +import org.apache.iceberg.relocated.com.google.common.collect.ListMultimap; +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.Multimaps; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.spark.JobGroupUtils; +import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.spark.source.SerializableTableWithSize; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +abstract class BaseSparkAction { + + protected static final String MANIFEST = "Manifest"; + protected static final String MANIFEST_LIST = "Manifest List"; + protected static final String STATISTICS_FILES = "Statistics Files"; + protected static final String OTHERS = "Others"; + + protected static final String FILE_PATH = "file_path"; + protected static final String LAST_MODIFIED = "last_modified"; + + protected static final Splitter COMMA_SPLITTER = Splitter.on(","); + protected static final Joiner COMMA_JOINER = Joiner.on(','); + + private static final Logger LOG = LoggerFactory.getLogger(BaseSparkAction.class); + private static final AtomicInteger JOB_COUNTER = new AtomicInteger(); + private static final int DELETE_NUM_RETRIES = 3; + private static final int DELETE_GROUP_SIZE = 100000; + + private final SparkSession spark; + private final JavaSparkContext sparkContext; + private final Map options = Maps.newHashMap(); + + protected BaseSparkAction(SparkSession spark) { + this.spark = spark; + this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } + + protected SparkSession spark() { + return spark; + } + + protected JavaSparkContext sparkContext() { + return sparkContext; + } + + protected abstract ThisT self(); + + public ThisT option(String name, String value) { + options.put(name, value); + return self(); + } + + public ThisT options(Map newOptions) { + options.putAll(newOptions); + return self(); + } + + protected Map options() { + return options; + } + + protected T withJobGroupInfo(JobGroupInfo info, Supplier supplier) { + return JobGroupUtils.withJobGroupInfo(sparkContext, info, supplier); + } + + protected JobGroupInfo newJobGroupInfo(String groupId, String desc) { + return new JobGroupInfo(groupId + "-" + JOB_COUNTER.incrementAndGet(), desc); + } + + protected Table newStaticTable(TableMetadata metadata, FileIO io) { + StaticTableOperations ops = new StaticTableOperations(metadata, io); + return new BaseTable(ops, metadata.metadataFileLocation()); + } + + protected Table newStaticTable(String metadataFileLocation, FileIO io) { + StaticTableOperations ops = new StaticTableOperations(metadataFileLocation, io); + return new BaseTable(ops, metadataFileLocation); + } + + protected Dataset contentFileDS(Table table) { + return contentFileDS(table, null); + } + + protected Dataset contentFileDS(Table table, Set snapshotIds) { + Table serializableTable = SerializableTableWithSize.copyOf(table); + Broadcast

    tableBroadcast = sparkContext.broadcast(serializableTable); + int numShufflePartitions = spark.sessionState().conf().numShufflePartitions(); + + Dataset manifestBeanDS = + manifestDF(table, snapshotIds) + .selectExpr( + "content", + "path", + "length", + "0 as sequenceNumber", + "partition_spec_id as partitionSpecId", + "added_snapshot_id as addedSnapshotId") + .dropDuplicates("path") + .repartition(numShufflePartitions) // avoid adaptive execution combining tasks + .as(ManifestFileBean.ENCODER); + + return manifestBeanDS.flatMap(new ReadManifest(tableBroadcast), FileInfo.ENCODER); + } + + protected Dataset manifestDS(Table table) { + return manifestDS(table, null); + } + + protected Dataset manifestDS(Table table, Set snapshotIds) { + return manifestDF(table, snapshotIds) + .select(col("path"), lit(MANIFEST).as("type")) + .as(FileInfo.ENCODER); + } + + private Dataset manifestDF(Table table, Set snapshotIds) { + Dataset manifestDF = loadMetadataTable(table, ALL_MANIFESTS); + if (snapshotIds != null) { + Column filterCond = col(AllManifestsTable.REF_SNAPSHOT_ID.name()).isInCollection(snapshotIds); + return manifestDF.filter(filterCond); + } else { + return manifestDF; + } + } + + protected Dataset manifestListDS(Table table) { + return manifestListDS(table, null); + } + + protected Dataset manifestListDS(Table table, Set snapshotIds) { + List manifestLists = ReachableFileUtil.manifestListLocations(table, snapshotIds); + return toFileInfoDS(manifestLists, MANIFEST_LIST); + } + + protected Dataset statisticsFileDS(Table table, Set snapshotIds) { + List statisticsFiles = + ReachableFileUtil.statisticsFilesLocationsForSnapshots(table, snapshotIds); + return toFileInfoDS(statisticsFiles, STATISTICS_FILES); + } + + protected Dataset otherMetadataFileDS(Table table) { + return otherMetadataFileDS(table, false /* include all reachable old metadata locations */); + } + + protected Dataset allReachableOtherMetadataFileDS(Table table) { + return otherMetadataFileDS(table, true /* include all reachable old metadata locations */); + } + + private Dataset otherMetadataFileDS(Table table, boolean recursive) { + List otherMetadataFiles = Lists.newArrayList(); + otherMetadataFiles.addAll(ReachableFileUtil.metadataFileLocations(table, recursive)); + otherMetadataFiles.add(ReachableFileUtil.versionHintLocation(table)); + otherMetadataFiles.addAll(ReachableFileUtil.statisticsFilesLocations(table)); + return toFileInfoDS(otherMetadataFiles, OTHERS); + } + + protected Dataset loadMetadataTable(Table table, MetadataTableType type) { + return SparkTableUtil.loadMetadataTable(spark, table, type); + } + + private Dataset toFileInfoDS(List paths, String type) { + List fileInfoList = Lists.transform(paths, path -> new FileInfo(path, type)); + return spark.createDataset(fileInfoList, FileInfo.ENCODER); + } + + /** + * Deletes files and keeps track of how many files were removed for each file type. + * + * @param executorService an executor service to use for parallel deletes + * @param deleteFunc a delete func + * @param files an iterator of Spark rows of the structure (path: String, type: String) + * @return stats on which files were deleted + */ + protected DeleteSummary deleteFiles( + ExecutorService executorService, Consumer deleteFunc, Iterator files) { + + DeleteSummary summary = new DeleteSummary(); + + Tasks.foreach(files) + .retry(DELETE_NUM_RETRIES) + .stopRetryOn(NotFoundException.class) + .suppressFailureWhenFinished() + .executeWith(executorService) + .onFailure( + (fileInfo, exc) -> { + String path = fileInfo.getPath(); + String type = fileInfo.getType(); + LOG.warn("Delete failed for {}: {}", type, path, exc); + }) + .run( + fileInfo -> { + String path = fileInfo.getPath(); + String type = fileInfo.getType(); + deleteFunc.accept(path); + summary.deletedFile(path, type); + }); + + return summary; + } + + protected DeleteSummary deleteFiles(SupportsBulkOperations io, Iterator files) { + DeleteSummary summary = new DeleteSummary(); + Iterator> fileGroups = Iterators.partition(files, DELETE_GROUP_SIZE); + + Tasks.foreach(fileGroups) + .suppressFailureWhenFinished() + .run(fileGroup -> deleteFileGroup(fileGroup, io, summary)); + + return summary; + } + + private static void deleteFileGroup( + List fileGroup, SupportsBulkOperations io, DeleteSummary summary) { + + ListMultimap filesByType = Multimaps.index(fileGroup, FileInfo::getType); + ListMultimap pathsByType = + Multimaps.transformValues(filesByType, FileInfo::getPath); + + for (Map.Entry> entry : pathsByType.asMap().entrySet()) { + String type = entry.getKey(); + Collection paths = entry.getValue(); + int failures = 0; + try { + io.deleteFiles(paths); + } catch (BulkDeletionFailureException e) { + failures = e.numberFailedObjects(); + } + summary.deletedFiles(type, paths.size() - failures); + } + } + + static class DeleteSummary { + private final AtomicLong dataFilesCount = new AtomicLong(0L); + private final AtomicLong positionDeleteFilesCount = new AtomicLong(0L); + private final AtomicLong equalityDeleteFilesCount = new AtomicLong(0L); + private final AtomicLong manifestsCount = new AtomicLong(0L); + private final AtomicLong manifestListsCount = new AtomicLong(0L); + private final AtomicLong statisticsFilesCount = new AtomicLong(0L); + private final AtomicLong otherFilesCount = new AtomicLong(0L); + + public void deletedFiles(String type, int numFiles) { + if (FileContent.DATA.name().equalsIgnoreCase(type)) { + dataFilesCount.addAndGet(numFiles); + + } else if (FileContent.POSITION_DELETES.name().equalsIgnoreCase(type)) { + positionDeleteFilesCount.addAndGet(numFiles); + + } else if (FileContent.EQUALITY_DELETES.name().equalsIgnoreCase(type)) { + equalityDeleteFilesCount.addAndGet(numFiles); + + } else if (MANIFEST.equalsIgnoreCase(type)) { + manifestsCount.addAndGet(numFiles); + + } else if (MANIFEST_LIST.equalsIgnoreCase(type)) { + manifestListsCount.addAndGet(numFiles); + + } else if (STATISTICS_FILES.equalsIgnoreCase(type)) { + statisticsFilesCount.addAndGet(numFiles); + + } else if (OTHERS.equalsIgnoreCase(type)) { + otherFilesCount.addAndGet(numFiles); + + } else { + throw new ValidationException("Illegal file type: %s", type); + } + } + + public void deletedFile(String path, String type) { + if (FileContent.DATA.name().equalsIgnoreCase(type)) { + dataFilesCount.incrementAndGet(); + LOG.trace("Deleted data file: {}", path); + + } else if (FileContent.POSITION_DELETES.name().equalsIgnoreCase(type)) { + positionDeleteFilesCount.incrementAndGet(); + LOG.trace("Deleted positional delete file: {}", path); + + } else if (FileContent.EQUALITY_DELETES.name().equalsIgnoreCase(type)) { + equalityDeleteFilesCount.incrementAndGet(); + LOG.trace("Deleted equality delete file: {}", path); + + } else if (MANIFEST.equalsIgnoreCase(type)) { + manifestsCount.incrementAndGet(); + LOG.debug("Deleted manifest: {}", path); + + } else if (MANIFEST_LIST.equalsIgnoreCase(type)) { + manifestListsCount.incrementAndGet(); + LOG.debug("Deleted manifest list: {}", path); + + } else if (STATISTICS_FILES.equalsIgnoreCase(type)) { + statisticsFilesCount.incrementAndGet(); + LOG.debug("Deleted statistics file: {}", path); + + } else if (OTHERS.equalsIgnoreCase(type)) { + otherFilesCount.incrementAndGet(); + LOG.debug("Deleted other metadata file: {}", path); + + } else { + throw new ValidationException("Illegal file type: %s", type); + } + } + + public long dataFilesCount() { + return dataFilesCount.get(); + } + + public long positionDeleteFilesCount() { + return positionDeleteFilesCount.get(); + } + + public long equalityDeleteFilesCount() { + return equalityDeleteFilesCount.get(); + } + + public long manifestsCount() { + return manifestsCount.get(); + } + + public long manifestListsCount() { + return manifestListsCount.get(); + } + + public long statisticsFilesCount() { + return statisticsFilesCount.get(); + } + + public long otherFilesCount() { + return otherFilesCount.get(); + } + + public long totalFilesCount() { + return dataFilesCount() + + positionDeleteFilesCount() + + equalityDeleteFilesCount() + + manifestsCount() + + manifestListsCount() + + statisticsFilesCount() + + otherFilesCount(); + } + } + + private static class ReadManifest implements FlatMapFunction { + private final Broadcast
    table; + + ReadManifest(Broadcast
    table) { + this.table = table; + } + + @Override + public Iterator call(ManifestFileBean manifest) { + return new ClosingIterator<>(entries(manifest)); + } + + public CloseableIterator entries(ManifestFileBean manifest) { + ManifestContent content = manifest.content(); + FileIO io = table.getValue().io(); + Map specs = table.getValue().specs(); + List proj = ImmutableList.of(DataFile.FILE_PATH.name(), DataFile.CONTENT.name()); + + switch (content) { + case DATA: + return CloseableIterator.transform( + ManifestFiles.read(manifest, io, specs).select(proj).iterator(), + ReadManifest::toFileInfo); + case DELETES: + return CloseableIterator.transform( + ManifestFiles.readDeleteManifest(manifest, io, specs).select(proj).iterator(), + ReadManifest::toFileInfo); + default: + throw new IllegalArgumentException("Unsupported manifest content type:" + content); + } + } + + static FileInfo toFileInfo(ContentFile file) { + return new FileInfo(file.location(), file.content().toString()); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseTableCreationSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseTableCreationSparkAction.java new file mode 100644 index 000000000000..0ca73bef4d7e --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseTableCreationSparkAction.java @@ -0,0 +1,182 @@ +/* + * 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.actions; + +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.mapping.MappingUtil; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +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.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.spark.source.StagedSparkTable; +import org.apache.iceberg.util.LocationUtil; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.catalog.CatalogUtils; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.StagingTableCatalog; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.V1Table; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.types.StructType; + +abstract class BaseTableCreationSparkAction extends BaseSparkAction { + private static final Set ALLOWED_SOURCES = + ImmutableSet.of("parquet", "avro", "orc", "hive"); + protected static final String LOCATION = "location"; + protected static final String ICEBERG_METADATA_FOLDER = "metadata"; + protected static final List EXCLUDED_PROPERTIES = + ImmutableList.of("path", "transient_lastDdlTime", "serialization.format"); + + // Source Fields + private final V1Table sourceTable; + private final CatalogTable sourceCatalogTable; + private final String sourceTableLocation; + private final TableCatalog sourceCatalog; + private final Identifier sourceTableIdent; + + // Optional Parameters for destination + private final Map additionalProperties = Maps.newHashMap(); + + BaseTableCreationSparkAction( + SparkSession spark, CatalogPlugin sourceCatalog, Identifier sourceTableIdent) { + super(spark); + + this.sourceCatalog = checkSourceCatalog(sourceCatalog); + this.sourceTableIdent = sourceTableIdent; + + try { + this.sourceTable = (V1Table) this.sourceCatalog.loadTable(sourceTableIdent); + this.sourceCatalogTable = sourceTable.v1Table(); + } catch (org.apache.spark.sql.catalyst.analysis.NoSuchTableException e) { + throw new NoSuchTableException("Cannot find source table '%s'", sourceTableIdent); + } catch (ClassCastException e) { + throw new IllegalArgumentException( + String.format("Cannot use non-v1 table '%s' as a source", sourceTableIdent), e); + } + validateSourceTable(); + + this.sourceTableLocation = + CatalogUtils.URIToString(sourceCatalogTable.storage().locationUri().get()); + } + + protected abstract TableCatalog checkSourceCatalog(CatalogPlugin catalog); + + protected abstract StagingTableCatalog destCatalog(); + + protected abstract Identifier destTableIdent(); + + protected abstract Map destTableProps(); + + protected String sourceTableLocation() { + return sourceTableLocation; + } + + protected CatalogTable v1SourceTable() { + return sourceCatalogTable; + } + + protected TableCatalog sourceCatalog() { + return sourceCatalog; + } + + protected Identifier sourceTableIdent() { + return sourceTableIdent; + } + + protected void setProperties(Map properties) { + additionalProperties.putAll(properties); + } + + protected void setProperty(String key, String value) { + additionalProperties.put(key, value); + } + + protected Map additionalProperties() { + return additionalProperties; + } + + private void validateSourceTable() { + String sourceTableProvider = sourceCatalogTable.provider().get().toLowerCase(Locale.ROOT); + Preconditions.checkArgument( + ALLOWED_SOURCES.contains(sourceTableProvider), + "Cannot create an Iceberg table from source provider: '%s'", + sourceTableProvider); + Preconditions.checkArgument( + !sourceCatalogTable.storage().locationUri().isEmpty(), + "Cannot create an Iceberg table from a source without an explicit location"); + } + + protected StagingTableCatalog checkDestinationCatalog(CatalogPlugin catalog) { + Preconditions.checkArgument( + catalog instanceof SparkSessionCatalog || catalog instanceof SparkCatalog, + "Cannot create Iceberg table in non-Iceberg Catalog. " + + "Catalog '%s' was of class '%s' but '%s' or '%s' are required", + catalog.name(), + catalog.getClass().getName(), + SparkSessionCatalog.class.getName(), + SparkCatalog.class.getName()); + + return (StagingTableCatalog) catalog; + } + + protected StagedSparkTable stageDestTable() { + try { + Map props = destTableProps(); + StructType schema = sourceTable.schema(); + Transform[] partitioning = sourceTable.partitioning(); + return (StagedSparkTable) + destCatalog().stageCreate(destTableIdent(), schema, partitioning, props); + } catch (org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException e) { + throw new NoSuchNamespaceException( + "Cannot create table %s as the namespace does not exist", destTableIdent()); + } catch (org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException e) { + throw new AlreadyExistsException( + "Cannot create table %s as it already exists", destTableIdent()); + } + } + + protected void ensureNameMappingPresent(Table table) { + if (!table.properties().containsKey(TableProperties.DEFAULT_NAME_MAPPING)) { + NameMapping nameMapping = MappingUtil.create(table.schema()); + String nameMappingJson = NameMappingParser.toJson(nameMapping); + table.updateProperties().set(TableProperties.DEFAULT_NAME_MAPPING, nameMappingJson).commit(); + } + } + + protected String getMetadataLocation(Table table) { + String defaultValue = + LocationUtil.stripTrailingSlash(table.location()) + "/" + ICEBERG_METADATA_FOLDER; + return LocationUtil.stripTrailingSlash( + table.properties().getOrDefault(TableProperties.WRITE_METADATA_LOCATION, defaultValue)); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ComputePartitionStatsSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ComputePartitionStatsSparkAction.java new file mode 100644 index 000000000000..d46fa86c125f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ComputePartitionStatsSparkAction.java @@ -0,0 +1,103 @@ +/* + * 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.actions; + +import java.io.IOException; +import org.apache.iceberg.PartitionStatisticsFile; +import org.apache.iceberg.PartitionStatsHandler; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ComputePartitionStats; +import org.apache.iceberg.actions.ImmutableComputePartitionStats; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Computes the stats incrementally after the snapshot that has partition stats file till the given + * snapshot (uses current snapshot if not specified) and writes the combined result into a {@link + * PartitionStatisticsFile} after merging the stats for a given snapshot. Does a full compute if + * previous statistics file does not exist. Also registers the {@link PartitionStatisticsFile} to + * table metadata. + */ +public class ComputePartitionStatsSparkAction + extends BaseSparkAction implements ComputePartitionStats { + + private static final Logger LOG = LoggerFactory.getLogger(ComputePartitionStatsSparkAction.class); + private static final Result EMPTY_RESULT = + ImmutableComputePartitionStats.Result.builder().build(); + + private final Table table; + private Snapshot snapshot; + + ComputePartitionStatsSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.snapshot = table.currentSnapshot(); + } + + @Override + protected ComputePartitionStatsSparkAction self() { + return this; + } + + @Override + public ComputePartitionStats snapshot(long newSnapshotId) { + Snapshot newSnapshot = table.snapshot(newSnapshotId); + Preconditions.checkArgument(newSnapshot != null, "Snapshot not found: %s", newSnapshotId); + this.snapshot = newSnapshot; + return this; + } + + @Override + public Result execute() { + if (snapshot == null) { + LOG.info("No snapshot to compute partition stats for table {}", table.name()); + return EMPTY_RESULT; + } + + JobGroupInfo info = newJobGroupInfo("COMPUTE-PARTITION-STATS", jobDesc()); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + LOG.info("Computing partition stats for {} (snapshot {})", table.name(), snapshot.snapshotId()); + PartitionStatisticsFile statisticsFile; + try { + statisticsFile = PartitionStatsHandler.computeAndWriteStatsFile(table, snapshot.snapshotId()); + } catch (IOException e) { + throw new RuntimeIOException(e); + } + + if (statisticsFile == null) { + return EMPTY_RESULT; + } + + table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); + return ImmutableComputePartitionStats.Result.builder().statisticsFile(statisticsFile).build(); + } + + private String jobDesc() { + return String.format( + "Computing partition stats for %s (snapshot=%s)", table.name(), snapshot.snapshotId()); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java new file mode 100644 index 000000000000..1a0c022ad210 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java @@ -0,0 +1,179 @@ +/* + * 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.actions; + +import java.io.IOException; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.IcebergBuild; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.actions.ImmutableComputeTableStats; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; +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.ImmutableSet; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Computes the statistics of the given columns and stores it as Puffin files. */ +public class ComputeTableStatsSparkAction extends BaseSparkAction + implements ComputeTableStats { + + private static final Logger LOG = LoggerFactory.getLogger(ComputeTableStatsSparkAction.class); + private static final Result EMPTY_RESULT = ImmutableComputeTableStats.Result.builder().build(); + + private final Table table; + private List columns; + private Snapshot snapshot; + + ComputeTableStatsSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.snapshot = table.currentSnapshot(); + } + + @Override + protected ComputeTableStatsSparkAction self() { + return this; + } + + @Override + public ComputeTableStats columns(String... newColumns) { + Preconditions.checkArgument( + newColumns != null && newColumns.length > 0, "Columns cannot be null/empty"); + this.columns = ImmutableList.copyOf(ImmutableSet.copyOf(newColumns)); + return this; + } + + @Override + public ComputeTableStats snapshot(long newSnapshotId) { + Snapshot newSnapshot = table.snapshot(newSnapshotId); + Preconditions.checkArgument(newSnapshot != null, "Snapshot not found: %s", newSnapshotId); + this.snapshot = newSnapshot; + return this; + } + + @Override + public Result execute() { + if (snapshot == null) { + LOG.info("No snapshot to compute stats for table {}", table.name()); + return EMPTY_RESULT; + } + validateColumns(); + JobGroupInfo info = newJobGroupInfo("COMPUTE-TABLE-STATS", jobDesc()); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + LOG.info( + "Computing stats for columns {} in {} (snapshot {})", + columns(), + table.name(), + snapshotId()); + List blobs = generateNDVBlobs(); + StatisticsFile statisticsFile = writeStatsFile(blobs); + table.updateStatistics().setStatistics(statisticsFile).commit(); + return ImmutableComputeTableStats.Result.builder().statisticsFile(statisticsFile).build(); + } + + private StatisticsFile writeStatsFile(List blobs) { + LOG.info("Writing stats for table {} for snapshot {}", table.name(), snapshotId()); + OutputFile outputFile = table.io().newOutputFile(outputPath()); + try (PuffinWriter writer = Puffin.write(outputFile).createdBy(appIdentifier()).build()) { + blobs.forEach(writer::add); + writer.finish(); + return new GenericStatisticsFile( + snapshotId(), + outputFile.location(), + writer.fileSize(), + writer.footerSize(), + GenericBlobMetadata.from(writer.writtenBlobsMetadata())); + } catch (IOException e) { + throw new RuntimeIOException(e); + } + } + + private List generateNDVBlobs() { + return NDVSketchUtil.generateBlobs(spark(), table, snapshot, columns()); + } + + private List columns() { + if (columns == null) { + Schema schema = table.schemas().get(snapshot.schemaId()); + this.columns = + schema.columns().stream() + .filter(nestedField -> nestedField.type().isPrimitiveType()) + .map(Types.NestedField::name) + .collect(Collectors.toList()); + } + return columns; + } + + private void validateColumns() { + Schema schema = table.schemas().get(snapshot.schemaId()); + Preconditions.checkArgument(!columns().isEmpty(), "No columns found to compute stats"); + for (String columnName : columns()) { + Types.NestedField field = schema.findField(columnName); + Preconditions.checkArgument(field != null, "Can't find column %s in %s", columnName, schema); + Preconditions.checkArgument( + field.type().isPrimitiveType(), + "Can't compute stats on non-primitive type column: %s (%s)", + columnName, + field.type()); + } + } + + private String appIdentifier() { + String icebergVersion = IcebergBuild.fullVersion(); + String sparkVersion = spark().version(); + return String.format("Iceberg %s Spark %s", icebergVersion, sparkVersion); + } + + private long snapshotId() { + return snapshot.snapshotId(); + } + + private String jobDesc() { + return String.format( + "Computing table stats for %s (snapshot_id=%s, columns=%s)", + table.name(), snapshotId(), columns()); + } + + private String outputPath() { + TableOperations operations = ((HasTableOperations) table).operations(); + String fileName = String.format("%s-%s.stats", snapshotId(), UUID.randomUUID()); + return operations.metadataFileLocation(fileName); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java new file mode 100644 index 000000000000..78662159b0bb --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -0,0 +1,620 @@ +/* + * 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.actions; + +import static org.apache.iceberg.TableProperties.GC_ENABLED; +import static org.apache.iceberg.TableProperties.GC_ENABLED_DEFAULT; + +import java.net.URI; +import java.sql.Timestamp; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.function.Predicate; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.actions.FileURI; +import org.apache.iceberg.actions.ImmutableDeleteOrphanFiles; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.BulkDeletionFailureException; +import org.apache.iceberg.io.SupportsBulkOperations; +import org.apache.iceberg.io.SupportsPrefixOperations; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +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.Iterators; +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.JobGroupInfo; +import org.apache.iceberg.util.FileSystemWalker; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.MapPartitionsFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.util.SerializableConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Tuple2; + +/** + * An action that removes orphan metadata, data and delete files by listing a given location and + * comparing the actual files in that location with content and metadata files referenced by all + * valid snapshots. The location must be accessible for listing via the Hadoop {@link FileSystem}. + * + *

    By default, this action cleans up the table location returned by {@link Table#location()} and + * removes unreachable files that are older than 3 days using {@link Table#io()}. The behavior can + * be modified by passing a custom location to {@link #location} and a custom timestamp to {@link + * #olderThan(long)}. For example, someone might point this action to the data folder to clean up + * only orphan data files. + * + *

    Configure an alternative delete method using {@link #deleteWith(Consumer)}. + * + *

    For full control of the set of files being evaluated, use the {@link + * #compareToFileList(Dataset)} argument. This skips the directory listing - any files in the + * dataset provided which are not found in table metadata will be deleted, using the same {@link + * Table#location()} and {@link #olderThan(long)} filtering as above. + * + *

    Streaming mode can be enabled via the {@value #STREAM_RESULTS} option to avoid loading all + * orphan file paths into driver memory. When enabled, the result will contain only a sample of file + * paths (up to {@value #MAX_ORPHAN_FILE_SAMPLE_SIZE_DEFAULT}). The total count of deleted files is + * logged but not included in the result. + * + *

    Note: It is dangerous to call this action with a short retention interval as it might + * corrupt the state of the table if another operation is writing at the same time. + */ +public class DeleteOrphanFilesSparkAction extends BaseSparkAction + implements DeleteOrphanFiles { + + public static final String STREAM_RESULTS = "stream-results"; + public static final boolean STREAM_RESULTS_DEFAULT = false; + + // Test-only option to configure the max sample size for streaming mode + @VisibleForTesting + static final String MAX_ORPHAN_FILE_SAMPLE_SIZE = "max-orphan-file-sample-size"; + + private static final int MAX_ORPHAN_FILE_SAMPLE_SIZE_DEFAULT = 20000; + + private static final Logger LOG = LoggerFactory.getLogger(DeleteOrphanFilesSparkAction.class); + private static final Map EQUAL_SCHEMES_DEFAULT = ImmutableMap.of("s3n,s3a", "s3"); + private static final int MAX_DRIVER_LISTING_DEPTH = 3; + private static final int MAX_DRIVER_LISTING_DIRECT_SUB_DIRS = 10; + private static final int MAX_EXECUTOR_LISTING_DEPTH = 2000; + private static final int MAX_EXECUTOR_LISTING_DIRECT_SUB_DIRS = Integer.MAX_VALUE; + private static final int DELETE_GROUP_SIZE = 100000; + + private final SerializableConfiguration hadoopConf; + private final int listingParallelism; + private final Table table; + private Map equalSchemes = flattenMap(EQUAL_SCHEMES_DEFAULT); + private Map equalAuthorities = Collections.emptyMap(); + private PrefixMismatchMode prefixMismatchMode = PrefixMismatchMode.ERROR; + private String location; + private long olderThanTimestamp = System.currentTimeMillis() - TimeUnit.DAYS.toMillis(3); + private Dataset compareToFileList; + private Consumer deleteFunc = null; + private ExecutorService deleteExecutorService = null; + private boolean usePrefixListing = false; + private static final Encoder FILE_URI_ENCODER = Encoders.bean(FileURI.class); + + DeleteOrphanFilesSparkAction(SparkSession spark, Table table) { + super(spark); + + this.hadoopConf = new SerializableConfiguration(spark.sessionState().newHadoopConf()); + this.listingParallelism = spark.sessionState().conf().parallelPartitionDiscoveryParallelism(); + this.table = table; + this.location = table.location(); + + ValidationException.check( + PropertyUtil.propertyAsBoolean(table.properties(), GC_ENABLED, GC_ENABLED_DEFAULT), + "Cannot delete orphan files: GC is disabled (deleting files may corrupt other tables)"); + } + + @Override + protected DeleteOrphanFilesSparkAction self() { + return this; + } + + @Override + public DeleteOrphanFilesSparkAction executeDeleteWith(ExecutorService executorService) { + this.deleteExecutorService = executorService; + return this; + } + + @Override + public DeleteOrphanFilesSparkAction prefixMismatchMode(PrefixMismatchMode newPrefixMismatchMode) { + this.prefixMismatchMode = newPrefixMismatchMode; + return this; + } + + @Override + public DeleteOrphanFilesSparkAction equalSchemes(Map newEqualSchemes) { + this.equalSchemes = Maps.newHashMap(); + equalSchemes.putAll(flattenMap(EQUAL_SCHEMES_DEFAULT)); + equalSchemes.putAll(flattenMap(newEqualSchemes)); + return this; + } + + @Override + public DeleteOrphanFilesSparkAction equalAuthorities(Map newEqualAuthorities) { + this.equalAuthorities = Maps.newHashMap(); + equalAuthorities.putAll(flattenMap(newEqualAuthorities)); + return this; + } + + @Override + public DeleteOrphanFilesSparkAction location(String newLocation) { + this.location = newLocation; + return this; + } + + @Override + public DeleteOrphanFilesSparkAction olderThan(long newOlderThanTimestamp) { + this.olderThanTimestamp = newOlderThanTimestamp; + return this; + } + + @Override + public DeleteOrphanFilesSparkAction deleteWith(Consumer newDeleteFunc) { + this.deleteFunc = newDeleteFunc; + return this; + } + + public DeleteOrphanFilesSparkAction compareToFileList(Dataset files) { + StructType schema = files.schema(); + + StructField filePathField = schema.apply(FILE_PATH); + Preconditions.checkArgument( + filePathField.dataType() == DataTypes.StringType, + "Invalid %s column: %s is not a string", + FILE_PATH, + filePathField.dataType()); + + StructField lastModifiedField = schema.apply(LAST_MODIFIED); + Preconditions.checkArgument( + lastModifiedField.dataType() == DataTypes.TimestampType, + "Invalid %s column: %s is not a timestamp", + LAST_MODIFIED, + lastModifiedField.dataType()); + + this.compareToFileList = files; + return this; + } + + public DeleteOrphanFilesSparkAction usePrefixListing(boolean newUsePrefixListing) { + this.usePrefixListing = newUsePrefixListing; + return this; + } + + private Dataset filteredCompareToFileList() { + Dataset files = compareToFileList; + if (location != null) { + files = files.filter(files.col(FILE_PATH).startsWith(location)); + } + return files + .filter(files.col(LAST_MODIFIED).lt(new Timestamp(olderThanTimestamp))) + .select(files.col(FILE_PATH)) + .as(Encoders.STRING()); + } + + @Override + public DeleteOrphanFiles.Result execute() { + JobGroupInfo info = newJobGroupInfo("DELETE-ORPHAN-FILES", jobDesc()); + return withJobGroupInfo(info, this::doExecute); + } + + private String jobDesc() { + List options = Lists.newArrayList(); + options.add("older_than=" + olderThanTimestamp); + if (location != null) { + options.add("location=" + location); + } + String optionsAsString = COMMA_JOINER.join(options); + return String.format("Deleting orphan files (%s) from %s", optionsAsString, table.name()); + } + + private boolean streamResults() { + return PropertyUtil.propertyAsBoolean(options(), STREAM_RESULTS, STREAM_RESULTS_DEFAULT); + } + + private DeleteOrphanFiles.Result doExecute() { + Dataset actualFileIdentDS = actualFileIdentDS(); + Dataset validFileIdentDS = validFileIdentDS(); + + Dataset orphanFileDS = + findOrphanFiles(actualFileIdentDS, validFileIdentDS, prefixMismatchMode); + try { + return deleteFiles(orphanFileDS); + } finally { + orphanFileDS.unpersist(); + } + } + + /** + * Deletes orphan files from the cached dataset. + * + * @param orphanFileDS the cached dataset of orphan files + * @return result with orphan file paths + */ + private DeleteOrphanFiles.Result deleteFiles(Dataset orphanFileDS) { + int maxSampleSize = + PropertyUtil.propertyAsInt( + options(), MAX_ORPHAN_FILE_SAMPLE_SIZE, MAX_ORPHAN_FILE_SAMPLE_SIZE_DEFAULT); + List orphanFileList = Lists.newArrayListWithCapacity(maxSampleSize); + long filesCount = 0; + + Iterator orphanFiles = + streamResults() ? orphanFileDS.toLocalIterator() : orphanFileDS.collectAsList().iterator(); + + Iterator> fileGroups = Iterators.partition(orphanFiles, DELETE_GROUP_SIZE); + + while (fileGroups.hasNext()) { + List fileGroup = fileGroups.next(); + + collectPathsForOutput(fileGroup, orphanFileList, maxSampleSize); + + if (deleteFunc == null && table.io() instanceof SupportsBulkOperations) { + deleteBulk((SupportsBulkOperations) table.io(), fileGroup); + } else { + deleteNonBulk(fileGroup); + } + + filesCount += fileGroup.size(); + } + + LOG.info("Deleted {} orphan files", filesCount); + + return ImmutableDeleteOrphanFiles.Result.builder().orphanFileLocations(orphanFileList).build(); + } + + private void collectPathsForOutput( + List paths, List orphanFileList, int maxSampleSize) { + if (streamResults()) { + int lengthToAdd = Math.min(maxSampleSize - orphanFileList.size(), paths.size()); + orphanFileList.addAll(paths.subList(0, lengthToAdd)); + } else { + orphanFileList.addAll(paths); + } + } + + private void deleteBulk(SupportsBulkOperations io, List paths) { + try { + io.deleteFiles(paths); + LOG.info("Deleted {} files using bulk deletes", paths.size()); + } catch (BulkDeletionFailureException e) { + int deletedFilesCount = paths.size() - e.numberFailedObjects(); + LOG.warn( + "Deleted only {} of {} files using bulk deletes", deletedFilesCount, paths.size(), e); + } + } + + private void deleteNonBulk(List paths) { + Tasks.Builder deleteTasks = + Tasks.foreach(paths) + .noRetry() + .executeWith(deleteExecutorService) + .suppressFailureWhenFinished() + .onFailure((file, exc) -> LOG.warn("Failed to delete file: {}", file, exc)); + + if (deleteFunc == null) { + LOG.info( + "Table IO {} does not support bulk operations. Using non-bulk deletes.", + table.io().getClass().getName()); + deleteTasks.run(table.io()::deleteFile); + } else { + LOG.info("Custom delete function provided. Using non-bulk deletes"); + deleteTasks.run(deleteFunc::accept); + } + } + + @VisibleForTesting + static Dataset findOrphanFiles( + Dataset actualFileIdentDS, + Dataset validFileIdentDS, + PrefixMismatchMode prefixMismatchMode) { + + SetAccumulator> conflicts = new SetAccumulator<>(); + actualFileIdentDS.sparkSession().sparkContext().register(conflicts); + + Column joinCond = actualFileIdentDS.col("path").equalTo(validFileIdentDS.col("path")); + + Dataset orphanFileDS = + actualFileIdentDS + .joinWith(validFileIdentDS, joinCond, "leftouter") + .mapPartitions(new FindOrphanFiles(prefixMismatchMode, conflicts), Encoders.STRING()); + + // Cache and force computation to populate conflicts accumulator + orphanFileDS = orphanFileDS.cache(); + + try { + orphanFileDS.count(); + + if (prefixMismatchMode == PrefixMismatchMode.ERROR && !conflicts.value().isEmpty()) { + throw new ValidationException( + "Unable to determine whether certain files are orphan. Metadata references files that" + + " match listed/provided files except for authority/scheme. Please, inspect the" + + " conflicting authorities/schemes and provide which of them are equal by further" + + " configuring the action via equalSchemes() and equalAuthorities() methods. Set the" + + " prefix mismatch mode to 'NONE' to ignore remaining locations with conflicting" + + " authorities/schemes or to 'DELETE' iff you are ABSOLUTELY confident that" + + " remaining conflicting authorities/schemes are different. It will be impossible to" + + " recover deleted files. Conflicting authorities/schemes: %s.", + conflicts.value()); + } + + return orphanFileDS; + } catch (Exception e) { + orphanFileDS.unpersist(); + throw e; + } + } + + private Dataset validFileIdentDS() { + // transform before union to avoid extra serialization/deserialization + FileInfoToFileURI toFileURI = new FileInfoToFileURI(equalSchemes, equalAuthorities); + + Dataset contentFileIdentDS = toFileURI.apply(contentFileDS(table)); + Dataset manifestFileIdentDS = toFileURI.apply(manifestDS(table)); + Dataset manifestListIdentDS = toFileURI.apply(manifestListDS(table)); + Dataset otherMetadataFileIdentDS = toFileURI.apply(otherMetadataFileDS(table)); + + return contentFileIdentDS + .union(manifestFileIdentDS) + .union(manifestListIdentDS) + .union(otherMetadataFileIdentDS); + } + + private Dataset actualFileIdentDS() { + StringToFileURI toFileURI = new StringToFileURI(equalSchemes, equalAuthorities); + if (compareToFileList == null) { + return toFileURI.apply(listedFileDS()); + } else { + return toFileURI.apply(filteredCompareToFileList()); + } + } + + private Dataset listedFileDS() { + List subDirs = Lists.newArrayList(); + List matchingFiles = Lists.newArrayList(); + + if (usePrefixListing) { + Preconditions.checkArgument( + table.io() instanceof SupportsPrefixOperations, + "Cannot use prefix listing with FileIO {} which does not support prefix operations.", + table.io()); + + Predicate predicate = + fileInfo -> fileInfo.createdAtMillis() < olderThanTimestamp; + FileSystemWalker.listDirRecursivelyWithFileIO( + (SupportsPrefixOperations) table.io(), + location, + table.specs(), + predicate, + matchingFiles::add); + + JavaRDD matchingFileRDD = sparkContext().parallelize(matchingFiles, 1); + return spark().createDataset(matchingFileRDD.rdd(), Encoders.STRING()); + } else { + Predicate predicate = file -> file.getModificationTime() < olderThanTimestamp; + // list at most MAX_DRIVER_LISTING_DEPTH levels and only dirs that have + // less than MAX_DRIVER_LISTING_DIRECT_SUB_DIRS direct sub dirs on the driver + FileSystemWalker.listDirRecursivelyWithHadoop( + location, + table.specs(), + predicate, + hadoopConf.value(), + MAX_DRIVER_LISTING_DEPTH, + MAX_DRIVER_LISTING_DIRECT_SUB_DIRS, + subDirs::add, + matchingFiles::add); + + JavaRDD matchingFileRDD = sparkContext().parallelize(matchingFiles, 1); + + if (subDirs.isEmpty()) { + return spark().createDataset(matchingFileRDD.rdd(), Encoders.STRING()); + } + + int parallelism = Math.min(subDirs.size(), listingParallelism); + JavaRDD subDirRDD = sparkContext().parallelize(subDirs, parallelism); + + Broadcast conf = sparkContext().broadcast(hadoopConf); + ListDirsRecursively listDirs = + new ListDirsRecursively(conf, olderThanTimestamp, table.specs()); + JavaRDD matchingLeafFileRDD = subDirRDD.mapPartitions(listDirs); + + JavaRDD completeMatchingFileRDD = matchingFileRDD.union(matchingLeafFileRDD); + return spark().createDataset(completeMatchingFileRDD.rdd(), Encoders.STRING()); + } + } + + private static Map flattenMap(Map map) { + Map flattenedMap = Maps.newHashMap(); + if (map != null) { + for (String key : map.keySet()) { + String value = map.get(key); + for (String splitKey : COMMA_SPLITTER.split(key)) { + flattenedMap.put(splitKey.trim(), value.trim()); + } + } + } + return flattenedMap; + } + + private static class ListDirsRecursively implements FlatMapFunction, String> { + + private final Broadcast hadoopConf; + private final long olderThanTimestamp; + private final Map specs; + + ListDirsRecursively( + Broadcast hadoopConf, + long olderThanTimestamp, + Map specs) { + + this.hadoopConf = hadoopConf; + this.olderThanTimestamp = olderThanTimestamp; + this.specs = specs; + } + + @Override + public Iterator call(Iterator dirs) throws Exception { + List subDirs = Lists.newArrayList(); + List files = Lists.newArrayList(); + + Predicate predicate = file -> file.getModificationTime() < olderThanTimestamp; + + while (dirs.hasNext()) { + FileSystemWalker.listDirRecursivelyWithHadoop( + dirs.next(), + specs, + predicate, + hadoopConf.value().value(), + MAX_EXECUTOR_LISTING_DEPTH, + MAX_EXECUTOR_LISTING_DIRECT_SUB_DIRS, + subDirs::add, + files::add); + } + + if (!subDirs.isEmpty()) { + throw new RuntimeException( + "Could not list sub directories, reached maximum depth: " + MAX_EXECUTOR_LISTING_DEPTH); + } + + return files.iterator(); + } + } + + private static class FindOrphanFiles + implements MapPartitionsFunction, String> { + + private final PrefixMismatchMode mode; + private final SetAccumulator> conflicts; + + FindOrphanFiles(PrefixMismatchMode mode, SetAccumulator> conflicts) { + this.mode = mode; + this.conflicts = conflicts; + } + + @Override + public Iterator call(Iterator> rows) throws Exception { + Iterator orphanFiles = Iterators.transform(rows, this::toOrphanFile); + return Iterators.filter(orphanFiles, Objects::nonNull); + } + + private String toOrphanFile(Tuple2 row) { + FileURI actual = row._1; + FileURI valid = row._2; + + if (valid == null) { + return actual.getUriAsString(); + } + + boolean schemeMatch = valid.schemeMatch(actual); + boolean authorityMatch = valid.authorityMatch(actual); + + if ((!schemeMatch || !authorityMatch) && mode == PrefixMismatchMode.DELETE) { + return actual.getUriAsString(); + } else { + if (!schemeMatch) { + conflicts.add(Pair.of(valid.getScheme(), actual.getScheme())); + } + + if (!authorityMatch) { + conflicts.add(Pair.of(valid.getAuthority(), actual.getAuthority())); + } + + return null; + } + } + } + + @VisibleForTesting + static class StringToFileURI extends ToFileURI { + StringToFileURI(Map equalSchemes, Map equalAuthorities) { + super(equalSchemes, equalAuthorities); + } + + @Override + protected String uriAsString(String input) { + return input; + } + } + + @VisibleForTesting + static class FileInfoToFileURI extends ToFileURI { + FileInfoToFileURI(Map equalSchemes, Map equalAuthorities) { + super(equalSchemes, equalAuthorities); + } + + @Override + protected String uriAsString(FileInfo fileInfo) { + return fileInfo.getPath(); + } + } + + private abstract static class ToFileURI implements MapPartitionsFunction { + + private final Map equalSchemes; + private final Map equalAuthorities; + + ToFileURI(Map equalSchemes, Map equalAuthorities) { + this.equalSchemes = equalSchemes; + this.equalAuthorities = equalAuthorities; + } + + protected abstract String uriAsString(I input); + + Dataset apply(Dataset ds) { + return ds.mapPartitions(this, FILE_URI_ENCODER); + } + + @Override + public Iterator call(Iterator rows) throws Exception { + return Iterators.transform(rows, this::toFileURI); + } + + private FileURI toFileURI(I input) { + String uriAsString = uriAsString(input); + URI uri = new Path(uriAsString).toUri(); + String scheme = equalSchemes.getOrDefault(uri.getScheme(), uri.getScheme()); + String authority = equalAuthorities.getOrDefault(uri.getAuthority(), uri.getAuthority()); + return new FileURI(scheme, authority, uri.getPath(), uriAsString); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteReachableFilesSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteReachableFilesSparkAction.java new file mode 100644 index 000000000000..ea6ac9f3dbf5 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteReachableFilesSparkAction.java @@ -0,0 +1,156 @@ +/* + * 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.actions; + +import static org.apache.iceberg.TableProperties.GC_ENABLED; +import static org.apache.iceberg.TableProperties.GC_ENABLED_DEFAULT; + +import java.util.Iterator; +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.actions.DeleteReachableFiles; +import org.apache.iceberg.actions.ImmutableDeleteReachableFiles; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.SupportsBulkOperations; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An implementation of {@link DeleteReachableFiles} that uses metadata tables in Spark to determine + * which files should be deleted. + */ +@SuppressWarnings("UnnecessaryAnonymousClass") +public class DeleteReachableFilesSparkAction + extends BaseSparkAction implements DeleteReachableFiles { + + public static final String STREAM_RESULTS = "stream-results"; + public static final boolean STREAM_RESULTS_DEFAULT = false; + + private static final Logger LOG = LoggerFactory.getLogger(DeleteReachableFilesSparkAction.class); + + private final String metadataFileLocation; + + private Consumer deleteFunc = null; + private ExecutorService deleteExecutorService = null; + private FileIO io = new HadoopFileIO(spark().sessionState().newHadoopConf()); + + DeleteReachableFilesSparkAction(SparkSession spark, String metadataFileLocation) { + super(spark); + this.metadataFileLocation = metadataFileLocation; + } + + @Override + protected DeleteReachableFilesSparkAction self() { + return this; + } + + @Override + public DeleteReachableFilesSparkAction io(FileIO fileIO) { + this.io = fileIO; + return this; + } + + @Override + public DeleteReachableFilesSparkAction deleteWith(Consumer newDeleteFunc) { + this.deleteFunc = newDeleteFunc; + return this; + } + + @Override + public DeleteReachableFilesSparkAction executeDeleteWith(ExecutorService executorService) { + this.deleteExecutorService = executorService; + return this; + } + + @Override + public Result execute() { + Preconditions.checkArgument(io != null, "File IO cannot be null"); + String jobDesc = String.format("Deleting files reachable from %s", metadataFileLocation); + JobGroupInfo info = newJobGroupInfo("DELETE-REACHABLE-FILES", jobDesc); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + TableMetadata metadata = TableMetadataParser.read(io, metadataFileLocation); + + ValidationException.check( + PropertyUtil.propertyAsBoolean(metadata.properties(), GC_ENABLED, GC_ENABLED_DEFAULT), + "Cannot delete files: GC is disabled (deleting files may corrupt other tables)"); + + Dataset reachableFileDS = reachableFileDS(metadata); + + if (streamResults()) { + return deleteFiles(reachableFileDS.toLocalIterator()); + } else { + return deleteFiles(reachableFileDS.collectAsList().iterator()); + } + } + + private boolean streamResults() { + return PropertyUtil.propertyAsBoolean(options(), STREAM_RESULTS, STREAM_RESULTS_DEFAULT); + } + + private Dataset reachableFileDS(TableMetadata metadata) { + Table staticTable = newStaticTable(metadata, io); + return contentFileDS(staticTable) + .union(manifestDS(staticTable)) + .union(manifestListDS(staticTable)) + .union(allReachableOtherMetadataFileDS(staticTable)) + .distinct(); + } + + private DeleteReachableFiles.Result deleteFiles(Iterator files) { + DeleteSummary summary; + if (deleteFunc == null && io instanceof SupportsBulkOperations) { + summary = deleteFiles((SupportsBulkOperations) io, files); + } else { + + if (deleteFunc == null) { + LOG.info( + "Table IO {} does not support bulk operations. Using non-bulk deletes.", + io.getClass().getName()); + summary = deleteFiles(deleteExecutorService, io::deleteFile, files); + } else { + LOG.info("Custom delete function provided. Using non-bulk deletes"); + summary = deleteFiles(deleteExecutorService, deleteFunc, files); + } + } + + LOG.info("Deleted {} total files", summary.totalFilesCount()); + + return ImmutableDeleteReachableFiles.Result.builder() + .deletedDataFilesCount(summary.dataFilesCount()) + .deletedPositionDeleteFilesCount(summary.positionDeleteFilesCount()) + .deletedEqualityDeleteFilesCount(summary.equalityDeleteFilesCount()) + .deletedManifestsCount(summary.manifestsCount()) + .deletedManifestListsCount(summary.manifestListsCount()) + .deletedOtherFilesCount(summary.otherFilesCount()) + .build(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java new file mode 100644 index 000000000000..e49e7326736f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ExpireSnapshotsSparkAction.java @@ -0,0 +1,286 @@ +/* + * 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.actions; + +import static org.apache.iceberg.TableProperties.GC_ENABLED; +import static org.apache.iceberg.TableProperties.GC_ENABLED_DEFAULT; + +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ExecutorService; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import org.apache.iceberg.ExpireSnapshots.CleanupLevel; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.actions.ExpireSnapshots; +import org.apache.iceberg.actions.ImmutableExpireSnapshots; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.SupportsBulkOperations; +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.JobGroupInfo; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An action that performs the same operation as {@link org.apache.iceberg.ExpireSnapshots} but uses + * Spark to determine the delta in files between the pre and post-expiration table metadata. All of + * the same restrictions of {@link org.apache.iceberg.ExpireSnapshots} also apply to this action. + * + *

    This action first leverages {@link org.apache.iceberg.ExpireSnapshots} to expire snapshots and + * then uses metadata tables to find files that can be safely deleted. This is done by anti-joining + * two Datasets that contain all manifest and content files before and after the expiration. The + * snapshot expiration will be fully committed before any deletes are issued. + * + *

    This operation performs a shuffle so the parallelism can be controlled through + * 'spark.sql.shuffle.partitions'. + * + *

    Deletes are still performed locally after retrieving the results from the Spark executors. + */ +@SuppressWarnings("UnnecessaryAnonymousClass") +public class ExpireSnapshotsSparkAction extends BaseSparkAction + implements ExpireSnapshots { + + public static final String STREAM_RESULTS = "stream-results"; + public static final boolean STREAM_RESULTS_DEFAULT = false; + + private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsSparkAction.class); + + private final Table table; + private final TableOperations ops; + + private final Set expiredSnapshotIds = Sets.newHashSet(); + private Long expireOlderThanValue = null; + private Integer retainLastValue = null; + private Consumer deleteFunc = null; + private ExecutorService deleteExecutorService = null; + private Dataset expiredFileDS = null; + private Boolean cleanExpiredMetadata = null; + + ExpireSnapshotsSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.ops = ((HasTableOperations) table).operations(); + + ValidationException.check( + PropertyUtil.propertyAsBoolean(table.properties(), GC_ENABLED, GC_ENABLED_DEFAULT), + "Cannot expire snapshots: GC is disabled (deleting files may corrupt other tables)"); + } + + @Override + protected ExpireSnapshotsSparkAction self() { + return this; + } + + @Override + public ExpireSnapshotsSparkAction executeDeleteWith(ExecutorService executorService) { + this.deleteExecutorService = executorService; + return this; + } + + @Override + public ExpireSnapshotsSparkAction expireSnapshotId(long snapshotId) { + expiredSnapshotIds.add(snapshotId); + return this; + } + + @Override + public ExpireSnapshotsSparkAction expireOlderThan(long timestampMillis) { + this.expireOlderThanValue = timestampMillis; + return this; + } + + @Override + public ExpireSnapshotsSparkAction retainLast(int numSnapshots) { + Preconditions.checkArgument( + 1 <= numSnapshots, + "Number of snapshots to retain must be at least 1, cannot be: %s", + numSnapshots); + this.retainLastValue = numSnapshots; + return this; + } + + @Override + public ExpireSnapshotsSparkAction deleteWith(Consumer newDeleteFunc) { + this.deleteFunc = newDeleteFunc; + return this; + } + + @Override + public ExpireSnapshotsSparkAction cleanExpiredMetadata(boolean clean) { + this.cleanExpiredMetadata = clean; + return this; + } + + /** + * Expires snapshots and commits the changes to the table, returning a Dataset of files to delete. + * + *

    This does not delete data files. To delete data files, run {@link #execute()}. + * + *

    This may be called before or after {@link #execute()} to return the expired files. + * + * @return a Dataset of files that are no longer referenced by the table + */ + public Dataset expireFiles() { + if (expiredFileDS == null) { + // fetch metadata before expiration + TableMetadata originalMetadata = ops.current(); + + // perform expiration + org.apache.iceberg.ExpireSnapshots expireSnapshots = table.expireSnapshots(); + + for (long id : expiredSnapshotIds) { + expireSnapshots = expireSnapshots.expireSnapshotId(id); + } + + if (expireOlderThanValue != null) { + expireSnapshots = expireSnapshots.expireOlderThan(expireOlderThanValue); + } + + if (retainLastValue != null) { + expireSnapshots = expireSnapshots.retainLast(retainLastValue); + } + + if (cleanExpiredMetadata != null) { + expireSnapshots.cleanExpiredMetadata(cleanExpiredMetadata); + } + + expireSnapshots.cleanupLevel(CleanupLevel.NONE).commit(); + + // fetch valid files after expiration + TableMetadata updatedMetadata = ops.refresh(); + Dataset validFileDS = fileDS(updatedMetadata); + + // fetch files referenced by expired snapshots + Set deletedSnapshotIds = findExpiredSnapshotIds(originalMetadata, updatedMetadata); + Dataset deleteCandidateFileDS = fileDS(originalMetadata, deletedSnapshotIds); + + // determine expired files + this.expiredFileDS = deleteCandidateFileDS.except(validFileDS); + } + + return expiredFileDS; + } + + @Override + public ExpireSnapshots.Result execute() { + JobGroupInfo info = newJobGroupInfo("EXPIRE-SNAPSHOTS", jobDesc()); + return withJobGroupInfo(info, this::doExecute); + } + + private String jobDesc() { + List options = Lists.newArrayList(); + + if (expireOlderThanValue != null) { + options.add("older_than=" + expireOlderThanValue); + } + + if (retainLastValue != null) { + options.add("retain_last=" + retainLastValue); + } + + if (!expiredSnapshotIds.isEmpty()) { + Long first = expiredSnapshotIds.stream().findFirst().get(); + if (expiredSnapshotIds.size() > 1) { + options.add( + String.format("snapshot_ids: %s (%s more...)", first, expiredSnapshotIds.size() - 1)); + } else { + options.add(String.format("snapshot_id: %s", first)); + } + } + + if (cleanExpiredMetadata != null) { + options.add("clean_expired_metadata=" + cleanExpiredMetadata); + } + + return String.format("Expiring snapshots (%s) in %s", COMMA_JOINER.join(options), table.name()); + } + + private ExpireSnapshots.Result doExecute() { + if (streamResults()) { + return deleteFiles(expireFiles().toLocalIterator()); + } else { + return deleteFiles(expireFiles().collectAsList().iterator()); + } + } + + private boolean streamResults() { + return PropertyUtil.propertyAsBoolean(options(), STREAM_RESULTS, STREAM_RESULTS_DEFAULT); + } + + private Dataset fileDS(TableMetadata metadata) { + return fileDS(metadata, null); + } + + private Dataset fileDS(TableMetadata metadata, Set snapshotIds) { + Table staticTable = newStaticTable(metadata, table.io()); + return contentFileDS(staticTable, snapshotIds) + .union(manifestDS(staticTable, snapshotIds)) + .union(manifestListDS(staticTable, snapshotIds)) + .union(statisticsFileDS(staticTable, snapshotIds)); + } + + private Set findExpiredSnapshotIds( + TableMetadata originalMetadata, TableMetadata updatedMetadata) { + Set retainedSnapshots = + updatedMetadata.snapshots().stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); + return originalMetadata.snapshots().stream() + .map(Snapshot::snapshotId) + .filter(id -> !retainedSnapshots.contains(id)) + .collect(Collectors.toSet()); + } + + private ExpireSnapshots.Result deleteFiles(Iterator files) { + DeleteSummary summary; + if (deleteFunc == null && table.io() instanceof SupportsBulkOperations) { + summary = deleteFiles((SupportsBulkOperations) table.io(), files); + } else { + + if (deleteFunc == null) { + LOG.info( + "Table IO {} does not support bulk operations. Using non-bulk deletes.", + table.io().getClass().getName()); + summary = deleteFiles(deleteExecutorService, table.io()::deleteFile, files); + } else { + LOG.info("Custom delete function provided. Using non-bulk deletes"); + summary = deleteFiles(deleteExecutorService, deleteFunc, files); + } + } + + LOG.info("Deleted {} total files", summary.totalFilesCount()); + + return ImmutableExpireSnapshots.Result.builder() + .deletedDataFilesCount(summary.dataFilesCount()) + .deletedPositionDeleteFilesCount(summary.positionDeleteFilesCount()) + .deletedEqualityDeleteFilesCount(summary.equalityDeleteFilesCount()) + .deletedManifestsCount(summary.manifestsCount()) + .deletedManifestListsCount(summary.manifestListsCount()) + .deletedStatisticsFilesCount(summary.statisticsFilesCount()) + .build(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/FileInfo.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/FileInfo.java new file mode 100644 index 000000000000..51ff7c80fd18 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/FileInfo.java @@ -0,0 +1,52 @@ +/* + * 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.actions; + +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; + +public class FileInfo { + public static final Encoder ENCODER = Encoders.bean(FileInfo.class); + + private String path; + private String type; + + public FileInfo(String path, String type) { + this.path = path; + this.type = type; + } + + public FileInfo() {} + + public String getPath() { + return path; + } + + public void setPath(String path) { + this.path = path; + } + + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java new file mode 100644 index 000000000000..fd4639897743 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java @@ -0,0 +1,191 @@ +/* + * 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.actions; + +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; + +/** A serializable bean that contains a bare minimum to read a manifest. */ +public class ManifestFileBean implements ManifestFile, Serializable { + public static final Encoder ENCODER = Encoders.bean(ManifestFileBean.class); + + private String path = null; + private Long length = null; + private Integer partitionSpecId = null; + private Long addedSnapshotId = null; + private Integer content = null; + private Long sequenceNumber = null; + private Long firstRowId = null; + + public static ManifestFileBean fromManifest(ManifestFile manifest) { + ManifestFileBean bean = new ManifestFileBean(); + + bean.setPath(manifest.path()); + bean.setLength(manifest.length()); + bean.setPartitionSpecId(manifest.partitionSpecId()); + bean.setAddedSnapshotId(manifest.snapshotId()); + bean.setContent(manifest.content().id()); + bean.setSequenceNumber(manifest.sequenceNumber()); + bean.setFirstRowId(manifest.firstRowId()); + + return bean; + } + + public String getPath() { + return path; + } + + public void setPath(String path) { + this.path = path; + } + + public Long getLength() { + return length; + } + + public void setLength(Long length) { + this.length = length; + } + + public Integer getPartitionSpecId() { + return partitionSpecId; + } + + public void setPartitionSpecId(Integer partitionSpecId) { + this.partitionSpecId = partitionSpecId; + } + + public Long getAddedSnapshotId() { + return addedSnapshotId; + } + + public void setAddedSnapshotId(Long addedSnapshotId) { + this.addedSnapshotId = addedSnapshotId; + } + + public Integer getContent() { + return content; + } + + public void setContent(Integer content) { + this.content = content; + } + + public Long getSequenceNumber() { + return sequenceNumber; + } + + public void setSequenceNumber(Long sequenceNumber) { + this.sequenceNumber = sequenceNumber; + } + + public void setFirstRowId(Long firstRowId) { + this.firstRowId = firstRowId; + } + + @Override + public String path() { + return path; + } + + @Override + public long length() { + return length; + } + + @Override + public int partitionSpecId() { + return partitionSpecId; + } + + @Override + public ManifestContent content() { + return ManifestContent.fromId(content); + } + + @Override + public long sequenceNumber() { + return sequenceNumber; + } + + @Override + public long minSequenceNumber() { + return 0; + } + + @Override + public Long snapshotId() { + return addedSnapshotId; + } + + @Override + public Integer addedFilesCount() { + return null; + } + + @Override + public Long addedRowsCount() { + return null; + } + + @Override + public Integer existingFilesCount() { + return null; + } + + @Override + public Long existingRowsCount() { + return null; + } + + @Override + public Integer deletedFilesCount() { + return null; + } + + @Override + public Long deletedRowsCount() { + return null; + } + + @Override + public List partitions() { + return null; + } + + @Override + public ByteBuffer keyMetadata() { + return null; + } + + @Override + public Long firstRowId() { + return firstRowId; + } + + @Override + public ManifestFile copy() { + throw new UnsupportedOperationException("Cannot copy"); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java new file mode 100644 index 000000000000..bdffeb465405 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java @@ -0,0 +1,259 @@ +/* + * 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.actions; + +import java.util.Map; +import java.util.concurrent.ExecutorService; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ImmutableMigrateTable; +import org.apache.iceberg.actions.MigrateTable; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.spark.source.StagedSparkTable; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.StagingTableCatalog; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.Some; +import scala.collection.JavaConverters; + +/** + * Takes a Spark table in the source catalog and attempts to transform it into an Iceberg table in + * the same location with the same identifier. Once complete the identifier which previously + * referred to a non-Iceberg table will refer to the newly migrated Iceberg table. + */ +public class MigrateTableSparkAction extends BaseTableCreationSparkAction + implements MigrateTable { + + private static final Logger LOG = LoggerFactory.getLogger(MigrateTableSparkAction.class); + private static final String BACKUP_SUFFIX = "_BACKUP_"; + + private final StagingTableCatalog destCatalog; + private final Identifier destTableIdent; + + private Identifier backupIdent; + private boolean dropBackup = false; + private ExecutorService executorService; + + MigrateTableSparkAction( + SparkSession spark, CatalogPlugin sourceCatalog, Identifier sourceTableIdent) { + super(spark, sourceCatalog, sourceTableIdent); + this.destCatalog = checkDestinationCatalog(sourceCatalog); + this.destTableIdent = sourceTableIdent; + String backupName = sourceTableIdent.name() + BACKUP_SUFFIX; + this.backupIdent = Identifier.of(sourceTableIdent.namespace(), backupName); + } + + @Override + protected MigrateTableSparkAction self() { + return this; + } + + @Override + protected StagingTableCatalog destCatalog() { + return destCatalog; + } + + @Override + protected Identifier destTableIdent() { + return destTableIdent; + } + + @Override + public MigrateTableSparkAction tableProperties(Map properties) { + setProperties(properties); + return this; + } + + @Override + public MigrateTableSparkAction tableProperty(String property, String value) { + setProperty(property, value); + return this; + } + + @Override + public MigrateTableSparkAction dropBackup() { + this.dropBackup = true; + return this; + } + + @Override + public MigrateTableSparkAction backupTableName(String tableName) { + this.backupIdent = Identifier.of(sourceTableIdent().namespace(), tableName); + return this; + } + + @Override + public MigrateTableSparkAction executeWith(ExecutorService service) { + this.executorService = service; + return this; + } + + @Override + public MigrateTable.Result execute() { + String desc = String.format("Migrating table %s", destTableIdent().toString()); + JobGroupInfo info = newJobGroupInfo("MIGRATE-TABLE", desc); + return withJobGroupInfo(info, this::doExecute); + } + + private MigrateTable.Result doExecute() { + LOG.info("Starting the migration of {} to Iceberg", sourceTableIdent()); + + // move the source table to a new name, halting all modifications and allowing us to stage + // the creation of a new Iceberg table in its place + renameAndBackupSourceTable(); + + StagedSparkTable stagedTable = null; + Table icebergTable; + boolean threw = true; + try { + LOG.info("Staging a new Iceberg table {}", destTableIdent()); + stagedTable = stageDestTable(); + icebergTable = stagedTable.table(); + + LOG.info("Ensuring {} has a valid name mapping", destTableIdent()); + ensureNameMappingPresent(icebergTable); + + Some backupNamespace = Some.apply(backupIdent.namespace()[0]); + TableIdentifier v1BackupIdent = new TableIdentifier(backupIdent.name(), backupNamespace); + String stagingLocation = getMetadataLocation(icebergTable); + LOG.info("Generating Iceberg metadata for {} in {}", destTableIdent(), stagingLocation); + SparkTableUtil.importSparkTable( + spark(), v1BackupIdent, icebergTable, stagingLocation, executorService); + + LOG.info("Committing staged changes to {}", destTableIdent()); + stagedTable.commitStagedChanges(); + threw = false; + } finally { + if (threw) { + LOG.error( + "Failed to perform the migration, aborting table creation and restoring the original table"); + + restoreSourceTable(); + + if (stagedTable != null) { + try { + stagedTable.abortStagedChanges(); + } catch (Exception abortException) { + LOG.error("Cannot abort staged changes", abortException); + } + } + } else if (dropBackup) { + dropBackupTable(); + } + } + + Snapshot snapshot = icebergTable.currentSnapshot(); + long migratedDataFilesCount = + Long.parseLong(snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP)); + LOG.info( + "Successfully loaded Iceberg metadata for {} files to {}", + migratedDataFilesCount, + destTableIdent()); + return ImmutableMigrateTable.Result.builder() + .migratedDataFilesCount(migratedDataFilesCount) + .build(); + } + + @Override + protected Map destTableProps() { + Map properties = Maps.newHashMap(); + + // copy over relevant source table props + properties.putAll(JavaConverters.mapAsJavaMapConverter(v1SourceTable().properties()).asJava()); + EXCLUDED_PROPERTIES.forEach(properties::remove); + + // set default and user-provided props + properties.put(TableCatalog.PROP_PROVIDER, "iceberg"); + properties.putAll(additionalProperties()); + + // make sure we mark this table as migrated + properties.put("migrated", "true"); + + // inherit the source table location + properties.putIfAbsent(LOCATION, sourceTableLocation()); + + return properties; + } + + @Override + protected TableCatalog checkSourceCatalog(CatalogPlugin catalog) { + // currently the import code relies on being able to look up the table in the session catalog + Preconditions.checkArgument( + catalog instanceof SparkSessionCatalog, + "Cannot migrate a table from a non-Iceberg Spark Session Catalog. Found %s of class %s as the source catalog.", + catalog.name(), + catalog.getClass().getName()); + + return (TableCatalog) catalog; + } + + private void renameAndBackupSourceTable() { + try { + LOG.info("Renaming {} as {} for backup", sourceTableIdent(), backupIdent); + destCatalog().renameTable(sourceTableIdent(), backupIdent); + + } catch (org.apache.spark.sql.catalyst.analysis.NoSuchTableException e) { + throw new NoSuchTableException("Cannot find source table %s", sourceTableIdent()); + + } catch (org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException e) { + throw new AlreadyExistsException( + "Cannot rename %s as %s for backup. The backup table already exists.", + sourceTableIdent(), backupIdent); + } + } + + private void restoreSourceTable() { + try { + LOG.info("Restoring {} from {}", sourceTableIdent(), backupIdent); + destCatalog().renameTable(backupIdent, sourceTableIdent()); + + } catch (org.apache.spark.sql.catalyst.analysis.NoSuchTableException e) { + LOG.error( + "Cannot restore the original table, the backup table {} cannot be found", backupIdent, e); + + } catch (org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException e) { + LOG.error( + "Cannot restore the original table, a table with the original name exists. " + + "Use the backup table {} to restore the original table manually.", + backupIdent, + e); + } + } + + private void dropBackupTable() { + try { + destCatalog().dropTable(backupIdent); + } catch (Exception e) { + LOG.error( + "Cannot drop the backup table {}, after the migration is completed.", backupIdent, e); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java new file mode 100644 index 000000000000..11b14fafd916 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java @@ -0,0 +1,91 @@ +/* + * 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.actions; + +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.theta.CompactSketch; +import org.apache.datasketches.theta.Sketch; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.PuffinCompressionCodec; +import org.apache.iceberg.puffin.StandardBlobTypes; +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.Lists; +import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.classic.ExpressionColumnNode; +import org.apache.spark.sql.stats.ThetaSketchAgg; + +public class NDVSketchUtil { + + private NDVSketchUtil() {} + + public static final String APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY = "ndv"; + + static List generateBlobs( + SparkSession spark, Table table, Snapshot snapshot, List columns) { + Row sketches = computeNDVSketches(spark, table, snapshot, columns); + Schema schema = table.schemas().get(snapshot.schemaId()); + List blobs = Lists.newArrayListWithExpectedSize(columns.size()); + + for (int i = 0; i < columns.size(); i++) { + Types.NestedField field = schema.findField(columns.get(i)); + Sketch sketch = CompactSketch.wrap(Memory.wrap((byte[]) sketches.get(i))); + blobs.add(toBlob(field, sketch, snapshot)); + } + return blobs; + } + + private static Blob toBlob(Types.NestedField field, Sketch sketch, Snapshot snapshot) { + return new Blob( + StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1, + ImmutableList.of(field.fieldId()), + snapshot.snapshotId(), + snapshot.sequenceNumber(), + ByteBuffer.wrap(sketch.toByteArray()), + PuffinCompressionCodec.ZSTD, + ImmutableMap.of( + APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, + String.valueOf((long) sketch.getEstimate()))); + } + + private static Row computeNDVSketches( + SparkSession spark, Table table, Snapshot snapshot, List colNames) { + Dataset inputDF = SparkTableUtil.loadTable(spark, table, snapshot.snapshotId()); + return inputDF.select(toAggColumns(colNames)).first(); + } + + private static Column[] toAggColumns(List colNames) { + return colNames.stream().map(NDVSketchUtil::toAggColumn).toArray(Column[]::new); + } + + private static Column toAggColumn(String colName) { + ThetaSketchAgg agg = new ThetaSketchAgg(colName); + return new Column(ExpressionColumnNode.apply(agg.toAggregateExpression())); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java new file mode 100644 index 000000000000..bbe2847a9216 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RemoveDanglingDeletesSparkAction.java @@ -0,0 +1,206 @@ +/* + * 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.actions; + +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.min; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ImmutableRemoveDanglingDeleteFiles; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.spark.SparkDeleteFile; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DeleteFileSet; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An action that removes dangling delete files from the current snapshot. A delete file is dangling + * if its deletes no longer applies to any live data files. + * + *

    The following dangling delete files are removed: + * + *

      + *
    • Position delete files with a data sequence number less than that of any data file in the + * same partition + *
    • Equality delete files with a data sequence number less than or equal to that of any data + * file in the same partition + *
    + */ +class RemoveDanglingDeletesSparkAction + extends BaseSnapshotUpdateSparkAction + implements RemoveDanglingDeleteFiles { + + private static final Logger LOG = LoggerFactory.getLogger(RemoveDanglingDeletesSparkAction.class); + private final Table table; + + protected RemoveDanglingDeletesSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + } + + @Override + protected RemoveDanglingDeletesSparkAction self() { + return this; + } + + @Override + public Result execute() { + if (table.specs().size() == 1 && table.spec().isUnpartitioned()) { + // ManifestFilterManager already performs this table-wide delete on each commit + return ImmutableRemoveDanglingDeleteFiles.Result.builder() + .removedDeleteFiles(Collections.emptyList()) + .build(); + } + + String desc = String.format("Removing dangling delete files in %s", table.name()); + JobGroupInfo info = newJobGroupInfo("REMOVE-DELETES", desc); + return withJobGroupInfo(info, this::doExecute); + } + + Result doExecute() { + RewriteFiles rewriteFiles = table.newRewrite(); + DeleteFileSet danglingDeletes = DeleteFileSet.create(); + danglingDeletes.addAll(findDanglingDeletes()); + danglingDeletes.addAll(findDanglingDvs()); + + for (DeleteFile deleteFile : danglingDeletes) { + LOG.debug("Removing dangling delete file {}", deleteFile.location()); + rewriteFiles.deleteFile(deleteFile); + } + + if (!danglingDeletes.isEmpty()) { + commit(rewriteFiles); + } + + return ImmutableRemoveDanglingDeleteFiles.Result.builder() + .removedDeleteFiles(danglingDeletes) + .build(); + } + + /** + * Dangling delete files can be identified with following steps + * + *
      + *
    1. Group data files by partition keys and find the minimum data sequence number in each + * group. + *
    2. Left outer join delete files with partition-grouped data files on partition keys. + *
    3. Find dangling deletes by comparing each delete file's sequence number to its partition's + * minimum data sequence number. + *
    4. Collect results row to driver and use {@link SparkDeleteFile SparkDeleteFile} to wrap + * rows to valid delete files + *
    + */ + private List findDanglingDeletes() { + Dataset minSequenceNumberByPartition = + loadMetadataTable(table, MetadataTableType.ENTRIES) + // find live data files + .filter("data_file.content == 0 AND status < 2") + .selectExpr( + "data_file.partition as partition", + "data_file.spec_id as spec_id", + "sequence_number") + .groupBy("partition", "spec_id") + .agg(min("sequence_number")) + .toDF("grouped_partition", "grouped_spec_id", "min_data_sequence_number"); + + Dataset deleteEntries = + loadMetadataTable(table, MetadataTableType.ENTRIES) + // find live delete files + .filter("data_file.content != 0 AND status < 2"); + + Column joinOnPartition = + deleteEntries + .col("data_file.spec_id") + .equalTo(minSequenceNumberByPartition.col("grouped_spec_id")) + .and( + deleteEntries + .col("data_file.partition") + .equalTo(minSequenceNumberByPartition.col("grouped_partition"))); + + Column filterOnDanglingDeletes = + col("min_data_sequence_number") + // delete files without any data files in partition + .isNull() + // position delete files without any applicable data files in partition + .or( + col("data_file.content") + .equalTo("1") + .and(col("sequence_number").$less(col("min_data_sequence_number")))) + // equality delete files without any applicable data files in the partition + .or( + col("data_file.content") + .equalTo("2") + .and(col("sequence_number").$less$eq(col("min_data_sequence_number")))); + + Dataset danglingDeletes = + deleteEntries + .join(minSequenceNumberByPartition, joinOnPartition, "left") + .filter(filterOnDanglingDeletes) + .select("data_file.*"); + return danglingDeletes.collectAsList().stream() + // map on driver because SparkDeleteFile is not serializable + .map(row -> deleteFileWrapper(danglingDeletes.schema(), row)) + .collect(Collectors.toList()); + } + + private List findDanglingDvs() { + Dataset dvs = + loadMetadataTable(table, MetadataTableType.DELETE_FILES) + .where(col("file_format").equalTo(FileFormat.PUFFIN.name())); + Dataset dataFiles = loadMetadataTable(table, MetadataTableType.DATA_FILES); + + // a DV not pointing to a valid data file path is implicitly a dangling delete + List danglingDvs = + dvs.join( + dataFiles, + dvs.col("referenced_data_file").equalTo(dataFiles.col("file_path")), + "leftouter") + .filter(dataFiles.col("file_path").isNull()) + .select(dvs.col("*")) + .collectAsList(); + return danglingDvs.stream() + // map on driver because SparkDeleteFile is not serializable + .map(row -> deleteFileWrapper(dvs.schema(), row)) + .collect(Collectors.toList()); + } + + private DeleteFile deleteFileWrapper(StructType sparkFileType, Row row) { + int specId = row.getInt(row.fieldIndex("spec_id")); + Types.StructType combinedFileType = DataFile.getType(Partitioning.partitionType(table)); + // Set correct spec id + Types.StructType projection = DataFile.getType(table.specs().get(specId).partitionType()); + return new SparkDeleteFile(combinedFileType, projection, sparkFileType).wrap(row); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java new file mode 100644 index 000000000000..61d1fdfe605f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteDataFilesSparkAction.java @@ -0,0 +1,448 @@ +/* + * 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.actions; + +import java.math.RoundingMode; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.BinPackRewriteFilePlanner; +import org.apache.iceberg.actions.FileRewritePlan; +import org.apache.iceberg.actions.FileRewriteRunner; +import org.apache.iceberg.actions.ImmutableRewriteDataFiles; +import org.apache.iceberg.actions.ImmutableRewriteDataFiles.Result.Builder; +import org.apache.iceberg.actions.RewriteDataFiles; +import org.apache.iceberg.actions.RewriteDataFilesCommitManager; +import org.apache.iceberg.actions.RewriteFileGroup; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +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.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Queues; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.math.IntMath; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.iceberg.spark.SparkSQLProperties; +import org.apache.iceberg.spark.SparkUtil; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.internal.SQLConf; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RewriteDataFilesSparkAction + extends BaseSnapshotUpdateSparkAction implements RewriteDataFiles { + + private static final Logger LOG = LoggerFactory.getLogger(RewriteDataFilesSparkAction.class); + private static final Set VALID_OPTIONS = + ImmutableSet.of( + MAX_CONCURRENT_FILE_GROUP_REWRITES, + MAX_FILE_GROUP_SIZE_BYTES, + PARTIAL_PROGRESS_ENABLED, + PARTIAL_PROGRESS_MAX_COMMITS, + PARTIAL_PROGRESS_MAX_FAILED_COMMITS, + TARGET_FILE_SIZE_BYTES, + USE_STARTING_SEQUENCE_NUMBER, + REWRITE_JOB_ORDER, + OUTPUT_SPEC_ID, + REMOVE_DANGLING_DELETES, + BinPackRewriteFilePlanner.MAX_FILES_TO_REWRITE); + + private static final RewriteDataFilesSparkAction.Result EMPTY_RESULT = + ImmutableRewriteDataFiles.Result.builder().rewriteResults(ImmutableList.of()).build(); + + private final Table table; + + private Expression filter = Expressions.alwaysTrue(); + private int maxConcurrentFileGroupRewrites; + private int maxCommits; + private int maxFailedCommits; + private boolean partialProgressEnabled; + private boolean removeDanglingDeletes; + private boolean useStartingSequenceNumber; + private boolean caseSensitive; + private BinPackRewriteFilePlanner planner = null; + private FileRewriteRunner runner = null; + + RewriteDataFilesSparkAction(SparkSession spark, Table table) { + super(((org.apache.spark.sql.classic.SparkSession) spark).cloneSession()); + // Disable Adaptive Query Execution as this may change the output partitioning of our write + spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false); + // Disable executor cache for delete files as each partition is rewritten separately. + // Note: when compacting to a different target spec, data from multiple partitions + // may be grouped together, but caching is still disabled to avoid connection pool issues. + spark().conf().set(SparkSQLProperties.EXECUTOR_CACHE_DELETE_FILES_ENABLED, "false"); + this.caseSensitive = SparkUtil.caseSensitive(spark); + this.table = table; + } + + @Override + protected RewriteDataFilesSparkAction self() { + return this; + } + + @Override + public RewriteDataFilesSparkAction binPack() { + ensureRunnerNotSet(); + this.runner = new SparkBinPackFileRewriteRunner(spark(), table); + return this; + } + + @Override + public RewriteDataFilesSparkAction sort(SortOrder sortOrder) { + ensureRunnerNotSet(); + this.runner = new SparkSortFileRewriteRunner(spark(), table, sortOrder); + return this; + } + + @Override + public RewriteDataFilesSparkAction sort() { + ensureRunnerNotSet(); + this.runner = new SparkSortFileRewriteRunner(spark(), table); + return this; + } + + @Override + public RewriteDataFilesSparkAction zOrder(String... columnNames) { + ensureRunnerNotSet(); + this.runner = new SparkZOrderFileRewriteRunner(spark(), table, Arrays.asList(columnNames)); + return this; + } + + private void ensureRunnerNotSet() { + Preconditions.checkArgument( + runner == null, + "Cannot set rewrite mode, it has already been set to %s", + runner == null ? null : runner.description()); + } + + @Override + public RewriteDataFilesSparkAction filter(Expression expression) { + filter = Expressions.and(filter, expression); + return this; + } + + @Override + public RewriteDataFiles.Result execute() { + if (table.currentSnapshot() == null) { + return EMPTY_RESULT; + } + + long startingSnapshotId = table.currentSnapshot().snapshotId(); + + init(startingSnapshotId); + + FileRewritePlan plan = planner.plan(); + + if (plan.totalGroupCount() == 0) { + LOG.info("Nothing found to rewrite in {}", table.name()); + return EMPTY_RESULT; + } + + Builder resultBuilder = + partialProgressEnabled + ? doExecuteWithPartialProgress(plan, commitManager(startingSnapshotId)) + : doExecute(plan, commitManager(startingSnapshotId)); + ImmutableRewriteDataFiles.Result result = resultBuilder.build(); + + if (removeDanglingDeletes) { + RemoveDanglingDeletesSparkAction action = + new RemoveDanglingDeletesSparkAction(spark(), table); + int removedDeleteFiles = Iterables.size(action.execute().removedDeleteFiles()); + return result.withRemovedDeleteFilesCount( + result.removedDeleteFilesCount() + removedDeleteFiles); + } + + return result; + } + + private void init(long startingSnapshotId) { + this.planner = + runner instanceof SparkShufflingFileRewriteRunner + ? new SparkShufflingDataRewritePlanner(table, filter, startingSnapshotId, caseSensitive) + : new BinPackRewriteFilePlanner(table, filter, startingSnapshotId, caseSensitive); + + // Default to BinPack if no strategy selected + if (this.runner == null) { + this.runner = new SparkBinPackFileRewriteRunner(spark(), table); + } + + validateAndInitOptions(); + } + + @VisibleForTesting + RewriteFileGroup rewriteFiles( + FileRewritePlan plan, + RewriteFileGroup fileGroup) { + String desc = jobDesc(fileGroup, plan); + Set addedFiles = + withJobGroupInfo( + newJobGroupInfo("REWRITE-DATA-FILES", desc), () -> runner.rewrite(fileGroup)); + + fileGroup.setOutputFiles(addedFiles); + LOG.info("Rewrite Files Ready to be Committed - {}", desc); + return fileGroup; + } + + private ExecutorService rewriteService() { + return MoreExecutors.getExitingExecutorService( + (ThreadPoolExecutor) + Executors.newFixedThreadPool( + maxConcurrentFileGroupRewrites, + new ThreadFactoryBuilder().setNameFormat("Rewrite-Service-%d").build())); + } + + @VisibleForTesting + RewriteDataFilesCommitManager commitManager(long startingSnapshotId) { + return new RewriteDataFilesCommitManager( + table, startingSnapshotId, useStartingSequenceNumber, commitSummary()); + } + + private Builder doExecute( + FileRewritePlan plan, + RewriteDataFilesCommitManager commitManager) { + ExecutorService rewriteService = rewriteService(); + + ConcurrentLinkedQueue rewrittenGroups = Queues.newConcurrentLinkedQueue(); + + Tasks.Builder rewriteTaskBuilder = + Tasks.foreach(plan.groups()) + .executeWith(rewriteService) + .stopOnFailure() + .noRetry() + .onFailure( + (fileGroup, exception) -> { + LOG.warn( + "Failure during rewrite process for group {}", fileGroup.info(), exception); + }); + + try { + rewriteTaskBuilder.run( + fileGroup -> { + rewrittenGroups.add(rewriteFiles(plan, fileGroup)); + }); + } catch (Exception e) { + // At least one rewrite group failed, clean up all completed rewrites + LOG.error( + "Cannot complete rewrite, {} is not enabled and one of the file set groups failed to " + + "be rewritten. This error occurred during the writing of new files, not during the commit process. This " + + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling " + + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished " + + "being written.", + PARTIAL_PROGRESS_ENABLED, + PARTIAL_PROGRESS_ENABLED, + rewrittenGroups.size(), + e); + + Tasks.foreach(rewrittenGroups) + .suppressFailureWhenFinished() + .run(commitManager::abortFileGroup); + throw e; + } finally { + rewriteService.shutdown(); + } + + try { + commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups)); + } catch (ValidationException | CommitFailedException e) { + String errorMessage = + String.format( + "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that " + + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of " + + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. " + + "Separate smaller rewrite commits can succeed independently while any commits that conflict with " + + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table " + + "history, one for each commit.", + PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS); + throw new RuntimeException(errorMessage, e); + } + + List rewriteResults = + rewrittenGroups.stream().map(RewriteFileGroup::asResult).collect(Collectors.toList()); + return ImmutableRewriteDataFiles.Result.builder().rewriteResults(rewriteResults); + } + + private Builder doExecuteWithPartialProgress( + FileRewritePlan plan, + RewriteDataFilesCommitManager commitManager) { + ExecutorService rewriteService = rewriteService(); + + // start commit service + int groupsPerCommit = IntMath.divide(plan.totalGroupCount(), maxCommits, RoundingMode.CEILING); + RewriteDataFilesCommitManager.CommitService commitService = + commitManager.service(groupsPerCommit); + commitService.start(); + + Collection rewriteFailures = new ConcurrentLinkedQueue<>(); + // start rewrite tasks + Tasks.foreach(plan.groups()) + .suppressFailureWhenFinished() + .executeWith(rewriteService) + .noRetry() + .onFailure( + (fileGroup, exception) -> { + LOG.error("Failure during rewrite group {}", fileGroup.info(), exception); + rewriteFailures.add( + ImmutableRewriteDataFiles.FileGroupFailureResult.builder() + .info(fileGroup.info()) + .dataFilesCount(fileGroup.inputFileNum()) + .build()); + }) + .run(fileGroup -> commitService.offer(rewriteFiles(plan, fileGroup))); + rewriteService.shutdown(); + + // stop commit service + commitService.close(); + + int totalCommits = Math.min(plan.totalGroupCount(), maxCommits); + int failedCommits = totalCommits - commitService.succeededCommits(); + if (failedCommits > 0 && failedCommits <= maxFailedCommits) { + LOG.warn( + "{} is true but {} rewrite commits failed. Check the logs to determine why the individual " + + "commits failed. If this is persistent it may help to increase {} which will split the rewrite operation " + + "into smaller commits.", + PARTIAL_PROGRESS_ENABLED, + failedCommits, + PARTIAL_PROGRESS_MAX_COMMITS); + } else if (failedCommits > maxFailedCommits) { + String errorMessage = + String.format( + Locale.ROOT, + "%s is true but %d rewrite commits failed. This is more than the maximum allowed failures of %d. " + + "Check the logs to determine why the individual commits failed. If this is persistent it may help to " + + "increase %s which will split the rewrite operation into smaller commits.", + PARTIAL_PROGRESS_ENABLED, + failedCommits, + maxFailedCommits, + PARTIAL_PROGRESS_MAX_COMMITS); + throw new RuntimeException(errorMessage); + } + + return ImmutableRewriteDataFiles.Result.builder() + .rewriteResults(toRewriteResults(commitService.results())) + .rewriteFailures(rewriteFailures); + } + + private Iterable toRewriteResults(List commitResults) { + return commitResults.stream().map(RewriteFileGroup::asResult).collect(Collectors.toList()); + } + + void validateAndInitOptions() { + Set validOptions = Sets.newHashSet(runner.validOptions()); + validOptions.addAll(VALID_OPTIONS); + validOptions.addAll(planner.validOptions()); + + Set invalidKeys = Sets.newHashSet(options().keySet()); + invalidKeys.removeAll(validOptions); + + Preconditions.checkArgument( + invalidKeys.isEmpty(), + "Cannot use options %s, they are not supported by the action or the rewriter %s", + invalidKeys, + runner.description()); + + planner.init(options()); + runner.init(options()); + + maxConcurrentFileGroupRewrites = + PropertyUtil.propertyAsInt( + options(), + MAX_CONCURRENT_FILE_GROUP_REWRITES, + MAX_CONCURRENT_FILE_GROUP_REWRITES_DEFAULT); + + maxCommits = + PropertyUtil.propertyAsInt( + options(), PARTIAL_PROGRESS_MAX_COMMITS, PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT); + + maxFailedCommits = + PropertyUtil.propertyAsInt(options(), PARTIAL_PROGRESS_MAX_FAILED_COMMITS, maxCommits); + + partialProgressEnabled = + PropertyUtil.propertyAsBoolean( + options(), PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_ENABLED_DEFAULT); + + useStartingSequenceNumber = + PropertyUtil.propertyAsBoolean( + options(), USE_STARTING_SEQUENCE_NUMBER, USE_STARTING_SEQUENCE_NUMBER_DEFAULT); + + removeDanglingDeletes = + PropertyUtil.propertyAsBoolean( + options(), REMOVE_DANGLING_DELETES, REMOVE_DANGLING_DELETES_DEFAULT); + + Preconditions.checkArgument( + maxConcurrentFileGroupRewrites >= 1, + "Cannot set %s to %s, the value must be positive.", + MAX_CONCURRENT_FILE_GROUP_REWRITES, + maxConcurrentFileGroupRewrites); + + Preconditions.checkArgument( + !partialProgressEnabled || maxCommits > 0, + "Cannot set %s to %s, the value must be positive when %s is true", + PARTIAL_PROGRESS_MAX_COMMITS, + maxCommits, + PARTIAL_PROGRESS_ENABLED); + } + + private String jobDesc( + RewriteFileGroup group, + FileRewritePlan plan) { + StructLike partition = group.info().partition(); + if (partition.size() > 0) { + return String.format( + Locale.ROOT, + "Rewriting %d files (%s, file group %d/%d, %s (%d/%d)) in %s", + group.rewrittenFiles().size(), + runner.description(), + group.info().globalIndex(), + plan.totalGroupCount(), + partition, + group.info().partitionIndex(), + plan.groupsInPartition(partition), + table.name()); + } else { + return String.format( + Locale.ROOT, + "Rewriting %d files (%s, file group %d/%d) in %s", + group.rewrittenFiles().size(), + runner.description(), + group.info().globalIndex(), + plan.totalGroupCount(), + table.name()); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java new file mode 100644 index 000000000000..668360c06730 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -0,0 +1,606 @@ +/* + * 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.actions; + +import static org.apache.iceberg.MetadataTableType.ENTRIES; +import static org.apache.spark.sql.functions.col; + +import java.io.Serializable; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.RollingManifestWriter; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.ImmutableRewriteManifests; +import org.apache.iceberg.actions.RewriteManifests; +import org.apache.iceberg.exceptions.CleanableFailure; +import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.SupportsBulkOperations; +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.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.spark.SparkContentFile; +import org.apache.iceberg.spark.SparkDataFile; +import org.apache.iceberg.spark.SparkDeleteFile; +import org.apache.iceberg.spark.source.SerializableTableWithSize; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.apache.spark.api.java.function.MapPartitionsFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An action that rewrites manifests in a distributed manner and co-locates metadata for partitions. + * + *

    By default, this action rewrites all manifests for the current partition spec and writes the + * result to the metadata folder. The behavior can be modified by passing a custom predicate to + * {@link #rewriteIf(Predicate)} and a custom spec ID to {@link #specId(int)}. In addition, there is + * a way to configure a custom location for staged manifests via {@link #stagingLocation(String)}. + * The provided staging location will be ignored if snapshot ID inheritance is enabled. In such + * cases, the manifests are always written to the metadata folder and committed without staging. + */ +public class RewriteManifestsSparkAction + extends BaseSnapshotUpdateSparkAction implements RewriteManifests { + + public static final String USE_CACHING = "use-caching"; + public static final boolean USE_CACHING_DEFAULT = false; + + private static final Logger LOG = LoggerFactory.getLogger(RewriteManifestsSparkAction.class); + private static final RewriteManifests.Result EMPTY_RESULT = + ImmutableRewriteManifests.Result.builder() + .rewrittenManifests(ImmutableList.of()) + .addedManifests(ImmutableList.of()) + .build(); + + private static final String DATA_FILE_PARTITION_COLUMN_NAME = "data_file.partition"; + + private final Table table; + private final int formatVersion; + private final long targetManifestSizeBytes; + private final boolean shouldStageManifests; + + private PartitionSpec spec; + private Predicate predicate = manifest -> true; + private String outputLocation; + + private List partitionFieldClustering = null; + + RewriteManifestsSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.spec = table.spec(); + this.targetManifestSizeBytes = + PropertyUtil.propertyAsLong( + table.properties(), + TableProperties.MANIFEST_TARGET_SIZE_BYTES, + TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT); + + // default the output location to the metadata location + TableOperations ops = ((HasTableOperations) table).operations(); + Path metadataFilePath = new Path(ops.metadataFileLocation("file")); + this.outputLocation = metadataFilePath.getParent().toString(); + + // use the current table format version for new manifests + this.formatVersion = ops.current().formatVersion(); + + boolean snapshotIdInheritanceEnabled = + PropertyUtil.propertyAsBoolean( + table.properties(), + TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, + TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED_DEFAULT); + this.shouldStageManifests = formatVersion == 1 && !snapshotIdInheritanceEnabled; + } + + @Override + protected RewriteManifestsSparkAction self() { + return this; + } + + @Override + public RewriteManifestsSparkAction specId(int specId) { + Preconditions.checkArgument(table.specs().containsKey(specId), "Invalid spec id %s", specId); + this.spec = table.specs().get(specId); + return this; + } + + @Override + public RewriteManifestsSparkAction rewriteIf(Predicate newPredicate) { + this.predicate = newPredicate; + return this; + } + + @Override + public RewriteManifestsSparkAction stagingLocation(String newStagingLocation) { + if (shouldStageManifests) { + this.outputLocation = newStagingLocation; + } else { + LOG.warn("Ignoring provided staging location as new manifests will be committed directly"); + } + return this; + } + + @Override + public RewriteManifests.Result execute() { + String desc = String.format("Rewriting manifests in %s", table.name()); + JobGroupInfo info = newJobGroupInfo("REWRITE-MANIFESTS", desc); + return withJobGroupInfo(info, this::doExecute); + } + + @Override + public RewriteManifestsSparkAction sortBy(List partitionFields) { + // Collect set of available partition columns to cluster on + Set availablePartitionNames = + spec.fields().stream().map(PartitionField::name).collect(Collectors.toSet()); + + // Identify specified partition fields that are not available in the spec + List missingFields = + partitionFields.stream() + .filter(field -> !availablePartitionNames.contains(field)) + .collect(Collectors.toList()); + + // Check if these partition fields are included in the spec + Preconditions.checkArgument( + missingFields.isEmpty(), + "Cannot set manifest sorting because specified field(s) %s were not found in current partition spec %s. Spec ID %s", + missingFields, + this.spec, + this.spec.specId()); + + this.partitionFieldClustering = partitionFields; + return this; + } + + private RewriteManifests.Result doExecute() { + List rewrittenManifests = Lists.newArrayList(); + List addedManifests = Lists.newArrayList(); + + RewriteManifests.Result dataResult = rewriteManifests(ManifestContent.DATA); + Iterables.addAll(rewrittenManifests, dataResult.rewrittenManifests()); + Iterables.addAll(addedManifests, dataResult.addedManifests()); + + RewriteManifests.Result deletesResult = rewriteManifests(ManifestContent.DELETES); + Iterables.addAll(rewrittenManifests, deletesResult.rewrittenManifests()); + Iterables.addAll(addedManifests, deletesResult.addedManifests()); + + if (rewrittenManifests.isEmpty()) { + return EMPTY_RESULT; + } + + replaceManifests(rewrittenManifests, addedManifests); + + return ImmutableRewriteManifests.Result.builder() + .rewrittenManifests(rewrittenManifests) + .addedManifests(addedManifests) + .build(); + } + + private RewriteManifests.Result rewriteManifests(ManifestContent content) { + List matchingManifests = findMatchingManifests(content); + if (matchingManifests.isEmpty()) { + return EMPTY_RESULT; + } + + int targetNumManifests = targetNumManifests(totalSizeBytes(matchingManifests)); + if (targetNumManifests == 1 && matchingManifests.size() == 1) { + return EMPTY_RESULT; + } + + Dataset manifestEntryDF = buildManifestEntryDF(matchingManifests); + + List newManifests; + if (spec.isUnpartitioned()) { + newManifests = writeUnpartitionedManifests(content, manifestEntryDF, targetNumManifests); + } else { + newManifests = writePartitionedManifests(content, manifestEntryDF, targetNumManifests); + } + + return ImmutableRewriteManifests.Result.builder() + .rewrittenManifests(matchingManifests) + .addedManifests(newManifests) + .build(); + } + + private Dataset buildManifestEntryDF(List manifests) { + Dataset manifestDF = + spark() + .createDataset(Lists.transform(manifests, ManifestFile::path), Encoders.STRING()) + .toDF("manifest"); + + Dataset manifestEntryDF = + loadMetadataTable(table, ENTRIES) + .filter("status < 2") // select only live entries + .selectExpr( + "input_file_name() as manifest", + "snapshot_id", + "sequence_number", + "file_sequence_number", + "data_file"); + + Column joinCond = manifestDF.col("manifest").equalTo(manifestEntryDF.col("manifest")); + return manifestEntryDF + .join(manifestDF, joinCond, "left_semi") + .select("snapshot_id", "sequence_number", "file_sequence_number", "data_file"); + } + + private List writeUnpartitionedManifests( + ManifestContent content, Dataset manifestEntryDF, int numManifests) { + + WriteManifests writeFunc = newWriteManifestsFunc(content, manifestEntryDF.schema()); + Dataset transformedManifestEntryDF = manifestEntryDF.repartition(numManifests); + return writeFunc.apply(transformedManifestEntryDF).collectAsList(); + } + + private List writePartitionedManifests( + ManifestContent content, Dataset manifestEntryDF, int numManifests) { + + return withReusableDS( + manifestEntryDF, + df -> { + WriteManifests writeFunc = newWriteManifestsFunc(content, df.schema()); + Dataset transformedDF = repartitionAndSort(df, sortColumn(), numManifests); + return writeFunc.apply(transformedDF).collectAsList(); + }); + } + + private WriteManifests newWriteManifestsFunc(ManifestContent content, StructType sparkType) { + ManifestWriterFactory writers = manifestWriters(); + + StructType sparkFileType = (StructType) sparkType.apply("data_file").dataType(); + Types.StructType combinedFileType = DataFile.getType(Partitioning.partitionType(table)); + Types.StructType fileType = DataFile.getType(spec.partitionType()); + + if (content == ManifestContent.DATA) { + return new WriteDataManifests(writers, combinedFileType, fileType, sparkFileType); + } else { + return new WriteDeleteManifests(writers, combinedFileType, fileType, sparkFileType); + } + } + + private Column sortColumn() { + if (partitionFieldClustering != null) { + LOG.info( + "Clustering manifests for specId {} by partition columns by {} ", + spec.specId(), + partitionFieldClustering); + + // Map the top level partition column names to the column name referenced within the manifest + // entry dataframe + Column[] partitionColumns = + partitionFieldClustering.stream() + .map(p -> col(DATA_FILE_PARTITION_COLUMN_NAME + "." + p)) + .toArray(Column[]::new); + + // Form a new temporary column to cluster manifests on, based on the custom clustering columns + // order provided + return functions.struct(partitionColumns); + } else { + return new Column(DATA_FILE_PARTITION_COLUMN_NAME); + } + } + + private Dataset repartitionAndSort(Dataset df, Column col, int numPartitions) { + return df.repartitionByRange(numPartitions, col).sortWithinPartitions(col); + } + + private U withReusableDS(Dataset ds, Function, U> func) { + boolean useCaching = + PropertyUtil.propertyAsBoolean(options(), USE_CACHING, USE_CACHING_DEFAULT); + Dataset reusableDS = useCaching ? ds.cache() : ds; + + try { + return func.apply(reusableDS); + } finally { + if (useCaching) { + reusableDS.unpersist(false); + } + } + } + + private List findMatchingManifests(ManifestContent content) { + Snapshot currentSnapshot = table.currentSnapshot(); + + if (currentSnapshot == null) { + return ImmutableList.of(); + } + + List manifests = loadManifests(content, currentSnapshot); + + return manifests.stream() + .filter(manifest -> manifest.partitionSpecId() == spec.specId() && predicate.test(manifest)) + .collect(Collectors.toList()); + } + + private List loadManifests(ManifestContent content, Snapshot snapshot) { + switch (content) { + case DATA: + return snapshot.dataManifests(table.io()); + case DELETES: + return snapshot.deleteManifests(table.io()); + default: + throw new IllegalArgumentException("Unknown manifest content: " + content); + } + } + + private int targetNumManifests(long totalSizeBytes) { + return (int) ((totalSizeBytes + targetManifestSizeBytes - 1) / targetManifestSizeBytes); + } + + private long totalSizeBytes(Iterable manifests) { + long totalSizeBytes = 0L; + + for (ManifestFile manifest : manifests) { + ValidationException.check( + hasFileCounts(manifest), "No file counts in manifest: %s", manifest.path()); + totalSizeBytes += manifest.length(); + } + + return totalSizeBytes; + } + + private boolean hasFileCounts(ManifestFile manifest) { + return manifest.addedFilesCount() != null + && manifest.existingFilesCount() != null + && manifest.deletedFilesCount() != null; + } + + private void replaceManifests( + Iterable deletedManifests, Iterable addedManifests) { + try { + org.apache.iceberg.RewriteManifests rewriteManifests = table.rewriteManifests(); + deletedManifests.forEach(rewriteManifests::deleteManifest); + addedManifests.forEach(rewriteManifests::addManifest); + commit(rewriteManifests); + + if (shouldStageManifests) { + // delete new manifests as they were rewritten before the commit + deleteFiles(Iterables.transform(addedManifests, ManifestFile::path)); + } + } catch (CommitStateUnknownException commitStateUnknownException) { + // don't clean up added manifest files, because they may have been successfully committed. + throw commitStateUnknownException; + } catch (Exception e) { + if (e instanceof CleanableFailure) { + // delete all new manifests because the rewrite failed + deleteFiles(Iterables.transform(addedManifests, ManifestFile::path)); + } + + throw e; + } + } + + private void deleteFiles(Iterable locations) { + Iterable files = + Iterables.transform(locations, location -> new FileInfo(location, MANIFEST)); + if (table.io() instanceof SupportsBulkOperations) { + deleteFiles((SupportsBulkOperations) table.io(), files.iterator()); + } else { + deleteFiles( + ThreadPools.getWorkerPool(), file -> table.io().deleteFile(file), files.iterator()); + } + } + + private ManifestWriterFactory manifestWriters() { + return new ManifestWriterFactory( + sparkContext().broadcast(SerializableTableWithSize.copyOf(table)), + formatVersion, + spec.specId(), + outputLocation, + // allow the actual size of manifests to be 20% higher as the estimation is not precise + (long) (1.2 * targetManifestSizeBytes)); + } + + private static class WriteDataManifests extends WriteManifests { + + WriteDataManifests( + ManifestWriterFactory manifestWriters, + Types.StructType combinedPartitionType, + Types.StructType partitionType, + StructType sparkFileType) { + super(manifestWriters, combinedPartitionType, partitionType, sparkFileType); + } + + @Override + protected SparkDataFile newFileWrapper() { + return new SparkDataFile(combinedFileType(), fileType(), sparkFileType()); + } + + @Override + protected RollingManifestWriter newManifestWriter() { + return writers().newRollingManifestWriter(); + } + } + + private static class WriteDeleteManifests extends WriteManifests { + + WriteDeleteManifests( + ManifestWriterFactory manifestWriters, + Types.StructType combinedFileType, + Types.StructType fileType, + StructType sparkFileType) { + super(manifestWriters, combinedFileType, fileType, sparkFileType); + } + + @Override + protected SparkDeleteFile newFileWrapper() { + return new SparkDeleteFile(combinedFileType(), fileType(), sparkFileType()); + } + + @Override + protected RollingManifestWriter newManifestWriter() { + return writers().newRollingDeleteManifestWriter(); + } + } + + private abstract static class WriteManifests> + implements MapPartitionsFunction { + + private static final Encoder MANIFEST_ENCODER = + Encoders.javaSerialization(ManifestFile.class); + + private final ManifestWriterFactory writers; + private final Types.StructType combinedFileType; + private final Types.StructType fileType; + private final StructType sparkFileType; + + WriteManifests( + ManifestWriterFactory writers, + Types.StructType combinedFileType, + Types.StructType fileType, + StructType sparkFileType) { + this.writers = writers; + this.combinedFileType = combinedFileType; + this.fileType = fileType; + this.sparkFileType = sparkFileType; + } + + protected abstract SparkContentFile newFileWrapper(); + + protected abstract RollingManifestWriter newManifestWriter(); + + public Dataset apply(Dataset input) { + return input.mapPartitions(this, MANIFEST_ENCODER); + } + + @Override + public Iterator call(Iterator rows) throws Exception { + SparkContentFile fileWrapper = newFileWrapper(); + RollingManifestWriter writer = newManifestWriter(); + + try { + while (rows.hasNext()) { + Row row = rows.next(); + long snapshotId = row.getLong(0); + long sequenceNumber = row.getLong(1); + Long fileSequenceNumber = row.isNullAt(2) ? null : row.getLong(2); + Row file = row.getStruct(3); + writer.existing(fileWrapper.wrap(file), snapshotId, sequenceNumber, fileSequenceNumber); + } + } finally { + writer.close(); + } + + return writer.toManifestFiles().iterator(); + } + + protected ManifestWriterFactory writers() { + return writers; + } + + protected Types.StructType combinedFileType() { + return combinedFileType; + } + + protected Types.StructType fileType() { + return fileType; + } + + protected StructType sparkFileType() { + return sparkFileType; + } + } + + private static class ManifestWriterFactory implements Serializable { + private final Broadcast

    tableBroadcast; + private final int formatVersion; + private final int specId; + private final String outputLocation; + private final long maxManifestSizeBytes; + + ManifestWriterFactory( + Broadcast
    tableBroadcast, + int formatVersion, + int specId, + String outputLocation, + long maxManifestSizeBytes) { + this.tableBroadcast = tableBroadcast; + this.formatVersion = formatVersion; + this.specId = specId; + this.outputLocation = outputLocation; + this.maxManifestSizeBytes = maxManifestSizeBytes; + } + + public RollingManifestWriter newRollingManifestWriter() { + return new RollingManifestWriter<>(this::newManifestWriter, maxManifestSizeBytes); + } + + private ManifestWriter newManifestWriter() { + return ManifestFiles.write(formatVersion, spec(), newOutputFile(), null); + } + + public RollingManifestWriter newRollingDeleteManifestWriter() { + return new RollingManifestWriter<>(this::newDeleteManifestWriter, maxManifestSizeBytes); + } + + private ManifestWriter newDeleteManifestWriter() { + return ManifestFiles.writeDeleteManifest(formatVersion, spec(), newOutputFile(), null); + } + + private PartitionSpec spec() { + return table().specs().get(specId); + } + + private OutputFile newOutputFile() { + return table().io().newOutputFile(newManifestLocation()); + } + + private String newManifestLocation() { + String fileName = FileFormat.AVRO.addExtension("optimized-m-" + UUID.randomUUID()); + Path filePath = new Path(outputLocation, fileName); + return filePath.toString(); + } + + private Table table() { + return tableBroadcast.value(); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java new file mode 100644 index 000000000000..ca9d562194d2 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewritePositionDeleteFilesSparkAction.java @@ -0,0 +1,380 @@ +/* + * 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.actions; + +import java.math.RoundingMode; +import java.util.List; +import java.util.Locale; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.stream.Collectors; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.PositionDeletesTable; +import org.apache.iceberg.PositionDeletesTable.PositionDeletesBatchScan; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableUtil; +import org.apache.iceberg.actions.BinPackRewritePositionDeletePlanner; +import org.apache.iceberg.actions.FileRewritePlan; +import org.apache.iceberg.actions.ImmutableRewritePositionDeleteFiles; +import org.apache.iceberg.actions.RewritePositionDeleteFiles; +import org.apache.iceberg.actions.RewritePositionDeletesCommitManager; +import org.apache.iceberg.actions.RewritePositionDeletesCommitManager.CommitService; +import org.apache.iceberg.actions.RewritePositionDeletesGroup; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Queues; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.math.IntMath; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.iceberg.spark.SparkUtil; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Spark implementation of {@link RewritePositionDeleteFiles}. */ +public class RewritePositionDeleteFilesSparkAction + extends BaseSnapshotUpdateSparkAction + implements RewritePositionDeleteFiles { + + private static final Logger LOG = + LoggerFactory.getLogger(RewritePositionDeleteFilesSparkAction.class); + private static final Set VALID_OPTIONS = + ImmutableSet.of( + MAX_CONCURRENT_FILE_GROUP_REWRITES, + PARTIAL_PROGRESS_ENABLED, + PARTIAL_PROGRESS_MAX_COMMITS, + REWRITE_JOB_ORDER); + private static final Result EMPTY_RESULT = + ImmutableRewritePositionDeleteFiles.Result.builder().build(); + + private final Table table; + private BinPackRewritePositionDeletePlanner planner; + private final SparkRewritePositionDeleteRunner runner; + private Expression filter = Expressions.alwaysTrue(); + + private int maxConcurrentFileGroupRewrites; + private int maxCommits; + private boolean partialProgressEnabled; + private boolean caseSensitive; + + RewritePositionDeleteFilesSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.runner = new SparkRewritePositionDeleteRunner(spark(), table); + this.caseSensitive = SparkUtil.caseSensitive(spark); + } + + @Override + protected RewritePositionDeleteFilesSparkAction self() { + return this; + } + + @Override + public RewritePositionDeleteFilesSparkAction filter(Expression expression) { + filter = Expressions.and(filter, expression); + return this; + } + + @Override + public RewritePositionDeleteFiles.Result execute() { + if (table.currentSnapshot() == null) { + LOG.info("Nothing found to rewrite in empty table {}", table.name()); + return EMPTY_RESULT; + } + + this.planner = new BinPackRewritePositionDeletePlanner(table, filter, caseSensitive); + + validateAndInitOptions(); + + if (TableUtil.formatVersion(table) >= 3 && !requiresRewriteToDVs()) { + LOG.info("v2 deletes in {} have already been rewritten to v3 DVs", table.name()); + return EMPTY_RESULT; + } + + FileRewritePlan + plan = planner.plan(); + + if (plan.totalGroupCount() == 0) { + LOG.info("Nothing found to rewrite in {}", table.name()); + return EMPTY_RESULT; + } + + if (partialProgressEnabled) { + return doExecuteWithPartialProgress(plan, commitManager()); + } else { + return doExecute(plan, commitManager()); + } + } + + private boolean requiresRewriteToDVs() { + PositionDeletesBatchScan scan = + (PositionDeletesBatchScan) + MetadataTableUtils.createMetadataTableInstance( + table, MetadataTableType.POSITION_DELETES) + .newBatchScan(); + try (CloseableIterator it = + CloseableIterable.filter( + CloseableIterable.transform( + scan.baseTableFilter(filter) + .caseSensitive(caseSensitive) + .select(PositionDeletesTable.DELETE_FILE_PATH) + .ignoreResiduals() + .planFiles(), + task -> (PositionDeletesScanTask) task), + t -> t.file().format() != FileFormat.PUFFIN) + .iterator()) { + return it.hasNext(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private RewritePositionDeletesGroup rewriteDeleteFiles( + FileRewritePlan< + FileGroupInfo, PositionDeletesScanTask, DeleteFile, RewritePositionDeletesGroup> + plan, + RewritePositionDeletesGroup fileGroup) { + String desc = jobDesc(fileGroup, plan); + Set addedFiles = + withJobGroupInfo( + newJobGroupInfo("REWRITE-POSITION-DELETES", desc), () -> runner.rewrite(fileGroup)); + + fileGroup.setOutputFiles(addedFiles); + LOG.info("Rewrite position deletes ready to be committed - {}", desc); + return fileGroup; + } + + private ExecutorService rewriteService() { + return MoreExecutors.getExitingExecutorService( + (ThreadPoolExecutor) + Executors.newFixedThreadPool( + maxConcurrentFileGroupRewrites, + new ThreadFactoryBuilder() + .setNameFormat("Rewrite-Position-Delete-Service-%d") + .build())); + } + + private RewritePositionDeletesCommitManager commitManager() { + return new RewritePositionDeletesCommitManager(table, commitSummary()); + } + + private Result doExecute( + FileRewritePlan< + FileGroupInfo, PositionDeletesScanTask, DeleteFile, RewritePositionDeletesGroup> + plan, + RewritePositionDeletesCommitManager commitManager) { + ExecutorService rewriteService = rewriteService(); + + ConcurrentLinkedQueue rewrittenGroups = + Queues.newConcurrentLinkedQueue(); + + Tasks.Builder rewriteTaskBuilder = + Tasks.foreach(plan.groups()) + .executeWith(rewriteService) + .stopOnFailure() + .noRetry() + .onFailure( + (fileGroup, exception) -> + LOG.warn( + "Failure during rewrite process for group {}", + fileGroup.info(), + exception)); + + try { + rewriteTaskBuilder.run(fileGroup -> rewrittenGroups.add(rewriteDeleteFiles(plan, fileGroup))); + } catch (Exception e) { + // At least one rewrite group failed, clean up all completed rewrites + LOG.error( + "Cannot complete rewrite, {} is not enabled and one of the file set groups failed to " + + "be rewritten. This error occurred during the writing of new files, not during the commit process. This " + + "indicates something is wrong that doesn't involve conflicts with other Iceberg operations. Enabling " + + "{} may help in this case but the root cause should be investigated. Cleaning up {} groups which finished " + + "being written.", + PARTIAL_PROGRESS_ENABLED, + PARTIAL_PROGRESS_ENABLED, + rewrittenGroups.size(), + e); + + Tasks.foreach(rewrittenGroups).suppressFailureWhenFinished().run(commitManager::abort); + throw e; + } finally { + rewriteService.shutdown(); + } + + try { + commitManager.commitOrClean(Sets.newHashSet(rewrittenGroups)); + } catch (ValidationException | CommitFailedException e) { + String errorMessage = + String.format( + "Cannot commit rewrite because of a ValidationException or CommitFailedException. This usually means that " + + "this rewrite has conflicted with another concurrent Iceberg operation. To reduce the likelihood of " + + "conflicts, set %s which will break up the rewrite into multiple smaller commits controlled by %s. " + + "Separate smaller rewrite commits can succeed independently while any commits that conflict with " + + "another Iceberg operation will be ignored. This mode will create additional snapshots in the table " + + "history, one for each commit.", + PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS); + throw new RuntimeException(errorMessage, e); + } + + List rewriteResults = + rewrittenGroups.stream() + .map(RewritePositionDeletesGroup::asResult) + .collect(Collectors.toList()); + + return ImmutableRewritePositionDeleteFiles.Result.builder() + .rewriteResults(rewriteResults) + .build(); + } + + private Result doExecuteWithPartialProgress( + FileRewritePlan< + FileGroupInfo, PositionDeletesScanTask, DeleteFile, RewritePositionDeletesGroup> + plan, + RewritePositionDeletesCommitManager commitManager) { + ExecutorService rewriteService = rewriteService(); + + // start commit service + int groupsPerCommit = IntMath.divide(plan.totalGroupCount(), maxCommits, RoundingMode.CEILING); + CommitService commitService = commitManager.service(groupsPerCommit); + commitService.start(); + + // start rewrite tasks + Tasks.foreach(plan.groups()) + .suppressFailureWhenFinished() + .executeWith(rewriteService) + .noRetry() + .onFailure( + (fileGroup, exception) -> + LOG.error("Failure during rewrite group {}", fileGroup.info(), exception)) + .run(fileGroup -> commitService.offer(rewriteDeleteFiles(plan, fileGroup))); + rewriteService.shutdown(); + + // stop commit service + commitService.close(); + List commitResults = commitService.results(); + if (commitResults.isEmpty()) { + LOG.error( + "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual " + + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation " + + "into smaller commits.", + PARTIAL_PROGRESS_ENABLED, + PARTIAL_PROGRESS_MAX_COMMITS); + } + + List rewriteResults = + commitResults.stream() + .map(RewritePositionDeletesGroup::asResult) + .collect(Collectors.toList()); + return ImmutableRewritePositionDeleteFiles.Result.builder() + .rewriteResults(rewriteResults) + .build(); + } + + private void validateAndInitOptions() { + Set validOptions = Sets.newHashSet(planner.validOptions()); + validOptions.addAll(VALID_OPTIONS); + validOptions.addAll(planner.validOptions()); + + Set invalidKeys = Sets.newHashSet(options().keySet()); + invalidKeys.removeAll(validOptions); + + Preconditions.checkArgument( + invalidKeys.isEmpty(), + "Cannot use options %s, they are not supported by the action or the rewriter %s", + invalidKeys, + runner.description()); + + planner.init(options()); + runner.init(options()); + + this.maxConcurrentFileGroupRewrites = + PropertyUtil.propertyAsInt( + options(), + MAX_CONCURRENT_FILE_GROUP_REWRITES, + MAX_CONCURRENT_FILE_GROUP_REWRITES_DEFAULT); + + this.maxCommits = + PropertyUtil.propertyAsInt( + options(), PARTIAL_PROGRESS_MAX_COMMITS, PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT); + + this.partialProgressEnabled = + PropertyUtil.propertyAsBoolean( + options(), PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_ENABLED_DEFAULT); + + Preconditions.checkArgument( + maxConcurrentFileGroupRewrites >= 1, + "Cannot set %s to %s, the value must be positive.", + MAX_CONCURRENT_FILE_GROUP_REWRITES, + maxConcurrentFileGroupRewrites); + + Preconditions.checkArgument( + !partialProgressEnabled || maxCommits > 0, + "Cannot set %s to %s, the value must be positive when %s is true", + PARTIAL_PROGRESS_MAX_COMMITS, + maxCommits, + PARTIAL_PROGRESS_ENABLED); + } + + private String jobDesc( + RewritePositionDeletesGroup group, + FileRewritePlan< + FileGroupInfo, PositionDeletesScanTask, DeleteFile, RewritePositionDeletesGroup> + plan) { + StructLike partition = group.info().partition(); + if (partition.size() > 0) { + return String.format( + Locale.ROOT, + "Rewriting %d position delete files (%s, file group %d/%d, %s (%d/%d)) in %s", + group.rewrittenDeleteFiles().size(), + runner.description(), + group.info().globalIndex(), + plan.totalGroupCount(), + partition, + group.info().partitionIndex(), + plan.groupsInPartition(partition), + table.name()); + } else { + return String.format( + Locale.ROOT, + "Rewriting %d position files (%s, file group %d/%d) in %s", + group.rewrittenDeleteFiles().size(), + runner.description(), + group.info().globalIndex(), + plan.totalGroupCount(), + table.name()); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java new file mode 100644 index 000000000000..d6a13bcd515d --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteTablePathSparkAction.java @@ -0,0 +1,821 @@ +/* + * 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.actions; + +import java.io.BufferedWriter; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.nio.charset.StandardCharsets; +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.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionStatisticsFile; +import org.apache.iceberg.RewriteTablePathUtil; +import org.apache.iceberg.RewriteTablePathUtil.PositionDeleteReaderWriter; +import org.apache.iceberg.RewriteTablePathUtil.RewriteResult; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StaticTableOperations; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadata.MetadataLogEntry; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.actions.ImmutableRewriteTablePath; +import org.apache.iceberg.actions.RewriteTablePath; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataWriter; +import org.apache.iceberg.data.avro.PlannedDataReader; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.data.orc.GenericOrcWriter; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.DeleteSchemaUtil; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +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.JobGroupInfo; +import org.apache.iceberg.spark.source.SerializableTableWithSize; +import org.apache.iceberg.util.Pair; +import org.apache.spark.api.java.function.ForeachFunction; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.api.java.function.ReduceFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RewriteTablePathSparkAction extends BaseSparkAction + implements RewriteTablePath { + + private static final Logger LOG = LoggerFactory.getLogger(RewriteTablePathSparkAction.class); + private static final String RESULT_LOCATION = "file-list"; + static final String NOT_APPLICABLE = "N/A"; + + private String sourcePrefix; + private String targetPrefix; + private String startVersionName; + private String endVersionName; + private String stagingDir; + private boolean createFileList = true; + + private final Table table; + private Broadcast
    tableBroadcast = null; + + RewriteTablePathSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + } + + @Override + protected RewriteTablePath self() { + return this; + } + + @Override + public RewriteTablePath rewriteLocationPrefix(String sPrefix, String tPrefix) { + Preconditions.checkArgument( + sPrefix != null && !sPrefix.isEmpty(), "Source prefix('%s') cannot be empty.", sPrefix); + this.sourcePrefix = sPrefix; + this.targetPrefix = tPrefix; + return this; + } + + @Override + public RewriteTablePath startVersion(String sVersion) { + Preconditions.checkArgument( + sVersion != null && !sVersion.trim().isEmpty(), + "Start version('%s') cannot be empty.", + sVersion); + this.startVersionName = sVersion; + return this; + } + + @Override + public RewriteTablePath endVersion(String eVersion) { + Preconditions.checkArgument( + eVersion != null && !eVersion.trim().isEmpty(), + "End version('%s') cannot be empty.", + eVersion); + this.endVersionName = eVersion; + return this; + } + + @Override + public RewriteTablePath stagingLocation(String stagingLocation) { + Preconditions.checkArgument( + stagingLocation != null && !stagingLocation.isEmpty(), + "Staging location('%s') cannot be empty.", + stagingLocation); + this.stagingDir = stagingLocation; + return this; + } + + @Override + public RewriteTablePath createFileList(boolean createFileListFlag) { + this.createFileList = createFileListFlag; + return this; + } + + @Override + public Result execute() { + validateInputs(); + JobGroupInfo info = newJobGroupInfo("REWRITE-TABLE-PATH", jobDesc()); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + return rebuildMetadata(); + } + + private void validateInputs() { + Preconditions.checkArgument( + sourcePrefix != null && !sourcePrefix.isEmpty(), + "Source prefix('%s') cannot be empty.", + sourcePrefix); + Preconditions.checkArgument( + targetPrefix != null && !targetPrefix.isEmpty(), + "Target prefix('%s') cannot be empty.", + targetPrefix); + Preconditions.checkArgument( + !sourcePrefix.equals(targetPrefix), + "Source prefix cannot be the same as target prefix (%s)", + sourcePrefix); + + validateAndSetEndVersion(); + validateAndSetStartVersion(); + + if (stagingDir == null) { + stagingDir = + getMetadataLocation(table) + + "copy-table-staging-" + + UUID.randomUUID() + + RewriteTablePathUtil.FILE_SEPARATOR; + } else { + stagingDir = RewriteTablePathUtil.maybeAppendFileSeparator(stagingDir); + } + } + + private void validateAndSetEndVersion() { + TableMetadata tableMetadata = ((HasTableOperations) table).operations().current(); + + if (endVersionName == null) { + LOG.info("No end version specified. Will stage all files to the latest table version."); + Preconditions.checkNotNull( + tableMetadata.metadataFileLocation(), "Metadata file location should not be null"); + this.endVersionName = tableMetadata.metadataFileLocation(); + } else { + this.endVersionName = validateVersion(tableMetadata, endVersionName); + } + } + + private void validateAndSetStartVersion() { + TableMetadata tableMetadata = ((HasTableOperations) table).operations().current(); + + if (startVersionName != null) { + this.startVersionName = validateVersion(tableMetadata, startVersionName); + } + } + + private String validateVersion(TableMetadata tableMetadata, String versionFileName) { + String versionFile = null; + if (versionInFilePath(tableMetadata.metadataFileLocation(), versionFileName)) { + versionFile = tableMetadata.metadataFileLocation(); + } + + for (MetadataLogEntry log : tableMetadata.previousFiles()) { + if (versionInFilePath(log.file(), versionFileName)) { + versionFile = log.file(); + } + } + + Preconditions.checkArgument( + versionFile != null, + "Cannot find provided version file %s in metadata log.", + versionFileName); + Preconditions.checkArgument( + fileExist(versionFile), "Version file %s does not exist.", versionFile); + return versionFile; + } + + private boolean versionInFilePath(String path, String version) { + return RewriteTablePathUtil.fileName(path).equals(version); + } + + private String jobDesc() { + if (startVersionName == null) { + return String.format( + "Replacing path prefixes '%s' with '%s' in the metadata files of table %s," + + "up to version '%s'.", + sourcePrefix, targetPrefix, table.name(), endVersionName); + } else { + return String.format( + "Replacing path prefixes '%s' with '%s' in the metadata files of table %s," + + "from version '%s' to '%s'.", + sourcePrefix, targetPrefix, table.name(), startVersionName, endVersionName); + } + } + + /** + * Rebuild metadata in a staging location, with paths rewritten. + * + *
      + *
    • Rebuild version files to staging + *
    • Rebuild manifest list files to staging + *
    • Rebuild manifest to staging + *
    • Get all files needed to move + *
    + */ + private Result rebuildMetadata() { + TableMetadata startMetadata = + startVersionName != null + ? ((HasTableOperations) newStaticTable(startVersionName, table.io())) + .operations() + .current() + : null; + TableMetadata endMetadata = + ((HasTableOperations) newStaticTable(endVersionName, table.io())).operations().current(); + + // rebuild version files + RewriteResult rewriteVersionResult = rewriteVersionFiles(endMetadata); + Set deltaSnapshots = deltaSnapshots(startMetadata, rewriteVersionResult.toRewrite()); + + Set manifestsToRewrite = manifestsToRewrite(deltaSnapshots, startMetadata); + Set validSnapshots = + Sets.difference(snapshotSet(endMetadata), snapshotSet(startMetadata)); + + // rebuild manifest-list files + RewriteResult rewriteManifestListResult = + validSnapshots.stream() + .map(snapshot -> rewriteManifestList(snapshot, endMetadata, manifestsToRewrite)) + .reduce(new RewriteResult<>(), RewriteResult::append); + + // rebuild manifest files + Set metaFiles = rewriteManifestListResult.toRewrite(); + RewriteContentFileResult rewriteManifestResult = + rewriteManifests(deltaSnapshots, endMetadata, rewriteManifestListResult.toRewrite()); + + // rebuild position delete files + Set deleteFiles = + rewriteManifestResult.toRewrite().stream() + .filter(e -> e instanceof DeleteFile) + .map(e -> (DeleteFile) e) + .collect(Collectors.toSet()); + rewritePositionDeletes(deleteFiles); + + ImmutableRewriteTablePath.Result.Builder builder = + ImmutableRewriteTablePath.Result.builder() + .stagingLocation(stagingDir) + .rewrittenDeleteFilePathsCount(deleteFiles.size()) + .rewrittenManifestFilePathsCount(metaFiles.size()) + .latestVersion(RewriteTablePathUtil.fileName(endVersionName)); + + if (!createFileList) { + return builder.fileListLocation(NOT_APPLICABLE).build(); + } + + Set> copyPlan = Sets.newHashSet(); + copyPlan.addAll(rewriteVersionResult.copyPlan()); + copyPlan.addAll(rewriteManifestListResult.copyPlan()); + copyPlan.addAll(rewriteManifestResult.copyPlan()); + String fileListLocation = saveFileList(copyPlan); + + return builder.fileListLocation(fileListLocation).build(); + } + + private String saveFileList(Set> filesToMove) { + String fileListPath = stagingDir + RESULT_LOCATION; + OutputFile fileList = table.io().newOutputFile(fileListPath); + writeAsCsv(filesToMove, fileList); + return fileListPath; + } + + private void writeAsCsv(Set> rows, OutputFile outputFile) { + try (BufferedWriter writer = + new BufferedWriter( + new OutputStreamWriter(outputFile.createOrOverwrite(), StandardCharsets.UTF_8))) { + for (Pair pair : rows) { + writer.write(String.join(",", pair.first(), pair.second())); + writer.newLine(); + } + } catch (IOException e) { + throw new RuntimeIOException(e); + } + } + + private Set deltaSnapshots(TableMetadata startMetadata, Set allSnapshots) { + if (startMetadata == null) { + return allSnapshots; + } else { + Set startSnapshotIds = + startMetadata.snapshots().stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); + return allSnapshots.stream() + .filter(s -> !startSnapshotIds.contains(s.snapshotId())) + .collect(Collectors.toSet()); + } + } + + private RewriteResult rewriteVersionFiles(TableMetadata endMetadata) { + RewriteResult result = new RewriteResult<>(); + result.toRewrite().addAll(endMetadata.snapshots()); + result.copyPlan().addAll(rewriteVersionFile(endMetadata, endVersionName)); + + List versions = endMetadata.previousFiles(); + for (int i = versions.size() - 1; i >= 0; i--) { + String versionFilePath = versions.get(i).file(); + if (versionFilePath.equals(startVersionName)) { + break; + } + + Preconditions.checkArgument( + fileExist(versionFilePath), + String.format("Version file %s doesn't exist", versionFilePath)); + TableMetadata tableMetadata = + new StaticTableOperations(versionFilePath, table.io()).current(); + + result.toRewrite().addAll(tableMetadata.snapshots()); + result.copyPlan().addAll(rewriteVersionFile(tableMetadata, versionFilePath)); + } + + return result; + } + + private Set> rewriteVersionFile( + TableMetadata metadata, String versionFilePath) { + Set> result = Sets.newHashSet(); + String stagingPath = + RewriteTablePathUtil.stagingPath(versionFilePath, sourcePrefix, stagingDir); + TableMetadata newTableMetadata = + RewriteTablePathUtil.replacePaths(metadata, sourcePrefix, targetPrefix); + TableMetadataParser.overwrite(newTableMetadata, table.io().newOutputFile(stagingPath)); + result.add( + Pair.of( + stagingPath, + RewriteTablePathUtil.newPath(versionFilePath, sourcePrefix, targetPrefix))); + + // include statistics files in copy plan + result.addAll( + statsFileCopyPlan(metadata.statisticsFiles(), newTableMetadata.statisticsFiles())); + result.addAll( + partitionStatsFileCopyPlan( + metadata.partitionStatisticsFiles(), newTableMetadata.partitionStatisticsFiles())); + return result; + } + + private Set> statsFileCopyPlan( + List beforeStats, List afterStats) { + Set> result = Sets.newHashSet(); + if (beforeStats.isEmpty()) { + return result; + } + + Preconditions.checkArgument( + beforeStats.size() == afterStats.size(), + "Before and after path rewrite, statistic files count should be same"); + for (int i = 0; i < beforeStats.size(); i++) { + StatisticsFile before = beforeStats.get(i); + StatisticsFile after = afterStats.get(i); + Preconditions.checkArgument( + before.fileSizeInBytes() == after.fileSizeInBytes(), + "Before and after path rewrite, statistic file size should be same"); + result.add(Pair.of(before.path(), after.path())); + } + return result; + } + + private Set> partitionStatsFileCopyPlan( + List beforeStats, List afterStats) { + Set> result = Sets.newHashSet(); + if (beforeStats.isEmpty()) { + return result; + } + + Preconditions.checkArgument( + beforeStats.size() == afterStats.size(), + "Before and after path rewrite, partition statistic files count should be same"); + for (int i = 0; i < beforeStats.size(); i++) { + PartitionStatisticsFile before = beforeStats.get(i); + PartitionStatisticsFile after = afterStats.get(i); + Preconditions.checkArgument( + before.fileSizeInBytes() == after.fileSizeInBytes(), + "Before and after path rewrite, partition statistic file size should be same"); + result.add(Pair.of(before.path(), after.path())); + } + return result; + } + + /** + * Rewrite a manifest list representing a snapshot. + * + * @param snapshot snapshot represented by the manifest list + * @param tableMetadata metadata of table + * @param manifestsToRewrite filter of manifests to rewrite. + * @return a result including a copy plan for the manifests contained in the manifest list, as + * well as for the manifest list itself + */ + private RewriteResult rewriteManifestList( + Snapshot snapshot, TableMetadata tableMetadata, Set manifestsToRewrite) { + RewriteResult result = new RewriteResult<>(); + + String path = snapshot.manifestListLocation(); + String outputPath = RewriteTablePathUtil.stagingPath(path, sourcePrefix, stagingDir); + RewriteResult rewriteResult = + RewriteTablePathUtil.rewriteManifestList( + snapshot, + table.io(), + tableMetadata, + manifestsToRewrite, + sourcePrefix, + targetPrefix, + stagingDir, + outputPath); + + result.append(rewriteResult); + // add the manifest list copy plan itself to the result + result + .copyPlan() + .add(Pair.of(outputPath, RewriteTablePathUtil.newPath(path, sourcePrefix, targetPrefix))); + return result; + } + + private Set manifestsToRewrite( + Set deltaSnapshots, TableMetadata startMetadata) { + try { + Table endStaticTable = newStaticTable(endVersionName, table.io()); + Dataset lastVersionFiles = manifestDS(endStaticTable).select("path"); + if (startMetadata == null) { + return Sets.newHashSet(lastVersionFiles.distinct().as(Encoders.STRING()).collectAsList()); + } else { + Set deltaSnapshotIds = + deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); + return Sets.newHashSet( + lastVersionFiles + .distinct() + .filter( + functions + .column(ManifestFile.SNAPSHOT_ID.name()) + .isInCollection(deltaSnapshotIds)) + .as(Encoders.STRING()) + .collectAsList()); + } + } catch (Exception e) { + throw new UnsupportedOperationException( + "Unable to build the manifest files dataframe. The end version in use may contain invalid snapshots. " + + "Please choose an earlier version without invalid snapshots.", + e); + } + } + + public static class RewriteContentFileResult extends RewriteResult> { + @Override + public RewriteContentFileResult append(RewriteResult> r1) { + this.copyPlan().addAll(r1.copyPlan()); + this.toRewrite().addAll(r1.toRewrite()); + return this; + } + + public RewriteContentFileResult appendDataFile(RewriteResult r1) { + this.copyPlan().addAll(r1.copyPlan()); + this.toRewrite().addAll(r1.toRewrite()); + return this; + } + + public RewriteContentFileResult appendDeleteFile(RewriteResult r1) { + this.copyPlan().addAll(r1.copyPlan()); + this.toRewrite().addAll(r1.toRewrite()); + return this; + } + } + + /** Rewrite manifest files in a distributed manner and return rewritten data files path pairs. */ + private RewriteContentFileResult rewriteManifests( + Set deltaSnapshots, TableMetadata tableMetadata, Set toRewrite) { + if (toRewrite.isEmpty()) { + return new RewriteContentFileResult(); + } + + Encoder manifestFileEncoder = Encoders.javaSerialization(ManifestFile.class); + Dataset manifestDS = + spark().createDataset(Lists.newArrayList(toRewrite), manifestFileEncoder); + Set deltaSnapshotIds = + deltaSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet()); + + return manifestDS + .repartition(toRewrite.size()) + .map( + toManifests( + tableBroadcast(), + sparkContext().broadcast(deltaSnapshotIds), + stagingDir, + tableMetadata.formatVersion(), + sourcePrefix, + targetPrefix), + Encoders.bean(RewriteContentFileResult.class)) + // duplicates are expected here as the same data file can have different statuses + // (e.g. added and deleted) + .reduce((ReduceFunction) RewriteContentFileResult::append); + } + + private static MapFunction toManifests( + Broadcast
    table, + Broadcast> deltaSnapshotIds, + String stagingLocation, + int format, + String sourcePrefix, + String targetPrefix) { + + return manifestFile -> { + RewriteContentFileResult result = new RewriteContentFileResult(); + switch (manifestFile.content()) { + case DATA: + result.appendDataFile( + writeDataManifest( + manifestFile, + table, + deltaSnapshotIds, + stagingLocation, + format, + sourcePrefix, + targetPrefix)); + break; + case DELETES: + result.appendDeleteFile( + writeDeleteManifest( + manifestFile, + table, + deltaSnapshotIds, + stagingLocation, + format, + sourcePrefix, + targetPrefix)); + break; + default: + throw new UnsupportedOperationException( + "Unsupported manifest type: " + manifestFile.content()); + } + return result; + }; + } + + private static RewriteResult writeDataManifest( + ManifestFile manifestFile, + Broadcast
    table, + Broadcast> snapshotIds, + String stagingLocation, + int format, + String sourcePrefix, + String targetPrefix) { + try { + String stagingPath = + RewriteTablePathUtil.stagingPath(manifestFile.path(), sourcePrefix, stagingLocation); + FileIO io = table.getValue().io(); + OutputFile outputFile = io.newOutputFile(stagingPath); + Map specsById = table.getValue().specs(); + Set deltaSnapshotIds = snapshotIds.value(); + return RewriteTablePathUtil.rewriteDataManifest( + manifestFile, + deltaSnapshotIds, + outputFile, + io, + format, + specsById, + sourcePrefix, + targetPrefix); + } catch (IOException e) { + throw new RuntimeIOException(e); + } + } + + private static RewriteResult writeDeleteManifest( + ManifestFile manifestFile, + Broadcast
    table, + Broadcast> snapshotIds, + String stagingLocation, + int format, + String sourcePrefix, + String targetPrefix) { + try { + String stagingPath = + RewriteTablePathUtil.stagingPath(manifestFile.path(), sourcePrefix, stagingLocation); + FileIO io = table.getValue().io(); + OutputFile outputFile = io.newOutputFile(stagingPath); + Map specsById = table.getValue().specs(); + Set deltaSnapshotIds = snapshotIds.value(); + return RewriteTablePathUtil.rewriteDeleteManifest( + manifestFile, + deltaSnapshotIds, + outputFile, + io, + format, + specsById, + sourcePrefix, + targetPrefix, + stagingLocation); + } catch (IOException e) { + throw new RuntimeIOException(e); + } + } + + private void rewritePositionDeletes(Set toRewrite) { + if (toRewrite.isEmpty()) { + return; + } + + Encoder deleteFileEncoder = Encoders.javaSerialization(DeleteFile.class); + Dataset deleteFileDs = + spark().createDataset(Lists.newArrayList(toRewrite), deleteFileEncoder); + + PositionDeleteReaderWriter posDeleteReaderWriter = new SparkPositionDeleteReaderWriter(); + deleteFileDs + .repartition(toRewrite.size()) + .foreach( + rewritePositionDelete( + tableBroadcast(), sourcePrefix, targetPrefix, stagingDir, posDeleteReaderWriter)); + } + + private static class SparkPositionDeleteReaderWriter implements PositionDeleteReaderWriter { + @Override + public CloseableIterable reader( + InputFile inputFile, FileFormat format, PartitionSpec spec) { + return positionDeletesReader(inputFile, format, spec); + } + + @Override + public PositionDeleteWriter writer( + OutputFile outputFile, + FileFormat format, + PartitionSpec spec, + StructLike partition, + Schema rowSchema) + throws IOException { + return positionDeletesWriter(outputFile, format, spec, partition, rowSchema); + } + } + + private ForeachFunction rewritePositionDelete( + Broadcast
    tableArg, + String sourcePrefixArg, + String targetPrefixArg, + String stagingLocationArg, + PositionDeleteReaderWriter posDeleteReaderWriter) { + return deleteFile -> { + FileIO io = tableArg.getValue().io(); + String newPath = + RewriteTablePathUtil.stagingPath( + deleteFile.location(), sourcePrefixArg, stagingLocationArg); + OutputFile outputFile = io.newOutputFile(newPath); + PartitionSpec spec = tableArg.getValue().specs().get(deleteFile.specId()); + RewriteTablePathUtil.rewritePositionDeleteFile( + deleteFile, + outputFile, + io, + spec, + sourcePrefixArg, + targetPrefixArg, + posDeleteReaderWriter); + }; + } + + private static CloseableIterable positionDeletesReader( + InputFile inputFile, FileFormat format, PartitionSpec spec) { + Schema deleteSchema = DeleteSchemaUtil.posDeleteReadSchema(spec.schema()); + switch (format) { + case AVRO: + return Avro.read(inputFile) + .project(deleteSchema) + .reuseContainers() + .createReaderFunc(fileSchema -> PlannedDataReader.create(deleteSchema)) + .build(); + + case PARQUET: + return Parquet.read(inputFile) + .project(deleteSchema) + .reuseContainers() + .createReaderFunc( + fileSchema -> GenericParquetReaders.buildReader(deleteSchema, fileSchema)) + .build(); + + case ORC: + return ORC.read(inputFile) + .project(deleteSchema) + .createReaderFunc(fileSchema -> GenericOrcReader.buildReader(deleteSchema, fileSchema)) + .build(); + + default: + throw new UnsupportedOperationException("Unsupported file format: " + format); + } + } + + private static PositionDeleteWriter positionDeletesWriter( + OutputFile outputFile, + FileFormat format, + PartitionSpec spec, + StructLike partition, + Schema rowSchema) + throws IOException { + switch (format) { + case AVRO: + return Avro.writeDeletes(outputFile) + .createWriterFunc(DataWriter::create) + .withPartition(partition) + .rowSchema(rowSchema) + .withSpec(spec) + .buildPositionWriter(); + case PARQUET: + return Parquet.writeDeletes(outputFile) + .createWriterFunc(GenericParquetWriter::create) + .withPartition(partition) + .rowSchema(rowSchema) + .withSpec(spec) + .buildPositionWriter(); + case ORC: + return ORC.writeDeletes(outputFile) + .createWriterFunc(GenericOrcWriter::buildWriter) + .withPartition(partition) + .rowSchema(rowSchema) + .withSpec(spec) + .buildPositionWriter(); + default: + throw new UnsupportedOperationException("Unsupported file format: " + format); + } + } + + private Set snapshotSet(TableMetadata metadata) { + if (metadata == null) { + return Sets.newHashSet(); + } else { + return Sets.newHashSet(metadata.snapshots()); + } + } + + private boolean fileExist(String path) { + if (path == null || path.trim().isEmpty()) { + return false; + } + return table.io().newInputFile(path).exists(); + } + + private String getMetadataLocation(Table tbl) { + String currentMetadataPath = + ((HasTableOperations) tbl).operations().current().metadataFileLocation(); + int lastIndex = currentMetadataPath.lastIndexOf(RewriteTablePathUtil.FILE_SEPARATOR); + String metadataDir = ""; + if (lastIndex != -1) { + metadataDir = currentMetadataPath.substring(0, lastIndex + 1); + } + + Preconditions.checkArgument( + !metadataDir.isEmpty(), "Failed to get the metadata file root directory"); + return metadataDir; + } + + @VisibleForTesting + Broadcast
    tableBroadcast() { + if (tableBroadcast == null) { + this.tableBroadcast = sparkContext().broadcast(SerializableTableWithSize.copyOf(table)); + } + + return tableBroadcast; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SetAccumulator.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SetAccumulator.java new file mode 100644 index 000000000000..745169fc1efd --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SetAccumulator.java @@ -0,0 +1,61 @@ +/* + * 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.actions; + +import java.util.Collections; +import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.spark.util.AccumulatorV2; + +public class SetAccumulator extends AccumulatorV2> { + + private final Set set = Collections.synchronizedSet(Sets.newHashSet()); + + @Override + public boolean isZero() { + return set.isEmpty(); + } + + @Override + public AccumulatorV2> copy() { + SetAccumulator newAccumulator = new SetAccumulator<>(); + newAccumulator.set.addAll(set); + return newAccumulator; + } + + @Override + public void reset() { + set.clear(); + } + + @Override + public void add(T v) { + set.add(v); + } + + @Override + public void merge(AccumulatorV2> other) { + set.addAll(other.value()); + } + + @Override + public Set value() { + return set; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java new file mode 100644 index 000000000000..5f7f408cb099 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java @@ -0,0 +1,225 @@ +/* + * 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.actions; + +import java.util.Map; +import java.util.concurrent.ExecutorService; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.ImmutableSnapshotTable; +import org.apache.iceberg.actions.SnapshotTable; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; +import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.spark.source.StagedSparkTable; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.StagingTableCatalog; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import scala.collection.JavaConverters; + +/** + * Creates a new Iceberg table based on a source Spark table. The new Iceberg table will have a + * different data and metadata directory allowing it to exist independently of the source table. + */ +public class SnapshotTableSparkAction extends BaseTableCreationSparkAction + implements SnapshotTable { + + private static final Logger LOG = LoggerFactory.getLogger(SnapshotTableSparkAction.class); + + private StagingTableCatalog destCatalog; + private Identifier destTableIdent; + private String destTableLocation = null; + private ExecutorService executorService; + + SnapshotTableSparkAction( + SparkSession spark, CatalogPlugin sourceCatalog, Identifier sourceTableIdent) { + super(spark, sourceCatalog, sourceTableIdent); + } + + @Override + protected SnapshotTableSparkAction self() { + return this; + } + + @Override + protected StagingTableCatalog destCatalog() { + return destCatalog; + } + + @Override + protected Identifier destTableIdent() { + return destTableIdent; + } + + @Override + public SnapshotTableSparkAction as(String ident) { + String ctx = "snapshot destination"; + CatalogPlugin defaultCatalog = spark().sessionState().catalogManager().currentCatalog(); + CatalogAndIdentifier catalogAndIdent = + Spark3Util.catalogAndIdentifier(ctx, spark(), ident, defaultCatalog); + this.destCatalog = checkDestinationCatalog(catalogAndIdent.catalog()); + this.destTableIdent = catalogAndIdent.identifier(); + return this; + } + + @Override + public SnapshotTableSparkAction tableProperties(Map properties) { + setProperties(properties); + return this; + } + + @Override + public SnapshotTableSparkAction tableProperty(String property, String value) { + setProperty(property, value); + return this; + } + + @Override + public SnapshotTableSparkAction executeWith(ExecutorService service) { + this.executorService = service; + return this; + } + + @Override + public SnapshotTable.Result execute() { + String desc = String.format("Snapshotting table %s as %s", sourceTableIdent(), destTableIdent); + JobGroupInfo info = newJobGroupInfo("SNAPSHOT-TABLE", desc); + return withJobGroupInfo(info, this::doExecute); + } + + private SnapshotTable.Result doExecute() { + Preconditions.checkArgument( + destCatalog() != null && destTableIdent() != null, + "The destination catalog and identifier cannot be null. " + + "Make sure to configure the action with a valid destination table identifier via the `as` method."); + + LOG.info( + "Staging a new Iceberg table {} as a snapshot of {}", destTableIdent(), sourceTableIdent()); + StagedSparkTable stagedTable = stageDestTable(); + Table icebergTable = stagedTable.table(); + + // TODO: Check the dest table location does not overlap with the source table location + + boolean threw = true; + try { + LOG.info("Ensuring {} has a valid name mapping", destTableIdent()); + ensureNameMappingPresent(icebergTable); + + TableIdentifier v1TableIdent = v1SourceTable().identifier(); + String stagingLocation = getMetadataLocation(icebergTable); + LOG.info("Generating Iceberg metadata for {} in {}", destTableIdent(), stagingLocation); + SparkTableUtil.importSparkTable( + spark(), v1TableIdent, icebergTable, stagingLocation, executorService); + + LOG.info("Committing staged changes to {}", destTableIdent()); + stagedTable.commitStagedChanges(); + threw = false; + } finally { + if (threw) { + LOG.error("Error when populating the staged table with metadata, aborting changes"); + + try { + stagedTable.abortStagedChanges(); + } catch (Exception abortException) { + LOG.error("Cannot abort staged changes", abortException); + } + } + } + + Snapshot snapshot = icebergTable.currentSnapshot(); + long importedDataFilesCount = + Long.parseLong(snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP)); + LOG.info( + "Successfully loaded Iceberg metadata for {} files to {}", + importedDataFilesCount, + destTableIdent()); + return ImmutableSnapshotTable.Result.builder() + .importedDataFilesCount(importedDataFilesCount) + .build(); + } + + @Override + protected Map destTableProps() { + Map properties = Maps.newHashMap(); + + // copy over relevant source table props + properties.putAll(JavaConverters.mapAsJavaMapConverter(v1SourceTable().properties()).asJava()); + EXCLUDED_PROPERTIES.forEach(properties::remove); + + // remove any possible location properties from origin properties + properties.remove(LOCATION); + properties.remove(TableProperties.WRITE_METADATA_LOCATION); + properties.remove(TableProperties.WRITE_FOLDER_STORAGE_LOCATION); + properties.remove(TableProperties.OBJECT_STORE_PATH); + properties.remove(TableProperties.WRITE_DATA_LOCATION); + + // set default and user-provided props + properties.put(TableCatalog.PROP_PROVIDER, "iceberg"); + properties.putAll(additionalProperties()); + + // make sure we mark this table as a snapshot table + properties.put(TableProperties.GC_ENABLED, "false"); + properties.put("snapshot", "true"); + + // set the destination table location if provided + if (destTableLocation != null) { + properties.put(LOCATION, destTableLocation); + } + + return properties; + } + + @Override + protected TableCatalog checkSourceCatalog(CatalogPlugin catalog) { + // currently the import code relies on being able to look up the table in the session catalog + Preconditions.checkArgument( + catalog.name().equalsIgnoreCase("spark_catalog"), + "Cannot snapshot a table that isn't in the session catalog (i.e. spark_catalog). " + + "Found source catalog: %s.", + catalog.name()); + + Preconditions.checkArgument( + catalog instanceof TableCatalog, + "Cannot snapshot as catalog %s of class %s in not a table catalog", + catalog.name(), + catalog.getClass().getName()); + + return (TableCatalog) catalog; + } + + @Override + public SnapshotTableSparkAction tableLocation(String location) { + Preconditions.checkArgument( + !sourceTableLocation().equals(location), + "The snapshot table location cannot be same as the source table location. " + + "This would mix snapshot table files with original table files."); + this.destTableLocation = location; + return this; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java new file mode 100644 index 000000000000..b7361c336a69 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkActions.java @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ActionsProvider; +import org.apache.iceberg.actions.ComputePartitionStats; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; + +/** + * An implementation of {@link ActionsProvider} for Spark. + * + *

    This class is the primary API for interacting with actions in Spark that users should use to + * instantiate particular actions. + */ +public class SparkActions implements ActionsProvider { + + private final SparkSession spark; + + private SparkActions(SparkSession spark) { + this.spark = spark; + } + + public static SparkActions get(SparkSession spark) { + return new SparkActions(spark); + } + + public static SparkActions get() { + return new SparkActions(SparkSession.active()); + } + + @Override + public SnapshotTableSparkAction snapshotTable(String tableIdent) { + String ctx = "snapshot source"; + CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); + CatalogAndIdentifier catalogAndIdent = + Spark3Util.catalogAndIdentifier(ctx, spark, tableIdent, defaultCatalog); + return new SnapshotTableSparkAction( + spark, catalogAndIdent.catalog(), catalogAndIdent.identifier()); + } + + @Override + public MigrateTableSparkAction migrateTable(String tableIdent) { + String ctx = "migrate target"; + CatalogPlugin defaultCatalog = spark.sessionState().catalogManager().currentCatalog(); + CatalogAndIdentifier catalogAndIdent = + Spark3Util.catalogAndIdentifier(ctx, spark, tableIdent, defaultCatalog); + return new MigrateTableSparkAction( + spark, catalogAndIdent.catalog(), catalogAndIdent.identifier()); + } + + @Override + public RewriteDataFilesSparkAction rewriteDataFiles(Table table) { + return new RewriteDataFilesSparkAction(spark, table); + } + + @Override + public DeleteOrphanFilesSparkAction deleteOrphanFiles(Table table) { + return new DeleteOrphanFilesSparkAction(spark, table); + } + + @Override + public RewriteManifestsSparkAction rewriteManifests(Table table) { + return new RewriteManifestsSparkAction(spark, table); + } + + @Override + public ExpireSnapshotsSparkAction expireSnapshots(Table table) { + return new ExpireSnapshotsSparkAction(spark, table); + } + + @Override + public DeleteReachableFilesSparkAction deleteReachableFiles(String metadataLocation) { + return new DeleteReachableFilesSparkAction(spark, metadataLocation); + } + + @Override + public RewritePositionDeleteFilesSparkAction rewritePositionDeletes(Table table) { + return new RewritePositionDeleteFilesSparkAction(spark, table); + } + + @Override + public ComputeTableStats computeTableStats(Table table) { + return new ComputeTableStatsSparkAction(spark, table); + } + + @Override + public ComputePartitionStats computePartitionStats(Table table) { + return new ComputePartitionStatsSparkAction(spark, table); + } + + @Override + public RemoveDanglingDeleteFiles removeDanglingDeleteFiles(Table table) { + return new RemoveDanglingDeletesSparkAction(spark, table); + } + + @Override + public RewriteTablePathSparkAction rewriteTablePath(Table table) { + return new RewriteTablePathSparkAction(spark, table); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackFileRewriteRunner.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackFileRewriteRunner.java new file mode 100644 index 000000000000..6d2ef585b1f5 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkBinPackFileRewriteRunner.java @@ -0,0 +1,71 @@ +/* + * 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.actions; + +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewriteFileGroup; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; + +class SparkBinPackFileRewriteRunner extends SparkDataFileRewriteRunner { + + SparkBinPackFileRewriteRunner(SparkSession spark, Table table) { + super(spark, table); + } + + @Override + public String description() { + return "BIN-PACK"; + } + + @Override + protected void doRewrite(String groupId, RewriteFileGroup group) { + // read the files packing them into splits of the required size + Dataset scanDF = + spark() + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, groupId) + .option(SparkReadOptions.SPLIT_SIZE, group.inputSplitSize()) + .option(SparkReadOptions.FILE_OPEN_COST, "0") + .load(groupId); + + // write the packed data into new files where each split becomes a new file + scanDF + .write() + .format("iceberg") + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupId) + .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, group.maxOutputFileSize()) + .option(SparkWriteOptions.DISTRIBUTION_MODE, distributionMode(group).modeName()) + .option(SparkWriteOptions.OUTPUT_SPEC_ID, group.outputSpecId()) + .mode("append") + .save(groupId); + } + + // invoke a shuffle if the original spec does not match the output spec + private DistributionMode distributionMode(RewriteFileGroup group) { + boolean requiresRepartition = + !group.fileScanTasks().get(0).spec().equals(spec(group.outputSpecId())); + return requiresRepartition ? DistributionMode.RANGE : DistributionMode.NONE; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkDataFileRewriteRunner.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkDataFileRewriteRunner.java new file mode 100644 index 000000000000..b37011b3dd76 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkDataFileRewriteRunner.java @@ -0,0 +1,61 @@ +/* + * 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.actions; + +import java.util.Set; +import java.util.UUID; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewriteDataFiles.FileGroupInfo; +import org.apache.iceberg.actions.RewriteFileGroup; +import org.apache.iceberg.spark.FileRewriteCoordinator; +import org.apache.iceberg.spark.ScanTaskSetManager; +import org.apache.iceberg.spark.SparkTableCache; +import org.apache.spark.sql.SparkSession; + +abstract class SparkDataFileRewriteRunner + extends SparkRewriteRunner { + private final SparkTableCache tableCache = SparkTableCache.get(); + private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); + + SparkDataFileRewriteRunner(SparkSession spark, Table table) { + super(spark, table); + } + + abstract void doRewrite(String groupId, RewriteFileGroup fileGroup); + + @Override + public Set rewrite(RewriteFileGroup group) { + String groupId = UUID.randomUUID().toString(); + try { + tableCache.add(groupId, table()); + taskSetManager.stageTasks(table(), groupId, group.fileScanTasks()); + + doRewrite(groupId, group); + + return coordinator.fetchNewFiles(table(), groupId); + } finally { + tableCache.remove(groupId); + taskSetManager.removeTasks(table(), groupId); + coordinator.clearRewrite(table(), groupId); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewritePositionDeleteRunner.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewritePositionDeleteRunner.java new file mode 100644 index 000000000000..4bbd2280565a --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewritePositionDeleteRunner.java @@ -0,0 +1,150 @@ +/* + * 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.actions; + +import static org.apache.iceberg.MetadataTableType.POSITION_DELETES; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.lit; + +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.stream.IntStream; +import org.apache.iceberg.DataFilesTable; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewritePositionDeleteFiles; +import org.apache.iceberg.actions.RewritePositionDeletesGroup; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator; +import org.apache.iceberg.spark.ScanTaskSetManager; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkTableCache; +import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.spark.SparkValueConverter; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.internal.SQLConf; + +class SparkRewritePositionDeleteRunner + extends SparkRewriteRunner< + RewritePositionDeleteFiles.FileGroupInfo, + PositionDeletesScanTask, + DeleteFile, + RewritePositionDeletesGroup> { + + private final SparkTableCache tableCache = SparkTableCache.get(); + private final ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + private final PositionDeletesRewriteCoordinator coordinator = + PositionDeletesRewriteCoordinator.get(); + + SparkRewritePositionDeleteRunner(SparkSession spark, Table table) { + // Disable Adaptive Query Execution as this may change the output partitioning of our write + super(((org.apache.spark.sql.classic.SparkSession) spark).cloneSession(), table); + this.spark().conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), false); + } + + @Override + public String description() { + return "BIN-PACK"; + } + + @Override + public Set rewrite(RewritePositionDeletesGroup group) { + String groupId = UUID.randomUUID().toString(); + Table deletesTable = MetadataTableUtils.createMetadataTableInstance(table(), POSITION_DELETES); + try { + tableCache.add(groupId, deletesTable); + taskSetManager.stageTasks(deletesTable, groupId, group.fileScanTasks()); + + doRewrite(groupId, group); + + return coordinator.fetchNewFiles(deletesTable, groupId); + } finally { + tableCache.remove(groupId); + taskSetManager.removeTasks(deletesTable, groupId); + coordinator.clearRewrite(deletesTable, groupId); + } + } + + protected void doRewrite(String groupId, RewritePositionDeletesGroup group) { + // all position deletes are of the same partition, because they are in same file group + Preconditions.checkArgument(!group.rewrittenDeleteFiles().isEmpty(), "Empty group"); + DeleteFile deleteFile = group.rewrittenDeleteFiles().iterator().next(); + Types.StructType partitionType = table().specs().get(deleteFile.specId()).partitionType(); + StructLike partition = deleteFile.partition(); + + // read the deletes packing them into splits of the required size + Dataset posDeletes = + spark() + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, groupId) + .option(SparkReadOptions.SPLIT_SIZE, group.inputSplitSize()) + .option(SparkReadOptions.FILE_OPEN_COST, "0") + .load(groupId); + + // keep only valid position deletes + Dataset dataFiles = dataFiles(partitionType, partition); + Column joinCond = posDeletes.col("file_path").equalTo(dataFiles.col("file_path")); + Dataset validDeletes = posDeletes.join(dataFiles, joinCond, "leftsemi"); + + // write the packed deletes into new files where each split becomes a new file + validDeletes + .sortWithinPartitions("file_path", "pos") + .write() + .format("iceberg") + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupId) + .option(SparkWriteOptions.TARGET_DELETE_FILE_SIZE_BYTES, group.maxOutputFileSize()) + .mode("append") + .save(groupId); + } + + /** Returns entries of {@link DataFilesTable} of specified partition */ + private Dataset dataFiles(Types.StructType partitionType, StructLike partition) { + List fields = partitionType.fields(); + Optional condition = + IntStream.range(0, fields.size()) + .mapToObj( + i -> { + Type type = fields.get(i).type(); + Object value = partition.get(i, type.typeId().javaClass()); + Object convertedValue = SparkValueConverter.convertToSpark(type, value); + Column col = col("partition.`" + fields.get(i).name() + "`"); + return col.eqNullSafe(lit(convertedValue)); + }) + .reduce(Column::and); + if (condition.isPresent()) { + return SparkTableUtil.loadMetadataTable(spark(), table(), MetadataTableType.DATA_FILES) + .filter(condition.get()); + } else { + return SparkTableUtil.loadMetadataTable(spark(), table(), MetadataTableType.DATA_FILES); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewriteRunner.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewriteRunner.java new file mode 100644 index 000000000000..cb65177061b6 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkRewriteRunner.java @@ -0,0 +1,82 @@ +/* + * 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.actions; + +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.FileRewriteRunner; +import org.apache.iceberg.actions.RewriteDataFiles; +import org.apache.iceberg.actions.RewriteFileGroup; +import org.apache.iceberg.actions.RewriteGroupBase; +import org.apache.iceberg.actions.RewritePositionDeleteFiles; +import org.apache.iceberg.actions.RewritePositionDeletesGroup; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.spark.sql.SparkSession; + +/** + * Base class for Spark file rewrite runners which implement the {@link FileRewriteRunner} API for + * Spark based file rewrites. This class encapsulates the common interface and attributes for Spark + * file rewrite runners. The actual implementation of the file rewrite logic is implemented by the + * subclasses. + * + * @param the Java type of the plan info like {@link RewriteDataFiles.FileGroupInfo} or {@link + * RewritePositionDeleteFiles.FileGroupInfo} + * @param the Java type of the input scan tasks (input) + * @param the Java type of the content files (input and output) + * @param the Java type of the rewrite file group like {@link RewriteFileGroup} or {@link + * RewritePositionDeletesGroup} + */ +abstract class SparkRewriteRunner< + I, + T extends ContentScanTask, + F extends ContentFile, + G extends RewriteGroupBase> + implements FileRewriteRunner { + private final SparkSession spark; + private final Table table; + + SparkRewriteRunner(SparkSession spark, Table table) { + this.spark = spark; + this.table = table; + } + + SparkSession spark() { + return spark; + } + + Table table() { + return table; + } + + @Override + public Set validOptions() { + return ImmutableSet.of(); + } + + @Override + public void init(Map options) {} + + PartitionSpec spec(int specId) { + return table().specs().get(specId); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewritePlanner.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewritePlanner.java new file mode 100644 index 000000000000..736114937eff --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingDataRewritePlanner.java @@ -0,0 +1,82 @@ +/* + * 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.actions; + +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.BinPackRewriteFilePlanner; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.util.PropertyUtil; + +/** + * Produces plans for shuffling rewrites. Since shuffle and sort could considerably improve the + * compression ratio, the planner introduces an additional {@link #COMPRESSION_FACTOR} option which + * is used when calculating the {@link #expectedOutputFiles(long)}. + */ +class SparkShufflingDataRewritePlanner extends BinPackRewriteFilePlanner { + /** + * The number of shuffle partitions and consequently the number of output files created by the + * Spark sort is based on the size of the input data files used in this file rewriter. Due to + * compression, the disk file sizes may not accurately represent the size of files in the output. + * This parameter lets the user adjust the file size used for estimating actual output data size. + * A factor greater than 1.0 would generate more files than we would expect based on the on-disk + * file size. A value less than 1.0 would create fewer files than we would expect based on the + * on-disk size. + */ + public static final String COMPRESSION_FACTOR = "compression-factor"; + + public static final double COMPRESSION_FACTOR_DEFAULT = 1.0; + + private double compressionFactor; + + SparkShufflingDataRewritePlanner( + Table table, Expression filter, Long snapshotId, boolean caseSensitive) { + super(table, filter, snapshotId, caseSensitive); + } + + @Override + public Set validOptions() { + return ImmutableSet.builder() + .addAll(super.validOptions()) + .add(COMPRESSION_FACTOR) + .build(); + } + + @Override + public void init(Map options) { + super.init(options); + this.compressionFactor = compressionFactor(options); + } + + @Override + protected int expectedOutputFiles(long inputSize) { + return Math.max(1, super.expectedOutputFiles((long) (inputSize * compressionFactor))); + } + + private double compressionFactor(Map options) { + double value = + PropertyUtil.propertyAsDouble(options, COMPRESSION_FACTOR, COMPRESSION_FACTOR_DEFAULT); + Preconditions.checkArgument( + value > 0, "'%s' is set to %s but must be > 0", COMPRESSION_FACTOR, value); + return value; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java new file mode 100644 index 000000000000..569eb252cba5 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java @@ -0,0 +1,228 @@ +/* + * 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.actions; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewriteFileGroup; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkFunctionCatalog; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.SortOrderUtil; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; +import org.apache.spark.sql.catalyst.plans.logical.OrderAwareCoalesce; +import org.apache.spark.sql.catalyst.plans.logical.OrderAwareCoalescer; +import org.apache.spark.sql.connector.distributions.Distribution; +import org.apache.spark.sql.connector.distributions.Distributions; +import org.apache.spark.sql.connector.distributions.OrderedDistribution; +import org.apache.spark.sql.connector.expressions.SortOrder; +import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering; +import org.apache.spark.sql.execution.datasources.v2.DistributionAndOrderingUtils$; +import scala.Option; + +abstract class SparkShufflingFileRewriteRunner extends SparkDataFileRewriteRunner { + + /** + * The number of shuffle partitions to use for each output file. By default, this file rewriter + * assumes each shuffle partition would become a separate output file. Attempting to generate + * large output files of 512 MB or higher may strain the memory resources of the cluster as such + * rewrites would require lots of Spark memory. This parameter can be used to further divide up + * the data which will end up in a single file. For example, if the target file size is 2 GB, but + * the cluster can only handle shuffles of 512 MB, this parameter could be set to 4. Iceberg will + * use a custom coalesce operation to stitch these sorted partitions back together into a single + * sorted file. + * + *

    Note using this parameter requires enabling Iceberg Spark session extensions. + */ + public static final String SHUFFLE_PARTITIONS_PER_FILE = "shuffle-partitions-per-file"; + + public static final int SHUFFLE_PARTITIONS_PER_FILE_DEFAULT = 1; + + private int numShufflePartitionsPerFile; + + protected SparkShufflingFileRewriteRunner(SparkSession spark, Table table) { + super(spark, table); + } + + protected abstract org.apache.iceberg.SortOrder sortOrder(); + + /** + * Retrieves and returns the schema for the rewrite using the current table schema. + * + *

    The schema with all columns required for correctly sorting the table. This may include + * additional computed columns which are not written to the table but are used for sorting. + */ + protected Schema sortSchema() { + return table().schema(); + } + + protected abstract Dataset sortedDF( + Dataset df, Function, Dataset> sortFunc); + + @Override + public Set validOptions() { + return ImmutableSet.builder() + .addAll(super.validOptions()) + .add(SHUFFLE_PARTITIONS_PER_FILE) + .build(); + } + + @Override + public void init(Map options) { + super.init(options); + this.numShufflePartitionsPerFile = numShufflePartitionsPerFile(options); + } + + @Override + public void doRewrite(String groupId, RewriteFileGroup fileGroup) { + Dataset scanDF = + spark() + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, groupId) + .load(groupId); + + Dataset sortedDF = + sortedDF( + scanDF, + sortFunction( + fileGroup.fileScanTasks(), + spec(fileGroup.outputSpecId()), + fileGroup.expectedOutputFiles())); + + sortedDF + .write() + .format("iceberg") + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, groupId) + .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, fileGroup.maxOutputFileSize()) + .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") + .option(SparkWriteOptions.OUTPUT_SPEC_ID, fileGroup.outputSpecId()) + .mode("append") + .save(groupId); + } + + private Function, Dataset> sortFunction( + List group, PartitionSpec outputSpec, int expectedOutputFiles) { + SortOrder[] ordering = Spark3Util.toOrdering(outputSortOrder(group, outputSpec)); + int numShufflePartitions = Math.max(1, expectedOutputFiles * numShufflePartitionsPerFile); + return df -> transformPlan(df, plan -> sortPlan(plan, ordering, numShufflePartitions)); + } + + private LogicalPlan sortPlan(LogicalPlan plan, SortOrder[] ordering, int numShufflePartitions) { + SparkFunctionCatalog catalog = SparkFunctionCatalog.get(); + OrderedWrite write = new OrderedWrite(ordering, numShufflePartitions); + LogicalPlan sortPlan = + DistributionAndOrderingUtils$.MODULE$.prepareQuery(write, plan, Option.apply(catalog)); + + if (numShufflePartitionsPerFile == 1) { + return sortPlan; + } else { + OrderAwareCoalescer coalescer = new OrderAwareCoalescer(numShufflePartitionsPerFile); + int numOutputPartitions = numShufflePartitions / numShufflePartitionsPerFile; + return new OrderAwareCoalesce(numOutputPartitions, coalescer, sortPlan); + } + } + + private Dataset transformPlan(Dataset df, Function func) { + Preconditions.checkArgument( + spark() instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark().getClass().getName()); + + Preconditions.checkArgument( + df instanceof org.apache.spark.sql.classic.Dataset, + "df is supposed to be org.apache.spark.sql.classic.Dataset, but got: %s", + df.getClass().getName()); + + return new org.apache.spark.sql.classic.Dataset<>( + ((org.apache.spark.sql.classic.SparkSession) spark()), + func.apply(((org.apache.spark.sql.classic.Dataset) df).logicalPlan()), + df.encoder()); + } + + private org.apache.iceberg.SortOrder outputSortOrder( + List group, PartitionSpec outputSpec) { + boolean requiresRepartitioning = !group.get(0).spec().equals(outputSpec); + if (requiresRepartitioning) { + // build in the requirement for partition sorting into our sort order + // as the original spec for this group does not match the output spec + return SortOrderUtil.buildSortOrder(sortSchema(), outputSpec, sortOrder()); + } else { + return sortOrder(); + } + } + + private int numShufflePartitionsPerFile(Map options) { + int value = + PropertyUtil.propertyAsInt( + options, SHUFFLE_PARTITIONS_PER_FILE, SHUFFLE_PARTITIONS_PER_FILE_DEFAULT); + Preconditions.checkArgument( + value > 0, "'%s' is set to %s but must be > 0", SHUFFLE_PARTITIONS_PER_FILE, value); + Preconditions.checkArgument( + value == 1 || Spark3Util.extensionsEnabled(spark()), + "Using '%s' requires enabling Iceberg Spark session extensions", + SHUFFLE_PARTITIONS_PER_FILE); + return value; + } + + private static class OrderedWrite implements RequiresDistributionAndOrdering { + private final OrderedDistribution distribution; + private final SortOrder[] ordering; + private final int numShufflePartitions; + + OrderedWrite(SortOrder[] ordering, int numShufflePartitions) { + this.distribution = Distributions.ordered(ordering); + this.ordering = ordering; + this.numShufflePartitions = numShufflePartitions; + } + + @Override + public Distribution requiredDistribution() { + return distribution; + } + + @Override + public boolean distributionStrictlyRequired() { + return true; + } + + @Override + public int requiredNumPartitions() { + return numShufflePartitions; + } + + @Override + public SortOrder[] requiredOrdering() { + return ordering; + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortFileRewriteRunner.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortFileRewriteRunner.java new file mode 100644 index 000000000000..9ec36c9463f5 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkSortFileRewriteRunner.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.actions; + +import java.util.function.Function; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; + +class SparkSortFileRewriteRunner extends SparkShufflingFileRewriteRunner { + + private final SortOrder sortOrder; + + SparkSortFileRewriteRunner(SparkSession spark, Table table) { + super(spark, table); + Preconditions.checkArgument( + table.sortOrder().isSorted(), + "Cannot sort data without a valid sort order, table '%s' is unsorted and no sort order is provided", + table.name()); + this.sortOrder = table.sortOrder(); + } + + SparkSortFileRewriteRunner(SparkSession spark, Table table, SortOrder sortOrder) { + super(spark, table); + Preconditions.checkArgument( + sortOrder != null && sortOrder.isSorted(), + "Cannot sort data without a valid sort order, the provided sort order is null or empty"); + this.sortOrder = sortOrder; + } + + @Override + public String description() { + return "SORT"; + } + + @Override + protected SortOrder sortOrder() { + return sortOrder; + } + + @Override + protected Dataset sortedDF(Dataset df, Function, Dataset> sortFunc) { + return sortFunc.apply(df); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderFileRewriteRunner.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderFileRewriteRunner.java new file mode 100644 index 000000000000..9a6811d1a46e --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderFileRewriteRunner.java @@ -0,0 +1,204 @@ +/* + * 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.actions; + +import static org.apache.spark.sql.functions.array; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +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.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkUtil; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.ZOrderByteUtils; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class SparkZOrderFileRewriteRunner extends SparkShufflingFileRewriteRunner { + private static final Logger LOG = LoggerFactory.getLogger(SparkZOrderFileRewriteRunner.class); + + private static final String Z_COLUMN = "ICEZVALUE"; + private static final Schema Z_SCHEMA = + new Schema(Types.NestedField.required(0, Z_COLUMN, Types.BinaryType.get())); + private static final SortOrder Z_SORT_ORDER = + SortOrder.builderFor(Z_SCHEMA) + .sortBy(Z_COLUMN, SortDirection.ASC, NullOrder.NULLS_LAST) + .build(); + + /** + * Controls the amount of bytes interleaved in the ZOrder algorithm. Default is all bytes being + * interleaved. + */ + public static final String MAX_OUTPUT_SIZE = "max-output-size"; + + public static final int MAX_OUTPUT_SIZE_DEFAULT = Integer.MAX_VALUE; + + /** + * Controls the number of bytes considered from an input column of a type with variable length + * (String, Binary). + * + *

    Default is to use the same size as primitives {@link ZOrderByteUtils#PRIMITIVE_BUFFER_SIZE}. + */ + public static final String VAR_LENGTH_CONTRIBUTION = "var-length-contribution"; + + public static final int VAR_LENGTH_CONTRIBUTION_DEFAULT = ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE; + + private final List zOrderColNames; + private int maxOutputSize; + private int varLengthContribution; + + SparkZOrderFileRewriteRunner(SparkSession spark, Table table, List zOrderColNames) { + super(spark, table); + this.zOrderColNames = validZOrderColNames(spark, table, zOrderColNames); + } + + @Override + public String description() { + return "Z-ORDER"; + } + + @Override + public Set validOptions() { + return ImmutableSet.builder() + .addAll(super.validOptions()) + .add(MAX_OUTPUT_SIZE) + .add(VAR_LENGTH_CONTRIBUTION) + .build(); + } + + @Override + public void init(Map options) { + super.init(options); + this.maxOutputSize = maxOutputSize(options); + this.varLengthContribution = varLengthContribution(options); + } + + @Override + protected SortOrder sortOrder() { + return Z_SORT_ORDER; + } + + /** + * Overrides the sortSchema method to include columns from Z_SCHEMA. + * + *

    This method generates a new Schema object which consists of columns from the original table + * schema and Z_SCHEMA. + */ + @Override + protected Schema sortSchema() { + return new Schema( + new ImmutableList.Builder() + .addAll(table().schema().columns()) + .addAll(Z_SCHEMA.columns()) + .build()); + } + + @Override + protected Dataset sortedDF(Dataset df, Function, Dataset> sortFunc) { + Dataset zValueDF = df.withColumn(Z_COLUMN, zValue(df)); + Dataset sortedDF = sortFunc.apply(zValueDF); + return sortedDF.drop(Z_COLUMN); + } + + private Column zValue(Dataset df) { + SparkZOrderUDF zOrderUDF = + new SparkZOrderUDF(zOrderColNames.size(), varLengthContribution, maxOutputSize); + + Column[] zOrderCols = + zOrderColNames.stream() + .map(df.schema()::apply) + .map(col -> zOrderUDF.sortedLexicographically(df.col(col.name()), col.dataType())) + .toArray(Column[]::new); + + return zOrderUDF.interleaveBytes(array(zOrderCols)); + } + + private int varLengthContribution(Map options) { + int value = + PropertyUtil.propertyAsInt( + options, VAR_LENGTH_CONTRIBUTION, VAR_LENGTH_CONTRIBUTION_DEFAULT); + Preconditions.checkArgument( + value > 0, + "Cannot use less than 1 byte for variable length types with ZOrder, '%s' was set to %s", + VAR_LENGTH_CONTRIBUTION, + value); + return value; + } + + private int maxOutputSize(Map options) { + int value = PropertyUtil.propertyAsInt(options, MAX_OUTPUT_SIZE, MAX_OUTPUT_SIZE_DEFAULT); + Preconditions.checkArgument( + value > 0, + "Cannot have the interleaved ZOrder value use less than 1 byte, '%s' was set to %s", + MAX_OUTPUT_SIZE, + value); + return value; + } + + private List validZOrderColNames( + SparkSession spark, Table table, List inputZOrderColNames) { + + Preconditions.checkArgument( + inputZOrderColNames != null && !inputZOrderColNames.isEmpty(), + "Cannot ZOrder when no columns are specified"); + + Schema schema = table.schema(); + Set identityPartitionFieldIds = table.spec().identitySourceIds(); + boolean caseSensitive = SparkUtil.caseSensitive(spark); + + List validZOrderColNames = Lists.newArrayList(); + + for (String colName : inputZOrderColNames) { + Types.NestedField field = + caseSensitive ? schema.findField(colName) : schema.caseInsensitiveFindField(colName); + Preconditions.checkArgument( + field != null, + "Cannot find column '%s' in table schema (case sensitive = %s): %s", + colName, + caseSensitive, + schema.asStruct()); + + if (identityPartitionFieldIds.contains(field.fieldId())) { + LOG.warn("Ignoring '{}' as such values are constant within a partition", colName); + } else { + validZOrderColNames.add(colName); + } + } + + Preconditions.checkArgument( + !validZOrderColNames.isEmpty(), + "Cannot ZOrder, all columns provided were identity partition columns and cannot be used"); + + return validZOrderColNames; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderUDF.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderUDF.java new file mode 100644 index 000000000000..d142e3fd1aee --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkZOrderUDF.java @@ -0,0 +1,325 @@ +/* + * 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.actions; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.nio.charset.CharsetEncoder; +import java.nio.charset.StandardCharsets; +import org.apache.iceberg.util.ZOrderByteUtils; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.expressions.UserDefinedFunction; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.types.BinaryType; +import org.apache.spark.sql.types.BooleanType; +import org.apache.spark.sql.types.ByteType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.DateType; +import org.apache.spark.sql.types.DoubleType; +import org.apache.spark.sql.types.FloatType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.types.LongType; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.TimestampType; +import scala.collection.JavaConverters; +import scala.collection.Seq; + +class SparkZOrderUDF implements Serializable { + private static final byte[] PRIMITIVE_EMPTY = new byte[ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE]; + + /** + * Every Spark task runs iteratively on a rows in a single thread so ThreadLocal should protect + * from concurrent access to any of these structures. + */ + private transient ThreadLocal outputBuffer; + + private transient ThreadLocal inputHolder; + private transient ThreadLocal inputBuffers; + private transient ThreadLocal encoder; + + private final int numCols; + + private int inputCol = 0; + private int totalOutputBytes = 0; + private final int varTypeSize; + private final int maxOutputSize; + + SparkZOrderUDF(int numCols, int varTypeSize, int maxOutputSize) { + this.numCols = numCols; + this.varTypeSize = varTypeSize; + this.maxOutputSize = maxOutputSize; + } + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + inputBuffers = ThreadLocal.withInitial(() -> new ByteBuffer[numCols]); + inputHolder = ThreadLocal.withInitial(() -> new byte[numCols][]); + outputBuffer = ThreadLocal.withInitial(() -> ByteBuffer.allocate(totalOutputBytes)); + encoder = ThreadLocal.withInitial(() -> StandardCharsets.UTF_8.newEncoder()); + } + + private ByteBuffer inputBuffer(int position, int size) { + ByteBuffer buffer = inputBuffers.get()[position]; + if (buffer == null) { + buffer = ByteBuffer.allocate(size); + inputBuffers.get()[position] = buffer; + } + return buffer; + } + + byte[] interleaveBits(Seq scalaBinary) { + byte[][] columnsBinary = JavaConverters.seqAsJavaList(scalaBinary).toArray(inputHolder.get()); + return ZOrderByteUtils.interleaveBits(columnsBinary, totalOutputBytes, outputBuffer.get()); + } + + private UserDefinedFunction tinyToOrderedBytesUDF() { + int position = inputCol; + UserDefinedFunction udf = + functions + .udf( + (Byte value) -> { + if (value == null) { + return PRIMITIVE_EMPTY; + } + return ZOrderByteUtils.tinyintToOrderedBytes( + value, inputBuffer(position, ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE)) + .array(); + }, + DataTypes.BinaryType) + .withName("TINY_ORDERED_BYTES"); + + this.inputCol++; + increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE); + + return udf; + } + + private UserDefinedFunction shortToOrderedBytesUDF() { + int position = inputCol; + UserDefinedFunction udf = + functions + .udf( + (Short value) -> { + if (value == null) { + return PRIMITIVE_EMPTY; + } + return ZOrderByteUtils.shortToOrderedBytes( + value, inputBuffer(position, ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE)) + .array(); + }, + DataTypes.BinaryType) + .withName("SHORT_ORDERED_BYTES"); + + this.inputCol++; + increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE); + + return udf; + } + + private UserDefinedFunction intToOrderedBytesUDF() { + int position = inputCol; + UserDefinedFunction udf = + functions + .udf( + (Integer value) -> { + if (value == null) { + return PRIMITIVE_EMPTY; + } + return ZOrderByteUtils.intToOrderedBytes( + value, inputBuffer(position, ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE)) + .array(); + }, + DataTypes.BinaryType) + .withName("INT_ORDERED_BYTES"); + + this.inputCol++; + increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE); + + return udf; + } + + private UserDefinedFunction longToOrderedBytesUDF() { + int position = inputCol; + UserDefinedFunction udf = + functions + .udf( + (Long value) -> { + if (value == null) { + return PRIMITIVE_EMPTY; + } + return ZOrderByteUtils.longToOrderedBytes( + value, inputBuffer(position, ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE)) + .array(); + }, + DataTypes.BinaryType) + .withName("LONG_ORDERED_BYTES"); + + this.inputCol++; + increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE); + + return udf; + } + + private UserDefinedFunction floatToOrderedBytesUDF() { + int position = inputCol; + UserDefinedFunction udf = + functions + .udf( + (Float value) -> { + if (value == null) { + return PRIMITIVE_EMPTY; + } + return ZOrderByteUtils.floatToOrderedBytes( + value, inputBuffer(position, ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE)) + .array(); + }, + DataTypes.BinaryType) + .withName("FLOAT_ORDERED_BYTES"); + + this.inputCol++; + increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE); + + return udf; + } + + private UserDefinedFunction doubleToOrderedBytesUDF() { + int position = inputCol; + UserDefinedFunction udf = + functions + .udf( + (Double value) -> { + if (value == null) { + return PRIMITIVE_EMPTY; + } + return ZOrderByteUtils.doubleToOrderedBytes( + value, inputBuffer(position, ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE)) + .array(); + }, + DataTypes.BinaryType) + .withName("DOUBLE_ORDERED_BYTES"); + + this.inputCol++; + increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE); + + return udf; + } + + private UserDefinedFunction booleanToOrderedBytesUDF() { + int position = inputCol; + UserDefinedFunction udf = + functions + .udf( + (Boolean value) -> { + ByteBuffer buffer = inputBuffer(position, ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE); + buffer.put(0, (byte) (value ? -127 : 0)); + return buffer.array(); + }, + DataTypes.BinaryType) + .withName("BOOLEAN-LEXICAL-BYTES"); + + this.inputCol++; + increaseOutputSize(ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE); + return udf; + } + + private UserDefinedFunction stringToOrderedBytesUDF() { + int position = inputCol; + UserDefinedFunction udf = + functions + .udf( + (String value) -> + ZOrderByteUtils.stringToOrderedBytes( + value, varTypeSize, inputBuffer(position, varTypeSize), encoder.get()) + .array(), + DataTypes.BinaryType) + .withName("STRING-LEXICAL-BYTES"); + + this.inputCol++; + increaseOutputSize(varTypeSize); + + return udf; + } + + private UserDefinedFunction bytesTruncateUDF() { + int position = inputCol; + UserDefinedFunction udf = + functions + .udf( + (byte[] value) -> + ZOrderByteUtils.byteTruncateOrFill( + value, varTypeSize, inputBuffer(position, varTypeSize)) + .array(), + DataTypes.BinaryType) + .withName("BYTE-TRUNCATE"); + + this.inputCol++; + increaseOutputSize(varTypeSize); + + return udf; + } + + private final UserDefinedFunction interleaveUDF = + functions + .udf((Seq arrayBinary) -> interleaveBits(arrayBinary), DataTypes.BinaryType) + .withName("INTERLEAVE_BYTES"); + + Column interleaveBytes(Column arrayBinary) { + return interleaveUDF.apply(arrayBinary); + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + Column sortedLexicographically(Column column, DataType type) { + if (type instanceof ByteType) { + return tinyToOrderedBytesUDF().apply(column); + } else if (type instanceof ShortType) { + return shortToOrderedBytesUDF().apply(column); + } else if (type instanceof IntegerType) { + return intToOrderedBytesUDF().apply(column); + } else if (type instanceof LongType) { + return longToOrderedBytesUDF().apply(column); + } else if (type instanceof FloatType) { + return floatToOrderedBytesUDF().apply(column); + } else if (type instanceof DoubleType) { + return doubleToOrderedBytesUDF().apply(column); + } else if (type instanceof StringType) { + return stringToOrderedBytesUDF().apply(column); + } else if (type instanceof BinaryType) { + return bytesTruncateUDF().apply(column); + } else if (type instanceof BooleanType) { + return booleanToOrderedBytesUDF().apply(column); + } else if (type instanceof TimestampType) { + return longToOrderedBytesUDF().apply(column.cast(DataTypes.LongType)); + } else if (type instanceof DateType) { + return longToOrderedBytesUDF().apply(functions.unix_date(column).cast(DataTypes.LongType)); + } else { + throw new IllegalArgumentException( + String.format( + "Cannot use column %s of type %s in ZOrdering, the type is unsupported", + column, type)); + } + } + + private void increaseOutputSize(int bytes) { + totalOutputBytes = Math.min(totalOutputBytes + bytes, maxOutputSize); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/AvroWithSparkSchemaVisitor.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/AvroWithSparkSchemaVisitor.java new file mode 100644 index 000000000000..4e8edb7e11ca --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/AvroWithSparkSchemaVisitor.java @@ -0,0 +1,81 @@ +/* + * 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.data; + +import org.apache.iceberg.avro.AvroWithPartnerByStructureVisitor; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.VariantType; + +public abstract class AvroWithSparkSchemaVisitor + extends AvroWithPartnerByStructureVisitor { + @Override + protected boolean isVariantType(DataType type) { + return type instanceof VariantType; + } + + @Override + protected boolean isStringType(DataType dataType) { + return dataType instanceof StringType; + } + + @Override + protected boolean isMapType(DataType dataType) { + return dataType instanceof MapType; + } + + @Override + protected DataType arrayElementType(DataType arrayType) { + Preconditions.checkArgument( + arrayType instanceof ArrayType, "Invalid array: %s is not an array", arrayType); + return ((ArrayType) arrayType).elementType(); + } + + @Override + protected DataType mapKeyType(DataType mapType) { + Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); + return ((MapType) mapType).keyType(); + } + + @Override + protected DataType mapValueType(DataType mapType) { + Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); + return ((MapType) mapType).valueType(); + } + + @Override + protected Pair fieldNameAndType(DataType structType, int pos) { + Preconditions.checkArgument( + structType instanceof StructType, "Invalid struct: %s is not a struct", structType); + StructField field = ((StructType) structType).apply(pos); + return Pair.of(field.name(), field.dataType()); + } + + @Override + protected DataType nullType() { + return DataTypes.NullType; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java new file mode 100644 index 000000000000..54a658bfad8e --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/ParquetWithSparkSchemaVisitor.java @@ -0,0 +1,248 @@ +/* + * 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.data; + +import java.util.Deque; +import java.util.List; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.variants.Variant; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Type.Repetition; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.VariantType; + +/** + * Visitor for traversing a Parquet type with a companion Spark type. + * + * @param the Java class returned by the visitor + */ +public class ParquetWithSparkSchemaVisitor { + private final Deque fieldNames = Lists.newLinkedList(); + + public static T visit(DataType sType, Type type, ParquetWithSparkSchemaVisitor visitor) { + Preconditions.checkArgument(sType != null, "Invalid DataType: null"); + if (type instanceof MessageType) { + Preconditions.checkArgument( + sType instanceof StructType, "Invalid struct: %s is not a struct", sType); + StructType struct = (StructType) sType; + return visitor.message( + struct, (MessageType) type, visitFields(struct, type.asGroupType(), visitor)); + + } else if (type.isPrimitive()) { + return visitor.primitive(sType, type.asPrimitiveType()); + + } else { + // if not a primitive, the typeId must be a group + GroupType group = type.asGroupType(); + LogicalTypeAnnotation annotation = group.getLogicalTypeAnnotation(); + if (LogicalTypeAnnotation.listType().equals(annotation)) { + Preconditions.checkArgument( + !group.isRepetition(Repetition.REPEATED), + "Invalid list: top-level group is repeated: %s", + group); + Preconditions.checkArgument( + group.getFieldCount() == 1, + "Invalid list: does not contain single repeated field: %s", + group); + + GroupType repeatedElement = group.getFields().get(0).asGroupType(); + Preconditions.checkArgument( + repeatedElement.isRepetition(Repetition.REPEATED), + "Invalid list: inner group is not repeated"); + Preconditions.checkArgument( + repeatedElement.getFieldCount() <= 1, + "Invalid list: repeated group is not a single field: %s", + group); + + Preconditions.checkArgument( + sType instanceof ArrayType, "Invalid list: %s is not an array", sType); + ArrayType array = (ArrayType) sType; + StructField element = + new StructField("element", array.elementType(), array.containsNull(), Metadata.empty()); + + visitor.fieldNames.push(repeatedElement.getName()); + try { + T elementResult = null; + if (repeatedElement.getFieldCount() > 0) { + elementResult = visitField(element, repeatedElement.getType(0), visitor); + } + + return visitor.list(array, group, elementResult); + + } finally { + visitor.fieldNames.pop(); + } + } else if (LogicalTypeAnnotation.mapType().equals(annotation)) { + Preconditions.checkArgument( + !group.isRepetition(Repetition.REPEATED), + "Invalid map: top-level group is repeated: %s", + group); + Preconditions.checkArgument( + group.getFieldCount() == 1, + "Invalid map: does not contain single repeated field: %s", + group); + + GroupType repeatedKeyValue = group.getType(0).asGroupType(); + Preconditions.checkArgument( + repeatedKeyValue.isRepetition(Repetition.REPEATED), + "Invalid map: inner group is not repeated"); + Preconditions.checkArgument( + repeatedKeyValue.getFieldCount() <= 2, + "Invalid map: repeated group does not have 2 fields"); + + Preconditions.checkArgument( + sType instanceof MapType, "Invalid map: %s is not a map", sType); + MapType map = (MapType) sType; + StructField keyField = new StructField("key", map.keyType(), false, Metadata.empty()); + StructField valueField = + new StructField("value", map.valueType(), map.valueContainsNull(), Metadata.empty()); + + visitor.fieldNames.push(repeatedKeyValue.getName()); + try { + T keyResult = null; + T valueResult = null; + switch (repeatedKeyValue.getFieldCount()) { + case 2: + // if there are 2 fields, both key and value are projected + keyResult = visitField(keyField, repeatedKeyValue.getType(0), visitor); + valueResult = visitField(valueField, repeatedKeyValue.getType(1), visitor); + break; + case 1: + // if there is just one, use the name to determine what it is + Type keyOrValue = repeatedKeyValue.getType(0); + if (keyOrValue.getName().equalsIgnoreCase("key")) { + keyResult = visitField(keyField, keyOrValue, visitor); + // value result remains null + } else { + valueResult = visitField(valueField, keyOrValue, visitor); + // key result remains null + } + break; + default: + // both results will remain null + } + + return visitor.map(map, group, keyResult, valueResult); + + } finally { + visitor.fieldNames.pop(); + } + } else if (LogicalTypeAnnotation.variantType(Variant.VARIANT_SPEC_VERSION).equals(annotation) + || sType instanceof VariantType) { + // For the Variant we both check the Parquet LogicalTypeAnnotation, and we rely on the + // Iceberg schema, since there are engines like Spark that produce VariantTypes without the + // annotation. + Preconditions.checkArgument( + sType instanceof VariantType, + "Invalid variant: Spark type %s is not a variant type", + sType); + VariantType variant = (VariantType) sType; + + return visitor.variant(variant, group); + } + + Preconditions.checkArgument( + sType instanceof StructType, "Invalid struct: %s is not a struct", sType); + StructType struct = (StructType) sType; + return visitor.struct(struct, group, visitFields(struct, group, visitor)); + } + } + + private static T visitField( + StructField sField, Type field, ParquetWithSparkSchemaVisitor visitor) { + visitor.fieldNames.push(field.getName()); + try { + return visit(sField.dataType(), field, visitor); + } finally { + visitor.fieldNames.pop(); + } + } + + private static List visitFields( + StructType struct, GroupType group, ParquetWithSparkSchemaVisitor visitor) { + List results = Lists.newArrayListWithExpectedSize(group.getFieldCount()); + + int fieldIndex = 0; + for (StructField sField : struct.fields()) { + if (sField.dataType() != DataTypes.NullType) { + Type field = group.getFields().get(fieldIndex); + Preconditions.checkArgument( + field.getName().equals(AvroSchemaUtil.makeCompatibleName(sField.name())), + "Structs do not match: field %s != %s", + field.getName(), + sField.name()); + results.add(visitField(sField, field, visitor)); + + fieldIndex += 1; + } + } + + // All the group fields should have been visited + Preconditions.checkArgument( + fieldIndex == group.getFieldCount(), "Structs do not match: %s and %s", struct, group); + + return results; + } + + public T message(StructType sStruct, MessageType message, List fields) { + return null; + } + + public T struct(StructType sStruct, GroupType struct, List fields) { + return null; + } + + public T list(ArrayType sArray, GroupType array, T element) { + return null; + } + + public T map(MapType sMap, GroupType map, T key, T value) { + return null; + } + + public T primitive(DataType sPrimitive, PrimitiveType primitive) { + return null; + } + + public T variant(VariantType sVariant, GroupType variant) { + throw new UnsupportedOperationException("Not implemented for variant"); + } + + protected String[] currentPath() { + return Lists.newArrayList(fieldNames.descendingIterator()).toArray(new String[0]); + } + + protected String[] path(String name) { + List list = Lists.newArrayList(fieldNames.descendingIterator()); + list.add(name); + return list.toArray(new String[0]); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroWriter.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroWriter.java new file mode 100644 index 000000000000..4946ac103188 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkAvroWriter.java @@ -0,0 +1,170 @@ +/* + * 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.data; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.Encoder; +import org.apache.iceberg.FieldMetrics; +import org.apache.iceberg.avro.MetricsAwareDatumWriter; +import org.apache.iceberg.avro.ValueWriter; +import org.apache.iceberg.avro.ValueWriters; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.ByteType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StructType; + +public class SparkAvroWriter implements MetricsAwareDatumWriter { + private final StructType dsSchema; + private ValueWriter writer = null; + + public SparkAvroWriter(StructType dsSchema) { + this.dsSchema = dsSchema; + } + + @Override + @SuppressWarnings("unchecked") + public void setSchema(Schema schema) { + this.writer = + (ValueWriter) + AvroWithSparkSchemaVisitor.visit(dsSchema, schema, new WriteBuilder()); + } + + @Override + public void write(InternalRow datum, Encoder out) throws IOException { + writer.write(datum, out); + } + + @Override + public Stream metrics() { + return writer.metrics(); + } + + private static class WriteBuilder extends AvroWithSparkSchemaVisitor> { + @Override + public ValueWriter record( + DataType struct, Schema record, List names, List> fields) { + return SparkValueWriters.struct( + fields, + IntStream.range(0, names.size()) + .mapToObj(i -> fieldNameAndType(struct, i).second()) + .collect(Collectors.toList())); + } + + @Override + public ValueWriter union(DataType type, Schema union, List> options) { + Preconditions.checkArgument( + options.contains(ValueWriters.nulls()), + "Cannot create writer for non-option union: %s", + union); + Preconditions.checkArgument( + options.size() == 2, "Cannot create writer for non-option union: %s", union); + if (union.getTypes().get(0).getType() == Schema.Type.NULL) { + return ValueWriters.option(0, options.get(1)); + } else { + return ValueWriters.option(1, options.get(0)); + } + } + + @Override + public ValueWriter array(DataType sArray, Schema array, ValueWriter elementWriter) { + return SparkValueWriters.array(elementWriter, arrayElementType(sArray)); + } + + @Override + public ValueWriter map(DataType sMap, Schema map, ValueWriter valueReader) { + return SparkValueWriters.map( + SparkValueWriters.strings(), mapKeyType(sMap), valueReader, mapValueType(sMap)); + } + + @Override + public ValueWriter map( + DataType sMap, Schema map, ValueWriter keyWriter, ValueWriter valueWriter) { + return SparkValueWriters.arrayMap( + keyWriter, mapKeyType(sMap), valueWriter, mapValueType(sMap)); + } + + @Override + public ValueWriter variant(DataType partner, ValueWriter metadata, ValueWriter value) { + return SparkValueWriters.variants(); + } + + @Override + public ValueWriter primitive(DataType type, Schema primitive) { + LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + // Spark uses the same representation + return ValueWriters.ints(); + + case "timestamp-micros": + // Spark uses the same representation + return ValueWriters.longs(); + + case "decimal": + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; + return SparkValueWriters.decimal(decimal.getPrecision(), decimal.getScale()); + + case "uuid": + return SparkValueWriters.uuids(); + + default: + throw new IllegalArgumentException("Unsupported logical type: " + logicalType); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueWriters.nulls(); + case BOOLEAN: + return ValueWriters.booleans(); + case INT: + if (type instanceof ByteType) { + return ValueWriters.tinyints(); + } else if (type instanceof ShortType) { + return ValueWriters.shorts(); + } + return ValueWriters.ints(); + case LONG: + return ValueWriters.longs(); + case FLOAT: + return ValueWriters.floats(); + case DOUBLE: + return ValueWriters.doubles(); + case STRING: + return SparkValueWriters.strings(); + case FIXED: + return ValueWriters.fixed(primitive.getFixedSize()); + case BYTES: + return ValueWriters.bytes(); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java new file mode 100644 index 000000000000..c20be44f6735 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcReader.java @@ -0,0 +1,135 @@ +/* + * 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.data; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.orc.OrcRowReader; +import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor; +import org.apache.iceberg.orc.OrcValueReader; +import org.apache.iceberg.orc.OrcValueReaders; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.orc.TypeDescription; +import org.apache.orc.storage.ql.exec.vector.StructColumnVector; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; +import org.apache.spark.sql.catalyst.InternalRow; + +/** + * Converts the OrcIterator, which returns ORC's VectorizedRowBatch to a set of Spark's UnsafeRows. + * + *

    It minimizes allocations by reusing most of the objects in the implementation. + */ +public class SparkOrcReader implements OrcRowReader { + private final OrcValueReader reader; + + public SparkOrcReader(org.apache.iceberg.Schema expectedSchema, TypeDescription readSchema) { + this(expectedSchema, readSchema, ImmutableMap.of()); + } + + @SuppressWarnings("unchecked") + public SparkOrcReader( + org.apache.iceberg.Schema expectedSchema, + TypeDescription readOrcSchema, + Map idToConstant) { + this.reader = + OrcSchemaWithTypeVisitor.visit( + expectedSchema, readOrcSchema, new ReadBuilder(idToConstant)); + } + + @Override + public InternalRow read(VectorizedRowBatch batch, int row) { + return (InternalRow) reader.read(new StructColumnVector(batch.size, batch.cols), row); + } + + @Override + public void setBatchContext(long batchOffsetInFile) { + reader.setBatchContext(batchOffsetInFile); + } + + private static class ReadBuilder extends OrcSchemaWithTypeVisitor> { + private final Map idToConstant; + + private ReadBuilder(Map idToConstant) { + this.idToConstant = idToConstant; + } + + @Override + public OrcValueReader record( + Types.StructType expected, + TypeDescription record, + List names, + List> fields) { + return SparkOrcValueReaders.struct(fields, expected, idToConstant); + } + + @Override + public OrcValueReader list( + Types.ListType iList, TypeDescription array, OrcValueReader elementReader) { + return SparkOrcValueReaders.array(elementReader); + } + + @Override + public OrcValueReader map( + Types.MapType iMap, + TypeDescription map, + OrcValueReader keyReader, + OrcValueReader valueReader) { + return SparkOrcValueReaders.map(keyReader, valueReader); + } + + @Override + public OrcValueReader primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) { + switch (primitive.getCategory()) { + case BOOLEAN: + return OrcValueReaders.booleans(); + case BYTE: + // Iceberg does not have a byte type. Use int + case SHORT: + // Iceberg does not have a short type. Use int + case DATE: + case INT: + return OrcValueReaders.ints(); + case LONG: + return OrcValueReaders.longs(); + case FLOAT: + return OrcValueReaders.floats(); + case DOUBLE: + return OrcValueReaders.doubles(); + case TIMESTAMP_INSTANT: + case TIMESTAMP: + return SparkOrcValueReaders.timestampTzs(); + case DECIMAL: + return SparkOrcValueReaders.decimals(primitive.getPrecision(), primitive.getScale()); + case CHAR: + case VARCHAR: + case STRING: + return SparkOrcValueReaders.utf8String(); + case BINARY: + if (Type.TypeID.UUID == iPrimitive.typeId()) { + return SparkOrcValueReaders.uuids(); + } + return OrcValueReaders.bytes(); + default: + throw new IllegalArgumentException("Unhandled type " + primitive); + } + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java new file mode 100644 index 000000000000..670537fbf872 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.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.data; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.orc.OrcValueReader; +import org.apache.iceberg.orc.OrcValueReaders; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.UUIDUtil; +import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; +import org.apache.orc.storage.ql.exec.vector.ColumnVector; +import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; +import org.apache.orc.storage.ql.exec.vector.ListColumnVector; +import org.apache.orc.storage.ql.exec.vector.MapColumnVector; +import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; +import org.apache.orc.storage.serde2.io.HiveDecimalWritable; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.GenericArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.types.UTF8String; + +public class SparkOrcValueReaders { + private SparkOrcValueReaders() {} + + public static OrcValueReader utf8String() { + return StringReader.INSTANCE; + } + + public static OrcValueReader uuids() { + return UUIDReader.INSTANCE; + } + + public static OrcValueReader timestampTzs() { + return TimestampTzReader.INSTANCE; + } + + public static OrcValueReader decimals(int precision, int scale) { + if (precision <= Decimal.MAX_LONG_DIGITS()) { + return new SparkOrcValueReaders.Decimal18Reader(precision, scale); + } else if (precision <= 38) { + return new SparkOrcValueReaders.Decimal38Reader(precision, scale); + } else { + throw new IllegalArgumentException("Invalid precision: " + precision); + } + } + + static OrcValueReader struct( + List> readers, Types.StructType struct, Map idToConstant) { + return new StructReader(readers, struct, idToConstant); + } + + static OrcValueReader array(OrcValueReader elementReader) { + return new ArrayReader(elementReader); + } + + static OrcValueReader map(OrcValueReader keyReader, OrcValueReader valueReader) { + return new MapReader(keyReader, valueReader); + } + + private static class ArrayReader implements OrcValueReader { + private final OrcValueReader elementReader; + + private ArrayReader(OrcValueReader elementReader) { + this.elementReader = elementReader; + } + + @Override + public ArrayData nonNullRead(ColumnVector vector, int row) { + ListColumnVector listVector = (ListColumnVector) vector; + int offset = (int) listVector.offsets[row]; + int length = (int) listVector.lengths[row]; + List elements = Lists.newArrayListWithExpectedSize(length); + for (int c = 0; c < length; ++c) { + elements.add(elementReader.read(listVector.child, offset + c)); + } + return new GenericArrayData(elements.toArray()); + } + + @Override + public void setBatchContext(long batchOffsetInFile) { + elementReader.setBatchContext(batchOffsetInFile); + } + } + + private static class MapReader implements OrcValueReader { + private final OrcValueReader keyReader; + private final OrcValueReader valueReader; + + private MapReader(OrcValueReader keyReader, OrcValueReader valueReader) { + this.keyReader = keyReader; + this.valueReader = valueReader; + } + + @Override + public MapData nonNullRead(ColumnVector vector, int row) { + MapColumnVector mapVector = (MapColumnVector) vector; + int offset = (int) mapVector.offsets[row]; + long length = mapVector.lengths[row]; + List keys = Lists.newArrayListWithExpectedSize((int) length); + List values = Lists.newArrayListWithExpectedSize((int) length); + for (int c = 0; c < length; c++) { + keys.add(keyReader.read(mapVector.keys, offset + c)); + values.add(valueReader.read(mapVector.values, offset + c)); + } + + return new ArrayBasedMapData( + new GenericArrayData(keys.toArray()), new GenericArrayData(values.toArray())); + } + + @Override + public void setBatchContext(long batchOffsetInFile) { + keyReader.setBatchContext(batchOffsetInFile); + valueReader.setBatchContext(batchOffsetInFile); + } + } + + static class StructReader extends OrcValueReaders.StructReader { + private final int numFields; + + protected StructReader( + List> readers, Types.StructType struct, Map idToConstant) { + super(readers, struct, idToConstant); + this.numFields = struct.fields().size(); + } + + @Override + protected InternalRow create() { + return new GenericInternalRow(numFields); + } + + @Override + protected void set(InternalRow struct, int pos, Object value) { + if (value != null) { + struct.update(pos, value); + } else { + struct.setNullAt(pos); + } + } + } + + private static class StringReader implements OrcValueReader { + private static final StringReader INSTANCE = new StringReader(); + + private StringReader() {} + + @Override + public UTF8String nonNullRead(ColumnVector vector, int row) { + BytesColumnVector bytesVector = (BytesColumnVector) vector; + return UTF8String.fromBytes( + bytesVector.vector[row], bytesVector.start[row], bytesVector.length[row]); + } + } + + private static class UUIDReader implements OrcValueReader { + private static final UUIDReader INSTANCE = new UUIDReader(); + + private UUIDReader() {} + + @Override + public UTF8String nonNullRead(ColumnVector vector, int row) { + BytesColumnVector bytesVector = (BytesColumnVector) vector; + ByteBuffer buffer = + ByteBuffer.wrap(bytesVector.vector[row], bytesVector.start[row], bytesVector.length[row]); + return UTF8String.fromString(UUIDUtil.convert(buffer).toString()); + } + } + + private static class TimestampTzReader implements OrcValueReader { + private static final TimestampTzReader INSTANCE = new TimestampTzReader(); + + private TimestampTzReader() {} + + @Override + public Long nonNullRead(ColumnVector vector, int row) { + TimestampColumnVector tcv = (TimestampColumnVector) vector; + return Math.floorDiv(tcv.time[row], 1_000) * 1_000_000 + Math.floorDiv(tcv.nanos[row], 1000); + } + } + + private static class Decimal18Reader implements OrcValueReader { + private final int precision; + private final int scale; + + Decimal18Reader(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public Decimal nonNullRead(ColumnVector vector, int row) { + HiveDecimalWritable value = ((DecimalColumnVector) vector).vector[row]; + + // The scale of decimal read from hive ORC file may be not equals to the expected scale. For + // data type + // decimal(10,3) and the value 10.100, the hive ORC writer will remove its trailing zero and + // store it + // as 101*10^(-1), its scale will adjust from 3 to 1. So here we could not assert that + // value.scale() == scale. + // we also need to convert the hive orc decimal to a decimal with expected precision and + // scale. + Preconditions.checkArgument( + value.precision() <= precision, + "Cannot read value as decimal(%s,%s), too large: %s", + precision, + scale, + value); + + return new Decimal().set(value.serialize64(scale), precision, scale); + } + } + + private static class Decimal38Reader implements OrcValueReader { + private final int precision; + private final int scale; + + Decimal38Reader(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public Decimal nonNullRead(ColumnVector vector, int row) { + BigDecimal value = + ((DecimalColumnVector) vector).vector[row].getHiveDecimal().bigDecimalValue(); + + Preconditions.checkArgument( + value.precision() <= precision, + "Cannot read value as decimal(%s,%s), too large: %s", + precision, + scale, + value); + + return new Decimal().set(new scala.math.BigDecimal(value), precision, scale); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriters.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriters.java new file mode 100644 index 000000000000..7f9810e4c60c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueWriters.java @@ -0,0 +1,221 @@ +/* + * 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.data; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.UUID; +import java.util.stream.Stream; +import org.apache.iceberg.FieldMetrics; +import org.apache.iceberg.orc.OrcValueWriter; +import org.apache.iceberg.util.UUIDUtil; +import org.apache.orc.TypeDescription; +import org.apache.orc.storage.common.type.HiveDecimal; +import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; +import org.apache.orc.storage.ql.exec.vector.ColumnVector; +import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; +import org.apache.orc.storage.ql.exec.vector.ListColumnVector; +import org.apache.orc.storage.ql.exec.vector.MapColumnVector; +import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.types.UTF8String; + +class SparkOrcValueWriters { + private SparkOrcValueWriters() {} + + static OrcValueWriter strings() { + return StringWriter.INSTANCE; + } + + static OrcValueWriter uuids() { + return UUIDWriter.INSTANCE; + } + + static OrcValueWriter timestampTz() { + return TimestampTzWriter.INSTANCE; + } + + static OrcValueWriter decimal(int precision, int scale) { + if (precision <= 18) { + return new Decimal18Writer(scale); + } else { + return new Decimal38Writer(); + } + } + + static OrcValueWriter list(OrcValueWriter element, List orcType) { + return new ListWriter<>(element, orcType); + } + + static OrcValueWriter map( + OrcValueWriter keyWriter, OrcValueWriter valueWriter, List orcTypes) { + return new MapWriter<>(keyWriter, valueWriter, orcTypes); + } + + private static class StringWriter implements OrcValueWriter { + private static final StringWriter INSTANCE = new StringWriter(); + + @Override + public void nonNullWrite(int rowId, UTF8String data, ColumnVector output) { + byte[] value = data.getBytes(); + ((BytesColumnVector) output).setRef(rowId, value, 0, value.length); + } + } + + private static class UUIDWriter implements OrcValueWriter { + private static final UUIDWriter INSTANCE = new UUIDWriter(); + + @Override + public void nonNullWrite(int rowId, UTF8String data, ColumnVector output) { + // ((BytesColumnVector) output).setRef(..) just stores a reference to the passed byte[], so + // can't use a ThreadLocal ByteBuffer here like in other places because subsequent writes + // would then overwrite previous values + ByteBuffer buffer = UUIDUtil.convertToByteBuffer(UUID.fromString(data.toString())); + ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length); + } + } + + private static class TimestampTzWriter implements OrcValueWriter { + private static final TimestampTzWriter INSTANCE = new TimestampTzWriter(); + + @Override + public void nonNullWrite(int rowId, Long micros, ColumnVector output) { + TimestampColumnVector cv = (TimestampColumnVector) output; + cv.time[rowId] = Math.floorDiv(micros, 1_000); // millis + cv.nanos[rowId] = (int) Math.floorMod(micros, 1_000_000) * 1_000; // nanos + } + } + + private static class Decimal18Writer implements OrcValueWriter { + private final int scale; + + Decimal18Writer(int scale) { + this.scale = scale; + } + + @Override + public void nonNullWrite(int rowId, Decimal decimal, ColumnVector output) { + ((DecimalColumnVector) output) + .vector[rowId].setFromLongAndScale(decimal.toUnscaledLong(), scale); + } + } + + private static class Decimal38Writer implements OrcValueWriter { + + @Override + public void nonNullWrite(int rowId, Decimal decimal, ColumnVector output) { + ((DecimalColumnVector) output) + .vector[rowId].set(HiveDecimal.create(decimal.toJavaBigDecimal())); + } + } + + private static class ListWriter implements OrcValueWriter { + private final OrcValueWriter writer; + private final SparkOrcWriter.FieldGetter fieldGetter; + + @SuppressWarnings("unchecked") + ListWriter(OrcValueWriter writer, List orcTypes) { + if (orcTypes.size() != 1) { + throw new IllegalArgumentException( + "Expected one (and same) ORC type for list elements, got: " + orcTypes); + } + this.writer = writer; + this.fieldGetter = + (SparkOrcWriter.FieldGetter) SparkOrcWriter.createFieldGetter(orcTypes.get(0)); + } + + @Override + public void nonNullWrite(int rowId, ArrayData value, ColumnVector output) { + ListColumnVector cv = (ListColumnVector) output; + // record the length and start of the list elements + cv.lengths[rowId] = value.numElements(); + cv.offsets[rowId] = cv.childCount; + cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); + // make sure the child is big enough + growColumnVector(cv.child, cv.childCount); + // Add each element + for (int e = 0; e < cv.lengths[rowId]; ++e) { + writer.write((int) (e + cv.offsets[rowId]), fieldGetter.getFieldOrNull(value, e), cv.child); + } + } + + @Override + public Stream> metrics() { + return writer.metrics(); + } + } + + private static class MapWriter implements OrcValueWriter { + private final OrcValueWriter keyWriter; + private final OrcValueWriter valueWriter; + private final SparkOrcWriter.FieldGetter keyFieldGetter; + private final SparkOrcWriter.FieldGetter valueFieldGetter; + + @SuppressWarnings("unchecked") + MapWriter( + OrcValueWriter keyWriter, + OrcValueWriter valueWriter, + List orcTypes) { + if (orcTypes.size() != 2) { + throw new IllegalArgumentException( + "Expected two ORC type descriptions for a map, got: " + orcTypes); + } + this.keyWriter = keyWriter; + this.valueWriter = valueWriter; + this.keyFieldGetter = + (SparkOrcWriter.FieldGetter) SparkOrcWriter.createFieldGetter(orcTypes.get(0)); + this.valueFieldGetter = + (SparkOrcWriter.FieldGetter) SparkOrcWriter.createFieldGetter(orcTypes.get(1)); + } + + @Override + public void nonNullWrite(int rowId, MapData map, ColumnVector output) { + ArrayData key = map.keyArray(); + ArrayData value = map.valueArray(); + MapColumnVector cv = (MapColumnVector) output; + // record the length and start of the list elements + cv.lengths[rowId] = value.numElements(); + cv.offsets[rowId] = cv.childCount; + cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); + // make sure the child is big enough + growColumnVector(cv.keys, cv.childCount); + growColumnVector(cv.values, cv.childCount); + // Add each element + for (int e = 0; e < cv.lengths[rowId]; ++e) { + int pos = (int) (e + cv.offsets[rowId]); + keyWriter.write(pos, keyFieldGetter.getFieldOrNull(key, e), cv.keys); + valueWriter.write(pos, valueFieldGetter.getFieldOrNull(value, e), cv.values); + } + } + + @Override + public Stream> metrics() { + return Stream.concat(keyWriter.metrics(), valueWriter.metrics()); + } + } + + private static void growColumnVector(ColumnVector cv, int requestedSize) { + if (cv.isNull.length < requestedSize) { + // Use growth factor of 3 to avoid frequent array allocations + cv.ensureSize(requestedSize * 3, true); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java new file mode 100644 index 000000000000..6fc8849c82b2 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java @@ -0,0 +1,247 @@ +/* + * 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.data; + +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import javax.annotation.Nullable; +import org.apache.iceberg.FieldMetrics; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.orc.GenericOrcWriters; +import org.apache.iceberg.orc.ORCSchemaUtil; +import org.apache.iceberg.orc.OrcRowWriter; +import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor; +import org.apache.iceberg.orc.OrcValueWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.orc.TypeDescription; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.SpecializedGetters; + +/** This class acts as an adaptor from an OrcFileAppender to a FileAppender<InternalRow>. */ +public class SparkOrcWriter implements OrcRowWriter { + + private final InternalRowWriter writer; + + public SparkOrcWriter(Schema iSchema, TypeDescription orcSchema) { + Preconditions.checkArgument( + orcSchema.getCategory() == TypeDescription.Category.STRUCT, + "Top level must be a struct " + orcSchema); + + writer = + (InternalRowWriter) OrcSchemaWithTypeVisitor.visit(iSchema, orcSchema, new WriteBuilder()); + } + + @Override + public void write(InternalRow value, VectorizedRowBatch output) { + Preconditions.checkArgument(value != null, "value must not be null"); + writer.writeRow(value, output); + } + + @Override + public List> writers() { + return writer.writers(); + } + + @Override + public Stream> metrics() { + return writer.metrics(); + } + + private static class WriteBuilder extends OrcSchemaWithTypeVisitor> { + private WriteBuilder() {} + + @Override + public OrcValueWriter record( + Types.StructType iStruct, + TypeDescription record, + List names, + List> fields) { + return new InternalRowWriter(fields, iStruct, record.getChildren()); + } + + @Override + public OrcValueWriter list( + Types.ListType iList, TypeDescription array, OrcValueWriter element) { + return SparkOrcValueWriters.list(element, array.getChildren()); + } + + @Override + public OrcValueWriter map( + Types.MapType iMap, TypeDescription map, OrcValueWriter key, OrcValueWriter value) { + return SparkOrcValueWriters.map(key, value, map.getChildren()); + } + + @Override + public OrcValueWriter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) { + switch (primitive.getCategory()) { + case BOOLEAN: + return GenericOrcWriters.booleans(); + case BYTE: + return GenericOrcWriters.bytes(); + case SHORT: + return GenericOrcWriters.shorts(); + case DATE: + case INT: + return GenericOrcWriters.ints(); + case LONG: + return GenericOrcWriters.longs(); + case FLOAT: + return GenericOrcWriters.floats(ORCSchemaUtil.fieldId(primitive)); + case DOUBLE: + return GenericOrcWriters.doubles(ORCSchemaUtil.fieldId(primitive)); + case BINARY: + if (Type.TypeID.UUID == iPrimitive.typeId()) { + return SparkOrcValueWriters.uuids(); + } + return GenericOrcWriters.byteArrays(); + case STRING: + case CHAR: + case VARCHAR: + return SparkOrcValueWriters.strings(); + case DECIMAL: + return SparkOrcValueWriters.decimal(primitive.getPrecision(), primitive.getScale()); + case TIMESTAMP_INSTANT: + case TIMESTAMP: + return SparkOrcValueWriters.timestampTz(); + default: + throw new IllegalArgumentException("Unhandled type " + primitive); + } + } + } + + private static class InternalRowWriter extends GenericOrcWriters.StructWriter { + private final List> fieldGetters; + + InternalRowWriter( + List> writers, Types.StructType iStruct, List orcTypes) { + super(iStruct, writers); + this.fieldGetters = Lists.newArrayListWithExpectedSize(orcTypes.size()); + + Map idToType = + orcTypes.stream().collect(Collectors.toMap(ORCSchemaUtil::fieldId, s -> s)); + + for (Types.NestedField iField : iStruct.fields()) { + fieldGetters.add(createFieldGetter(idToType.get(iField.fieldId()))); + } + } + + @Override + protected Object get(InternalRow struct, int index) { + return fieldGetters.get(index).getFieldOrNull(struct, index); + } + } + + static FieldGetter createFieldGetter(TypeDescription fieldType) { + // In the case of an UnknownType + if (fieldType == null) { + return (row, ordinal) -> null; + } + + final FieldGetter fieldGetter; + switch (fieldType.getCategory()) { + case BOOLEAN: + fieldGetter = SpecializedGetters::getBoolean; + break; + case BYTE: + fieldGetter = SpecializedGetters::getByte; + break; + case SHORT: + fieldGetter = SpecializedGetters::getShort; + break; + case DATE: + case INT: + fieldGetter = SpecializedGetters::getInt; + break; + case LONG: + case TIMESTAMP: + case TIMESTAMP_INSTANT: + fieldGetter = SpecializedGetters::getLong; + break; + case FLOAT: + fieldGetter = SpecializedGetters::getFloat; + break; + case DOUBLE: + fieldGetter = SpecializedGetters::getDouble; + break; + case BINARY: + if (ORCSchemaUtil.BinaryType.UUID + .toString() + .equalsIgnoreCase( + fieldType.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE))) { + fieldGetter = SpecializedGetters::getUTF8String; + } else { + fieldGetter = SpecializedGetters::getBinary; + } + // getBinary always makes a copy, so we don't need to worry about it + // being changed behind our back. + break; + case DECIMAL: + fieldGetter = + (row, ordinal) -> + row.getDecimal(ordinal, fieldType.getPrecision(), fieldType.getScale()); + break; + case STRING: + case CHAR: + case VARCHAR: + fieldGetter = SpecializedGetters::getUTF8String; + break; + case STRUCT: + fieldGetter = (row, ordinal) -> row.getStruct(ordinal, fieldType.getChildren().size()); + break; + case LIST: + fieldGetter = SpecializedGetters::getArray; + break; + case MAP: + fieldGetter = SpecializedGetters::getMap; + break; + default: + throw new IllegalArgumentException( + "Encountered an unsupported ORC type during a write from Spark."); + } + + return (row, ordinal) -> { + if (row.isNullAt(ordinal)) { + return null; + } + return fieldGetter.getFieldOrNull(row, ordinal); + }; + } + + interface FieldGetter extends Serializable { + + /** + * Returns a value from a complex Spark data holder such ArrayData, InternalRow, etc... Calls + * the appropriate getter for the expected data type. + * + * @param row Spark's data representation + * @param ordinal index in the data structure (e.g. column index for InterRow, list index in + * ArrayData, etc..) + * @return field value at ordinal + */ + @Nullable + T getFieldOrNull(SpecializedGetters row, int ordinal); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java new file mode 100644 index 000000000000..28a9a31c6a6e --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java @@ -0,0 +1,782 @@ +/* + * 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.data; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.parquet.ParquetSchemaUtil; +import org.apache.iceberg.parquet.ParquetValueReader; +import org.apache.iceberg.parquet.ParquetValueReaders; +import org.apache.iceberg.parquet.ParquetValueReaders.FloatAsDoubleReader; +import org.apache.iceberg.parquet.ParquetValueReaders.IntAsLongReader; +import org.apache.iceberg.parquet.ParquetValueReaders.PrimitiveReader; +import org.apache.iceberg.parquet.ParquetValueReaders.RepeatedKeyValueReader; +import org.apache.iceberg.parquet.ParquetValueReaders.RepeatedReader; +import org.apache.iceberg.parquet.ParquetValueReaders.ReusableEntry; +import org.apache.iceberg.parquet.ParquetValueReaders.StructReader; +import org.apache.iceberg.parquet.ParquetValueReaders.UnboxedReader; +import org.apache.iceberg.parquet.ParquetVariantReaders.DelegatingValueReader; +import org.apache.iceberg.parquet.ParquetVariantVisitor; +import org.apache.iceberg.parquet.TypeWithSchemaVisitor; +import org.apache.iceberg.parquet.VariantReaderBuilder; +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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkUtil; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.UUIDUtil; +import org.apache.iceberg.variants.Variant; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.GenericArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.types.CalendarInterval; +import org.apache.spark.unsafe.types.UTF8String; +import org.apache.spark.unsafe.types.VariantVal; + +public class SparkParquetReaders { + private SparkParquetReaders() {} + + public static ParquetValueReader buildReader( + Schema expectedSchema, MessageType fileSchema) { + return buildReader(expectedSchema, fileSchema, ImmutableMap.of()); + } + + @SuppressWarnings("unchecked") + public static ParquetValueReader buildReader( + Schema expectedSchema, MessageType fileSchema, Map idToConstant) { + if (ParquetSchemaUtil.hasIds(fileSchema)) { + return (ParquetValueReader) + TypeWithSchemaVisitor.visit( + expectedSchema.asStruct(), fileSchema, new ReadBuilder(fileSchema, idToConstant)); + } else { + return (ParquetValueReader) + TypeWithSchemaVisitor.visit( + expectedSchema.asStruct(), + fileSchema, + new FallbackReadBuilder(fileSchema, idToConstant)); + } + } + + private static class FallbackReadBuilder extends ReadBuilder { + FallbackReadBuilder(MessageType type, Map idToConstant) { + super(type, idToConstant); + } + + @Override + public ParquetValueReader message( + Types.StructType expected, MessageType message, List> fieldReaders) { + // the top level matches by ID, but the remaining IDs are missing + return super.struct(expected, message, fieldReaders); + } + + @Override + public ParquetValueReader struct( + Types.StructType ignored, GroupType struct, List> fieldReaders) { + // the expected struct is ignored because nested fields are never found when the + List> newFields = + Lists.newArrayListWithExpectedSize(fieldReaders.size()); + List fields = struct.getFields(); + for (int i = 0; i < fields.size(); i += 1) { + Type fieldType = fields.get(i); + int fieldD = type().getMaxDefinitionLevel(path(fieldType.getName())) - 1; + newFields.add(ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); + } + + return new InternalRowReader(newFields); + } + } + + private static class ReadBuilder extends TypeWithSchemaVisitor> { + private final MessageType type; + private final Map idToConstant; + + ReadBuilder(MessageType type, Map idToConstant) { + this.type = type; + this.idToConstant = idToConstant; + } + + @Override + public ParquetValueReader message( + Types.StructType expected, MessageType message, List> fieldReaders) { + return struct(expected, message.asGroupType(), fieldReaders); + } + + @Override + public ParquetValueReader struct( + Types.StructType expected, GroupType struct, List> fieldReaders) { + if (null == expected) { + return new InternalRowReader(ImmutableList.of()); + } + + // match the expected struct's order + Map> readersById = Maps.newHashMap(); + List fields = struct.getFields(); + for (int i = 0; i < fields.size(); i += 1) { + Type fieldType = fields.get(i); + int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1; + if (fieldType.getId() != null) { + int id = fieldType.getId().intValue(); + readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); + } + } + + int constantDefinitionLevel = type.getMaxDefinitionLevel(currentPath()); + List expectedFields = expected.fields(); + List> reorderedFields = + Lists.newArrayListWithExpectedSize(expectedFields.size()); + + for (Types.NestedField field : expectedFields) { + int id = field.fieldId(); + ParquetValueReader reader = + ParquetValueReaders.replaceWithMetadataReader( + id, readersById.get(id), idToConstant, constantDefinitionLevel); + reorderedFields.add(defaultReader(field, reader, constantDefinitionLevel)); + } + + return new InternalRowReader(reorderedFields); + } + + private ParquetValueReader defaultReader( + Types.NestedField field, ParquetValueReader reader, int constantDL) { + if (reader != null) { + return reader; + } else if (field.initialDefault() != null) { + return ParquetValueReaders.constant( + SparkUtil.internalToSpark(field.type(), field.initialDefault()), constantDL); + } else if (field.isOptional()) { + return ParquetValueReaders.nulls(); + } + + throw new IllegalArgumentException(String.format("Missing required field: %s", field.name())); + } + + @Override + public ParquetValueReader list( + Types.ListType expectedList, GroupType array, ParquetValueReader elementReader) { + String[] repeatedPath = currentPath(); + + int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1; + int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1; + + Type elementType = ParquetSchemaUtil.determineListElementType(array); + int elementD = type.getMaxDefinitionLevel(path(elementType.getName())) - 1; + + return new ArrayReader<>( + repeatedD, repeatedR, ParquetValueReaders.option(elementType, elementD, elementReader)); + } + + @Override + public ParquetValueReader map( + Types.MapType expectedMap, + GroupType map, + ParquetValueReader keyReader, + ParquetValueReader valueReader) { + GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); + String[] repeatedPath = currentPath(); + + int repeatedD = type.getMaxDefinitionLevel(repeatedPath) - 1; + int repeatedR = type.getMaxRepetitionLevel(repeatedPath) - 1; + + Type keyType = repeatedKeyValue.getType(0); + int keyD = type.getMaxDefinitionLevel(path(keyType.getName())) - 1; + Type valueType = repeatedKeyValue.getType(1); + int valueD = type.getMaxDefinitionLevel(path(valueType.getName())) - 1; + + return new MapReader<>( + repeatedD, + repeatedR, + ParquetValueReaders.option(keyType, keyD, keyReader), + ParquetValueReaders.option(valueType, valueD, valueReader)); + } + + @Override + public ParquetVariantVisitor> variantVisitor() { + return new VariantReaderBuilder(type, Arrays.asList(currentPath())); + } + + @Override + public ParquetValueReader variant( + Types.VariantType iVariant, GroupType variant, ParquetValueReader variantReader) { + return new VariantReader(variantReader); + } + + @Override + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public ParquetValueReader primitive( + org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { + ColumnDescriptor desc = type.getColumnDescription(currentPath()); + + if (primitive.getOriginalType() != null) { + switch (primitive.getOriginalType()) { + case ENUM: + case JSON: + case UTF8: + return new StringReader(desc); + case INT_8: + case INT_16: + case INT_32: + if (expected != null && expected.typeId() == Types.LongType.get().typeId()) { + return new IntAsLongReader(desc); + } else { + return new UnboxedReader<>(desc); + } + case DATE: + case INT_64: + return new UnboxedReader<>(desc); + case TIMESTAMP_MICROS: + case TIMESTAMP_MILLIS: + return ParquetValueReaders.timestamps(desc); + case DECIMAL: + DecimalLogicalTypeAnnotation decimal = + (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); + switch (primitive.getPrimitiveTypeName()) { + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return new BinaryDecimalReader(desc, decimal.getScale()); + case INT64: + return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); + case INT32: + return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); + default: + throw new UnsupportedOperationException( + "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); + } + case BSON: + return new ParquetValueReaders.ByteArrayReader(desc); + default: + throw new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getOriginalType()); + } + } + + switch (primitive.getPrimitiveTypeName()) { + case FIXED_LEN_BYTE_ARRAY: + case BINARY: + if (expected != null && expected.typeId() == TypeID.UUID) { + return new UUIDReader(desc); + } + return new ParquetValueReaders.ByteArrayReader(desc); + case INT32: + if (expected != null && expected.typeId() == TypeID.LONG) { + return new IntAsLongReader(desc); + } else { + return new UnboxedReader<>(desc); + } + case FLOAT: + if (expected != null && expected.typeId() == TypeID.DOUBLE) { + return new FloatAsDoubleReader(desc); + } else { + return new UnboxedReader<>(desc); + } + case BOOLEAN: + case INT64: + case DOUBLE: + return new UnboxedReader<>(desc); + case INT96: + // Impala & Spark used to write timestamps as INT96 without a logical type. For backwards + // compatibility we try to read INT96 as timestamps. + return ParquetValueReaders.int96Timestamps(desc); + default: + throw new UnsupportedOperationException("Unsupported type: " + primitive); + } + } + + protected MessageType type() { + return type; + } + } + + private static class BinaryDecimalReader extends PrimitiveReader { + private final int scale; + + BinaryDecimalReader(ColumnDescriptor desc, int scale) { + super(desc); + this.scale = scale; + } + + @Override + public Decimal read(Decimal ignored) { + Binary binary = column.nextBinary(); + return Decimal.fromDecimal(new BigDecimal(new BigInteger(binary.getBytes()), scale)); + } + } + + private static class IntegerDecimalReader extends PrimitiveReader { + private final int precision; + private final int scale; + + IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public Decimal read(Decimal ignored) { + return Decimal.apply(column.nextInteger(), precision, scale); + } + } + + private static class LongDecimalReader extends PrimitiveReader { + private final int precision; + private final int scale; + + LongDecimalReader(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public Decimal read(Decimal ignored) { + return Decimal.apply(column.nextLong(), precision, scale); + } + } + + private static class StringReader extends PrimitiveReader { + StringReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public UTF8String read(UTF8String ignored) { + Binary binary = column.nextBinary(); + ByteBuffer buffer = binary.toByteBuffer(); + if (buffer.hasArray()) { + return UTF8String.fromBytes( + buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining()); + } else { + return UTF8String.fromBytes(binary.getBytes()); + } + } + } + + private static class UUIDReader extends PrimitiveReader { + UUIDReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + @SuppressWarnings("ByteBufferBackingArray") + public UTF8String read(UTF8String ignored) { + return UTF8String.fromString(UUIDUtil.convert(column.nextBinary().toByteBuffer()).toString()); + } + } + + private static class ArrayReader extends RepeatedReader { + private int readPos = 0; + private int writePos = 0; + + ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader reader) { + super(definitionLevel, repetitionLevel, reader); + } + + @Override + @SuppressWarnings("unchecked") + protected ReusableArrayData newListData(ArrayData reuse) { + this.readPos = 0; + this.writePos = 0; + + if (reuse instanceof ReusableArrayData) { + return (ReusableArrayData) reuse; + } else { + return new ReusableArrayData(); + } + } + + @Override + @SuppressWarnings("unchecked") + protected E getElement(ReusableArrayData list) { + E value = null; + if (readPos < list.capacity()) { + value = (E) list.values[readPos]; + } + + readPos += 1; + + return value; + } + + @Override + protected void addElement(ReusableArrayData reused, E element) { + if (writePos >= reused.capacity()) { + reused.grow(); + } + + reused.values[writePos] = element; + + writePos += 1; + } + + @Override + protected ArrayData buildList(ReusableArrayData list) { + list.setNumElements(writePos); + return list; + } + } + + private static class MapReader + extends RepeatedKeyValueReader { + private int readPos = 0; + private int writePos = 0; + + private final ReusableEntry entry = new ReusableEntry<>(); + private final ReusableEntry nullEntry = new ReusableEntry<>(); + + MapReader( + int definitionLevel, + int repetitionLevel, + ParquetValueReader keyReader, + ParquetValueReader valueReader) { + super(definitionLevel, repetitionLevel, keyReader, valueReader); + } + + @Override + @SuppressWarnings("unchecked") + protected ReusableMapData newMapData(MapData reuse) { + this.readPos = 0; + this.writePos = 0; + + if (reuse instanceof ReusableMapData) { + return (ReusableMapData) reuse; + } else { + return new ReusableMapData(); + } + } + + @Override + @SuppressWarnings("unchecked") + protected Map.Entry getPair(ReusableMapData map) { + Map.Entry kv = nullEntry; + if (readPos < map.capacity()) { + entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]); + kv = entry; + } + + readPos += 1; + + return kv; + } + + @Override + protected void addPair(ReusableMapData map, K key, V value) { + if (writePos >= map.capacity()) { + map.grow(); + } + + map.keys.values[writePos] = key; + map.values.values[writePos] = value; + + writePos += 1; + } + + @Override + protected MapData buildMap(ReusableMapData map) { + map.setNumElements(writePos); + return map; + } + } + + /** Variant reader to convert from Variant to Spark VariantVal */ + private static class VariantReader extends DelegatingValueReader { + @SuppressWarnings("unchecked") + private VariantReader(ParquetValueReader reader) { + super((ParquetValueReader) reader); + } + + @Override + public VariantVal read(VariantVal reuse) { + Variant variant = super.readFromDelegate(null); + byte[] metadataBytes = new byte[variant.metadata().sizeInBytes()]; + ByteBuffer metadataBuffer = ByteBuffer.wrap(metadataBytes).order(ByteOrder.LITTLE_ENDIAN); + variant.metadata().writeTo(metadataBuffer, 0); + + byte[] valueBytes = new byte[variant.value().sizeInBytes()]; + ByteBuffer valueBuffer = ByteBuffer.wrap(valueBytes).order(ByteOrder.LITTLE_ENDIAN); + variant.value().writeTo(valueBuffer, 0); + + return new VariantVal(valueBytes, metadataBytes); + } + } + + private static class InternalRowReader extends StructReader { + private final int numFields; + + InternalRowReader(List> readers) { + super(readers); + this.numFields = readers.size(); + } + + @Override + protected GenericInternalRow newStructData(InternalRow reuse) { + if (reuse instanceof GenericInternalRow) { + return (GenericInternalRow) reuse; + } else { + return new GenericInternalRow(numFields); + } + } + + @Override + protected Object getField(GenericInternalRow intermediate, int pos) { + return intermediate.genericGet(pos); + } + + @Override + protected InternalRow buildStruct(GenericInternalRow struct) { + return struct; + } + + @Override + protected void set(GenericInternalRow row, int pos, Object value) { + row.update(pos, value); + } + + @Override + protected void setNull(GenericInternalRow row, int pos) { + row.setNullAt(pos); + } + + @Override + protected void setBoolean(GenericInternalRow row, int pos, boolean value) { + row.setBoolean(pos, value); + } + + @Override + protected void setInteger(GenericInternalRow row, int pos, int value) { + row.setInt(pos, value); + } + + @Override + protected void setLong(GenericInternalRow row, int pos, long value) { + row.setLong(pos, value); + } + + @Override + protected void setFloat(GenericInternalRow row, int pos, float value) { + row.setFloat(pos, value); + } + + @Override + protected void setDouble(GenericInternalRow row, int pos, double value) { + row.setDouble(pos, value); + } + } + + private static class ReusableMapData extends MapData { + private final ReusableArrayData keys; + private final ReusableArrayData values; + private int numElements; + + private ReusableMapData() { + this.keys = new ReusableArrayData(); + this.values = new ReusableArrayData(); + } + + private void grow() { + keys.grow(); + values.grow(); + } + + private int capacity() { + return keys.capacity(); + } + + public void setNumElements(int numElements) { + this.numElements = numElements; + keys.setNumElements(numElements); + values.setNumElements(numElements); + } + + @Override + public int numElements() { + return numElements; + } + + @Override + public MapData copy() { + return new ArrayBasedMapData(keyArray().copy(), valueArray().copy()); + } + + @Override + public ReusableArrayData keyArray() { + return keys; + } + + @Override + public ReusableArrayData valueArray() { + return values; + } + } + + private static class ReusableArrayData extends ArrayData { + private static final Object[] EMPTY = new Object[0]; + + private Object[] values = EMPTY; + private int numElements = 0; + + private void grow() { + if (values.length == 0) { + this.values = new Object[20]; + } else { + Object[] old = values; + this.values = new Object[old.length << 2]; + // copy the old array in case it has values that can be reused + System.arraycopy(old, 0, values, 0, old.length); + } + } + + private int capacity() { + return values.length; + } + + public void setNumElements(int numElements) { + this.numElements = numElements; + } + + @Override + public Object get(int ordinal, DataType dataType) { + return values[ordinal]; + } + + @Override + public int numElements() { + return numElements; + } + + @Override + public ArrayData copy() { + return new GenericArrayData(array()); + } + + @Override + public Object[] array() { + return Arrays.copyOfRange(values, 0, numElements); + } + + @Override + public void setNullAt(int i) { + values[i] = null; + } + + @Override + public void update(int ordinal, Object value) { + values[ordinal] = value; + } + + @Override + public boolean isNullAt(int ordinal) { + return null == values[ordinal]; + } + + @Override + public boolean getBoolean(int ordinal) { + return (boolean) values[ordinal]; + } + + @Override + public byte getByte(int ordinal) { + return (byte) values[ordinal]; + } + + @Override + public short getShort(int ordinal) { + return (short) values[ordinal]; + } + + @Override + public int getInt(int ordinal) { + return (int) values[ordinal]; + } + + @Override + public long getLong(int ordinal) { + return (long) values[ordinal]; + } + + @Override + public float getFloat(int ordinal) { + return (float) values[ordinal]; + } + + @Override + public double getDouble(int ordinal) { + return (double) values[ordinal]; + } + + @Override + public Decimal getDecimal(int ordinal, int precision, int scale) { + return (Decimal) values[ordinal]; + } + + @Override + public UTF8String getUTF8String(int ordinal) { + return (UTF8String) values[ordinal]; + } + + @Override + public byte[] getBinary(int ordinal) { + return (byte[]) values[ordinal]; + } + + @Override + public CalendarInterval getInterval(int ordinal) { + return (CalendarInterval) values[ordinal]; + } + + @Override + public InternalRow getStruct(int ordinal, int numFields) { + return (InternalRow) values[ordinal]; + } + + @Override + public ArrayData getArray(int ordinal) { + return (ArrayData) values[ordinal]; + } + + @Override + public MapData getMap(int ordinal) { + return (MapData) values[ordinal]; + } + + @Override + public VariantVal getVariant(int ordinal) { + return (VariantVal) values[ordinal]; + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java new file mode 100644 index 000000000000..8bdfe7c3a810 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java @@ -0,0 +1,650 @@ +/* + * 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.data; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.UUID; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import org.apache.iceberg.FieldMetrics; +import org.apache.iceberg.parquet.ParquetValueReaders.ReusableEntry; +import org.apache.iceberg.parquet.ParquetValueWriter; +import org.apache.iceberg.parquet.ParquetValueWriters; +import org.apache.iceberg.parquet.ParquetValueWriters.PrimitiveWriter; +import org.apache.iceberg.parquet.ParquetValueWriters.RepeatedKeyValueWriter; +import org.apache.iceberg.parquet.ParquetValueWriters.RepeatedWriter; +import org.apache.iceberg.parquet.ParquetVariantVisitor; +import org.apache.iceberg.parquet.TripleWriter; +import org.apache.iceberg.parquet.VariantWriterBuilder; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.DecimalUtil; +import org.apache.iceberg.util.UUIDUtil; +import org.apache.iceberg.variants.Variant; +import org.apache.iceberg.variants.VariantMetadata; +import org.apache.iceberg.variants.VariantValue; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ColumnWriteStore; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.ByteType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.NullType; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.VariantType; +import org.apache.spark.unsafe.types.UTF8String; +import org.apache.spark.unsafe.types.VariantVal; + +public class SparkParquetWriters { + private SparkParquetWriters() {} + + @SuppressWarnings("unchecked") + public static ParquetValueWriter buildWriter(StructType dfSchema, MessageType type) { + return (ParquetValueWriter) + ParquetWithSparkSchemaVisitor.visit(dfSchema, type, new WriteBuilder(type)); + } + + private static class WriteBuilder extends ParquetWithSparkSchemaVisitor> { + private final MessageType type; + + WriteBuilder(MessageType type) { + this.type = type; + } + + @Override + public ParquetValueWriter message( + StructType sStruct, MessageType message, List> fieldWriters) { + return struct(sStruct, message.asGroupType(), fieldWriters); + } + + @Override + public ParquetValueWriter struct( + StructType sStruct, GroupType struct, List> fieldWriters) { + List fields = struct.getFields(); + List> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size()); + for (int i = 0; i < fields.size(); i += 1) { + writers.add(newOption(struct.getType(i), fieldWriters.get(i))); + } + + StructField[] sFields = sStruct.fields(); + DataType[] types = new DataType[sFields.length]; + for (int i = 0; i < sFields.length; i += 1) { + types[i] = sFields[i].dataType(); + } + + return new InternalRowWriter(writers, types); + } + + @Override + public ParquetValueWriter list( + ArrayType sArray, GroupType array, ParquetValueWriter elementWriter) { + GroupType repeated = array.getFields().get(0).asGroupType(); + String[] repeatedPath = currentPath(); + + int repeatedD = type.getMaxDefinitionLevel(repeatedPath); + int repeatedR = type.getMaxRepetitionLevel(repeatedPath); + + return new ArrayDataWriter<>( + repeatedD, + repeatedR, + newOption(repeated.getType(0), elementWriter), + sArray.elementType()); + } + + @Override + public ParquetValueWriter map( + MapType sMap, + GroupType map, + ParquetValueWriter keyWriter, + ParquetValueWriter valueWriter) { + GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); + String[] repeatedPath = currentPath(); + + int repeatedD = type.getMaxDefinitionLevel(repeatedPath); + int repeatedR = type.getMaxRepetitionLevel(repeatedPath); + + return new MapDataWriter<>( + repeatedD, + repeatedR, + newOption(repeatedKeyValue.getType(0), keyWriter), + newOption(repeatedKeyValue.getType(1), valueWriter), + sMap.keyType(), + sMap.valueType()); + } + + @Override + public ParquetValueWriter variant(VariantType sVariant, GroupType variant) { + ParquetValueWriter writer = + ParquetVariantVisitor.visit( + variant, new VariantWriterBuilder(type, Arrays.asList(currentPath()))); + return new VariantWriter(writer); + } + + private ParquetValueWriter newOption(Type fieldType, ParquetValueWriter writer) { + int maxD = type.getMaxDefinitionLevel(path(fieldType.getName())); + return ParquetValueWriters.option(fieldType, maxD, writer); + } + + private static class LogicalTypeAnnotationParquetValueWriterVisitor + implements LogicalTypeAnnotation.LogicalTypeAnnotationVisitor> { + + private final ColumnDescriptor desc; + private final PrimitiveType primitive; + + LogicalTypeAnnotationParquetValueWriterVisitor( + ColumnDescriptor desc, PrimitiveType primitive) { + this.desc = desc; + this.primitive = primitive; + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { + return Optional.of(utf8Strings(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { + return Optional.of(utf8Strings(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.JsonLogicalTypeAnnotation jsonLogicalType) { + return Optional.of(utf8Strings(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.UUIDLogicalTypeAnnotation uuidLogicalType) { + return Optional.of(uuids(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.MapLogicalTypeAnnotation mapLogicalType) { + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(mapLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.ListLogicalTypeAnnotation listLogicalType) { + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(listLogicalType); + } + + @Override + public Optional> visit(DecimalLogicalTypeAnnotation decimal) { + switch (primitive.getPrimitiveTypeName()) { + case INT32: + return Optional.of(decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale())); + case INT64: + return Optional.of(decimalAsLong(desc, decimal.getPrecision(), decimal.getScale())); + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return Optional.of(decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale())); + } + return Optional.empty(); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { + return Optional.of(ParquetValueWriters.ints(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { + if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return Optional.of(ParquetValueWriters.longs(desc)); + } + return Optional.empty(); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { + if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return Optional.of(ParquetValueWriters.longs(desc)); + } + return Optional.empty(); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.IntLogicalTypeAnnotation intLogicalType) { + int bitWidth = intLogicalType.getBitWidth(); + if (bitWidth <= 8) { + return Optional.of(ParquetValueWriters.tinyints(desc)); + } else if (bitWidth <= 16) { + return Optional.of(ParquetValueWriters.shorts(desc)); + } else if (bitWidth <= 32) { + return Optional.of(ParquetValueWriters.ints(desc)); + } else { + return Optional.of(ParquetValueWriters.longs(desc)); + } + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.BsonLogicalTypeAnnotation bsonLogicalType) { + return Optional.of(byteArrays(desc)); + } + } + + @Override + public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) { + ColumnDescriptor desc = type.getColumnDescription(currentPath()); + LogicalTypeAnnotation logicalTypeAnnotation = primitive.getLogicalTypeAnnotation(); + + if (logicalTypeAnnotation != null) { + return logicalTypeAnnotation + .accept(new LogicalTypeAnnotationParquetValueWriterVisitor(desc, primitive)) + .orElseThrow( + () -> + new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getLogicalTypeAnnotation())); + } + + switch (primitive.getPrimitiveTypeName()) { + case FIXED_LEN_BYTE_ARRAY: + case BINARY: + return byteArrays(desc); + case BOOLEAN: + return ParquetValueWriters.booleans(desc); + case INT32: + return ints(sType, desc); + case INT64: + return ParquetValueWriters.longs(desc); + case FLOAT: + return ParquetValueWriters.floats(desc); + case DOUBLE: + return ParquetValueWriters.doubles(desc); + default: + throw new UnsupportedOperationException("Unsupported type: " + primitive); + } + } + } + + private static PrimitiveWriter ints(DataType type, ColumnDescriptor desc) { + if (type instanceof ByteType) { + return ParquetValueWriters.tinyints(desc); + } else if (type instanceof ShortType) { + return ParquetValueWriters.shorts(desc); + } + return ParquetValueWriters.ints(desc); + } + + private static PrimitiveWriter utf8Strings(ColumnDescriptor desc) { + return new UTF8StringWriter(desc); + } + + private static PrimitiveWriter uuids(ColumnDescriptor desc) { + return new UUIDWriter(desc); + } + + private static PrimitiveWriter decimalAsInteger( + ColumnDescriptor desc, int precision, int scale) { + return new IntegerDecimalWriter(desc, precision, scale); + } + + private static PrimitiveWriter decimalAsLong( + ColumnDescriptor desc, int precision, int scale) { + return new LongDecimalWriter(desc, precision, scale); + } + + private static PrimitiveWriter decimalAsFixed( + ColumnDescriptor desc, int precision, int scale) { + return new FixedDecimalWriter(desc, precision, scale); + } + + private static PrimitiveWriter byteArrays(ColumnDescriptor desc) { + return new ByteArrayWriter(desc); + } + + private static class UTF8StringWriter extends PrimitiveWriter { + private UTF8StringWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, UTF8String value) { + column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(value.getBytes())); + } + } + + private static class IntegerDecimalWriter extends PrimitiveWriter { + private final int precision; + private final int scale; + + private IntegerDecimalWriter(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public void write(int repetitionLevel, Decimal decimal) { + Preconditions.checkArgument( + decimal.scale() == scale, + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + decimal); + Preconditions.checkArgument( + decimal.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + decimal); + + column.writeInteger(repetitionLevel, (int) decimal.toUnscaledLong()); + } + } + + private static class LongDecimalWriter extends PrimitiveWriter { + private final int precision; + private final int scale; + + private LongDecimalWriter(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public void write(int repetitionLevel, Decimal decimal) { + Preconditions.checkArgument( + decimal.scale() == scale, + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + decimal); + Preconditions.checkArgument( + decimal.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + decimal); + + column.writeLong(repetitionLevel, decimal.toUnscaledLong()); + } + } + + private static class FixedDecimalWriter extends PrimitiveWriter { + private final int precision; + private final int scale; + private final ThreadLocal bytes; + + private FixedDecimalWriter(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + this.bytes = + ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); + } + + @Override + public void write(int repetitionLevel, Decimal decimal) { + byte[] binary = + DecimalUtil.toReusedFixLengthBytes( + precision, scale, decimal.toJavaBigDecimal(), bytes.get()); + column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(binary)); + } + } + + private static class UUIDWriter extends PrimitiveWriter { + private static final ThreadLocal BUFFER = + ThreadLocal.withInitial( + () -> { + ByteBuffer buffer = ByteBuffer.allocate(16); + buffer.order(ByteOrder.BIG_ENDIAN); + return buffer; + }); + + private UUIDWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, UTF8String string) { + UUID uuid = UUID.fromString(string.toString()); + ByteBuffer buffer = UUIDUtil.convertToByteBuffer(uuid, BUFFER.get()); + column.writeBinary(repetitionLevel, Binary.fromReusedByteBuffer(buffer)); + } + } + + private static class ByteArrayWriter extends PrimitiveWriter { + private ByteArrayWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, byte[] bytes) { + column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(bytes)); + } + } + + private static class ArrayDataWriter extends RepeatedWriter { + private final DataType elementType; + + private ArrayDataWriter( + int definitionLevel, + int repetitionLevel, + ParquetValueWriter writer, + DataType elementType) { + super(definitionLevel, repetitionLevel, writer); + this.elementType = elementType; + } + + @Override + protected Iterator elements(ArrayData list) { + return new ElementIterator<>(list); + } + + private class ElementIterator implements Iterator { + private final int size; + private final ArrayData list; + private int index; + + private ElementIterator(ArrayData list) { + this.list = list; + size = list.numElements(); + index = 0; + } + + @Override + public boolean hasNext() { + return index != size; + } + + @Override + @SuppressWarnings("unchecked") + public E next() { + if (index >= size) { + throw new NoSuchElementException(); + } + + E element; + if (list.isNullAt(index)) { + element = null; + } else { + element = (E) list.get(index, elementType); + } + + index += 1; + + return element; + } + } + } + + private static class MapDataWriter extends RepeatedKeyValueWriter { + private final DataType keyType; + private final DataType valueType; + + private MapDataWriter( + int definitionLevel, + int repetitionLevel, + ParquetValueWriter keyWriter, + ParquetValueWriter valueWriter, + DataType keyType, + DataType valueType) { + super(definitionLevel, repetitionLevel, keyWriter, valueWriter); + this.keyType = keyType; + this.valueType = valueType; + } + + @Override + protected Iterator> pairs(MapData map) { + return new EntryIterator<>(map); + } + + private class EntryIterator implements Iterator> { + private final int size; + private final ArrayData keys; + private final ArrayData values; + private final ReusableEntry entry; + private int index; + + private EntryIterator(MapData map) { + size = map.numElements(); + keys = map.keyArray(); + values = map.valueArray(); + entry = new ReusableEntry<>(); + index = 0; + } + + @Override + public boolean hasNext() { + return index != size; + } + + @Override + @SuppressWarnings("unchecked") + public Map.Entry next() { + if (index >= size) { + throw new NoSuchElementException(); + } + + if (values.isNullAt(index)) { + entry.set((K) keys.get(index, keyType), null); + } else { + entry.set((K) keys.get(index, keyType), (V) values.get(index, valueType)); + } + + index += 1; + + return entry; + } + } + } + + /** Variant writer converts from VariantVal to Variant */ + public static class VariantWriter implements ParquetValueWriter { + private final ParquetValueWriter writer; + + @SuppressWarnings("unchecked") + private VariantWriter(ParquetValueWriter writer) { + this.writer = (ParquetValueWriter) writer; + } + + @Override + public void write(int repetitionLevel, VariantVal variantVal) { + VariantMetadata metadata = + VariantMetadata.from( + ByteBuffer.wrap(variantVal.getMetadata()).order(ByteOrder.LITTLE_ENDIAN)); + VariantValue value = + VariantValue.from( + metadata, ByteBuffer.wrap(variantVal.getValue()).order(ByteOrder.LITTLE_ENDIAN)); + + writer.write(repetitionLevel, Variant.of(metadata, value)); + } + + @Override + public List> columns() { + return writer.columns(); + } + + @Override + public void setColumnStore(ColumnWriteStore columnStore) { + writer.setColumnStore(columnStore); + } + + @Override + public Stream> metrics() { + return writer.metrics(); + } + } + + private static class InternalRowWriter extends ParquetValueWriters.StructWriter { + private final DataType[] types; + + private InternalRowWriter(List> writers, DataType[] types) { + super(writerToFieldIndex(types, writers.size()), writers); + this.types = types; + } + + @Override + protected Object get(InternalRow struct, int index) { + return struct.get(index, types[index]); + } + + /** Returns a mapping from writer index to field index, skipping Unknown columns. */ + private static int[] writerToFieldIndex(DataType[] types, int numWriters) { + if (null == types) { + return IntStream.rangeClosed(0, numWriters).toArray(); + } + + // value writer index to record field index + int[] indexes = new int[numWriters]; + int writerIndex = 0; + for (int pos = 0; pos < types.length; pos += 1) { + if (!(types[pos] instanceof NullType)) { + indexes[writerIndex] = pos; + writerIndex += 1; + } + } + + return indexes; + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java new file mode 100644 index 000000000000..596f94cd053f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkPlannedAvroReader.java @@ -0,0 +1,198 @@ +/* + * 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.data; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.avro.LogicalType; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.Decoder; +import org.apache.iceberg.avro.AvroWithPartnerVisitor; +import org.apache.iceberg.avro.SupportsRowPosition; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkUtil; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.catalyst.InternalRow; + +public class SparkPlannedAvroReader implements DatumReader, SupportsRowPosition { + + private final Types.StructType expectedType; + private final Map idToConstant; + private ValueReader reader; + + public static SparkPlannedAvroReader create(org.apache.iceberg.Schema schema) { + return create(schema, ImmutableMap.of()); + } + + public static SparkPlannedAvroReader create( + org.apache.iceberg.Schema schema, Map constants) { + return new SparkPlannedAvroReader(schema, constants); + } + + private SparkPlannedAvroReader( + org.apache.iceberg.Schema expectedSchema, Map constants) { + this.expectedType = expectedSchema.asStruct(); + this.idToConstant = constants; + } + + @Override + @SuppressWarnings("unchecked") + public void setSchema(Schema fileSchema) { + this.reader = + (ValueReader) + AvroWithPartnerVisitor.visit( + expectedType, + fileSchema, + new ReadBuilder(idToConstant), + AvroWithPartnerVisitor.FieldIDAccessors.get()); + } + + @Override + public InternalRow read(InternalRow reuse, Decoder decoder) throws IOException { + return reader.read(decoder, reuse); + } + + @Override + public void setRowPositionSupplier(Supplier posSupplier) { + if (reader instanceof SupportsRowPosition) { + ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); + } + } + + private static class ReadBuilder extends AvroWithPartnerVisitor> { + private final Map idToConstant; + + private ReadBuilder(Map idToConstant) { + this.idToConstant = idToConstant; + } + + @Override + public ValueReader record(Type partner, Schema record, List> fieldReaders) { + if (partner == null) { + return ValueReaders.skipStruct(fieldReaders); + } + + Types.StructType expected = partner.asStructType(); + List>> readPlan = + ValueReaders.buildReadPlan( + expected, record, fieldReaders, idToConstant, SparkUtil::internalToSpark); + + // TODO: should this pass expected so that struct.get can reuse containers? + return SparkValueReaders.struct(readPlan, expected.fields().size()); + } + + @Override + public ValueReader union(Type partner, Schema union, List> options) { + return ValueReaders.union(options); + } + + @Override + public ValueReader array(Type partner, Schema array, ValueReader elementReader) { + return SparkValueReaders.array(elementReader); + } + + @Override + public ValueReader arrayMap( + Type partner, Schema map, ValueReader keyReader, ValueReader valueReader) { + return SparkValueReaders.arrayMap(keyReader, valueReader); + } + + @Override + public ValueReader map(Type partner, Schema map, ValueReader valueReader) { + return SparkValueReaders.map(SparkValueReaders.strings(), valueReader); + } + + @Override + public ValueReader variant( + Type partner, ValueReader metadataReader, ValueReader valueReader) { + return SparkValueReaders.variants(); + } + + @Override + public ValueReader primitive(Type partner, Schema primitive) { + LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + // Spark uses the same representation + return ValueReaders.ints(); + + case "timestamp-millis": + // adjust to microseconds + ValueReader longs = ValueReaders.longs(); + return (ValueReader) (decoder, ignored) -> longs.read(decoder, null) * 1000L; + + case "timestamp-micros": + // Spark uses the same representation + return ValueReaders.longs(); + + case "decimal": + return SparkValueReaders.decimal( + ValueReaders.decimalBytesReader(primitive), + ((LogicalTypes.Decimal) logicalType).getScale()); + + case "uuid": + return SparkValueReaders.uuids(); + + default: + throw new IllegalArgumentException("Unknown logical type: " + logicalType); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueReaders.nulls(); + case BOOLEAN: + return ValueReaders.booleans(); + case INT: + if (partner != null && partner.typeId() == Type.TypeID.LONG) { + return ValueReaders.intsAsLongs(); + } + return ValueReaders.ints(); + case LONG: + return ValueReaders.longs(); + case FLOAT: + if (partner != null && partner.typeId() == Type.TypeID.DOUBLE) { + return ValueReaders.floatsAsDoubles(); + } + return ValueReaders.floats(); + case DOUBLE: + return ValueReaders.doubles(); + case STRING: + return SparkValueReaders.strings(); + case FIXED: + return ValueReaders.fixed(primitive.getFixedSize()); + case BYTES: + return ValueReaders.bytes(); + case ENUM: + return SparkValueReaders.enums(primitive.getEnumSymbols()); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java new file mode 100644 index 000000000000..7bcb3e3fae01 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueReaders.java @@ -0,0 +1,352 @@ +/* + * 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.data; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import org.apache.avro.io.Decoder; +import org.apache.avro.util.Utf8; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.UUIDUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.GenericArrayData; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.types.UTF8String; +import org.apache.spark.unsafe.types.VariantVal; + +public class SparkValueReaders { + + private SparkValueReaders() {} + + static ValueReader strings() { + return StringReader.INSTANCE; + } + + static ValueReader enums(List symbols) { + return new EnumReader(symbols); + } + + static ValueReader uuids() { + return UUIDReader.INSTANCE; + } + + static ValueReader variants() { + return VariantReader.INSTANCE; + } + + static ValueReader decimal(ValueReader unscaledReader, int scale) { + return new DecimalReader(unscaledReader, scale); + } + + static ValueReader array(ValueReader elementReader) { + return new ArrayReader(elementReader); + } + + static ValueReader arrayMap( + ValueReader keyReader, ValueReader valueReader) { + return new ArrayMapReader(keyReader, valueReader); + } + + static ValueReader map(ValueReader keyReader, ValueReader valueReader) { + return new MapReader(keyReader, valueReader); + } + + static ValueReader struct( + List>> readPlan, int numFields) { + return new PlannedStructReader(readPlan, numFields); + } + + static ValueReader struct( + List> readers, Types.StructType struct, Map idToConstant) { + return new StructReader(readers, struct, idToConstant); + } + + private static class StringReader implements ValueReader { + private static final StringReader INSTANCE = new StringReader(); + + private StringReader() {} + + @Override + public UTF8String read(Decoder decoder, Object reuse) throws IOException { + // use the decoder's readString(Utf8) method because it may be a resolving decoder + Utf8 utf8 = null; + if (reuse instanceof UTF8String) { + utf8 = new Utf8(((UTF8String) reuse).getBytes()); + } + + Utf8 string = decoder.readString(utf8); + return UTF8String.fromBytes(string.getBytes(), 0, string.getByteLength()); + } + } + + private static class EnumReader implements ValueReader { + private final UTF8String[] symbols; + + private EnumReader(List symbols) { + this.symbols = new UTF8String[symbols.size()]; + for (int i = 0; i < this.symbols.length; i += 1) { + this.symbols[i] = UTF8String.fromBytes(symbols.get(i).getBytes(StandardCharsets.UTF_8)); + } + } + + @Override + public UTF8String read(Decoder decoder, Object ignore) throws IOException { + int index = decoder.readEnum(); + return symbols[index]; + } + } + + private static class UUIDReader implements ValueReader { + private static final ThreadLocal BUFFER = + ThreadLocal.withInitial( + () -> { + ByteBuffer buffer = ByteBuffer.allocate(16); + buffer.order(ByteOrder.BIG_ENDIAN); + return buffer; + }); + + private static final UUIDReader INSTANCE = new UUIDReader(); + + private UUIDReader() {} + + @Override + @SuppressWarnings("ByteBufferBackingArray") + public UTF8String read(Decoder decoder, Object reuse) throws IOException { + ByteBuffer buffer = BUFFER.get(); + buffer.rewind(); + + decoder.readFixed(buffer.array(), 0, 16); + + return UTF8String.fromString(UUIDUtil.convert(buffer).toString()); + } + } + + private static class DecimalReader implements ValueReader { + private final ValueReader bytesReader; + private final int scale; + + private DecimalReader(ValueReader bytesReader, int scale) { + this.bytesReader = bytesReader; + this.scale = scale; + } + + @Override + public Decimal read(Decoder decoder, Object reuse) throws IOException { + byte[] bytes = bytesReader.read(decoder, null); + return Decimal.apply(new BigDecimal(new BigInteger(bytes), scale)); + } + } + + private static class VariantReader implements ValueReader { + private static final VariantReader INSTANCE = new VariantReader(); + + private final ValueReader metadataReader; + private final ValueReader valueReader; + + private VariantReader() { + this.metadataReader = ValueReaders.bytes(); + this.valueReader = ValueReaders.bytes(); + } + + @Override + public VariantVal read(Decoder decoder, Object reuse) throws IOException { + byte[] metadata = metadataReader.read(decoder, null); + byte[] value = valueReader.read(decoder, null); + return new VariantVal(value, metadata); + } + + @Override + public void skip(Decoder decoder) throws IOException { + metadataReader.skip(decoder); + valueReader.skip(decoder); + } + } + + private static class ArrayReader implements ValueReader { + private final ValueReader elementReader; + private final List reusedList = Lists.newArrayList(); + + private ArrayReader(ValueReader elementReader) { + this.elementReader = elementReader; + } + + @Override + public GenericArrayData read(Decoder decoder, Object reuse) throws IOException { + reusedList.clear(); + long chunkLength = decoder.readArrayStart(); + + while (chunkLength > 0) { + for (int i = 0; i < chunkLength; i += 1) { + reusedList.add(elementReader.read(decoder, null)); + } + + chunkLength = decoder.arrayNext(); + } + + // this will convert the list to an array so it is okay to reuse the list + return new GenericArrayData(reusedList.toArray()); + } + } + + private static class ArrayMapReader implements ValueReader { + private final ValueReader keyReader; + private final ValueReader valueReader; + + private final List reusedKeyList = Lists.newArrayList(); + private final List reusedValueList = Lists.newArrayList(); + + private ArrayMapReader(ValueReader keyReader, ValueReader valueReader) { + this.keyReader = keyReader; + this.valueReader = valueReader; + } + + @Override + public ArrayBasedMapData read(Decoder decoder, Object reuse) throws IOException { + reusedKeyList.clear(); + reusedValueList.clear(); + + long chunkLength = decoder.readArrayStart(); + + while (chunkLength > 0) { + for (int i = 0; i < chunkLength; i += 1) { + reusedKeyList.add(keyReader.read(decoder, null)); + reusedValueList.add(valueReader.read(decoder, null)); + } + + chunkLength = decoder.arrayNext(); + } + + return new ArrayBasedMapData( + new GenericArrayData(reusedKeyList.toArray()), + new GenericArrayData(reusedValueList.toArray())); + } + } + + private static class MapReader implements ValueReader { + private final ValueReader keyReader; + private final ValueReader valueReader; + + private final List reusedKeyList = Lists.newArrayList(); + private final List reusedValueList = Lists.newArrayList(); + + private MapReader(ValueReader keyReader, ValueReader valueReader) { + this.keyReader = keyReader; + this.valueReader = valueReader; + } + + @Override + public ArrayBasedMapData read(Decoder decoder, Object reuse) throws IOException { + reusedKeyList.clear(); + reusedValueList.clear(); + + long chunkLength = decoder.readMapStart(); + + while (chunkLength > 0) { + for (int i = 0; i < chunkLength; i += 1) { + reusedKeyList.add(keyReader.read(decoder, null)); + reusedValueList.add(valueReader.read(decoder, null)); + } + + chunkLength = decoder.mapNext(); + } + + return new ArrayBasedMapData( + new GenericArrayData(reusedKeyList.toArray()), + new GenericArrayData(reusedValueList.toArray())); + } + } + + static class PlannedStructReader extends ValueReaders.PlannedStructReader { + private final int numFields; + + protected PlannedStructReader(List>> readPlan, int numFields) { + super(readPlan); + this.numFields = numFields; + } + + @Override + protected InternalRow reuseOrCreate(Object reuse) { + if (reuse instanceof GenericInternalRow + && ((GenericInternalRow) reuse).numFields() == numFields) { + return (InternalRow) reuse; + } + return new GenericInternalRow(numFields); + } + + @Override + protected Object get(InternalRow struct, int pos) { + return null; + } + + @Override + protected void set(InternalRow struct, int pos, Object value) { + if (value != null) { + struct.update(pos, value); + } else { + struct.setNullAt(pos); + } + } + } + + static class StructReader extends ValueReaders.StructReader { + private final int numFields; + + protected StructReader( + List> readers, Types.StructType struct, Map idToConstant) { + super(readers, struct, idToConstant); + this.numFields = readers.size(); + } + + @Override + protected InternalRow reuseOrCreate(Object reuse) { + if (reuse instanceof GenericInternalRow + && ((GenericInternalRow) reuse).numFields() == numFields) { + return (InternalRow) reuse; + } + return new GenericInternalRow(numFields); + } + + @Override + protected Object get(InternalRow struct, int pos) { + return null; + } + + @Override + protected void set(InternalRow struct, int pos, Object value) { + if (value != null) { + struct.update(pos, value); + } else { + struct.setNullAt(pos); + } + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java new file mode 100644 index 000000000000..d0d3483a7690 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/SparkValueWriters.java @@ -0,0 +1,280 @@ +/* + * 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.data; + +import java.io.IOException; +import java.lang.reflect.Array; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.List; +import java.util.UUID; +import org.apache.avro.io.Encoder; +import org.apache.avro.util.Utf8; +import org.apache.iceberg.avro.ValueWriter; +import org.apache.iceberg.avro.ValueWriters; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.DecimalUtil; +import org.apache.iceberg.util.UUIDUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.types.UTF8String; +import org.apache.spark.unsafe.types.VariantVal; + +public class SparkValueWriters { + + private SparkValueWriters() {} + + static ValueWriter strings() { + return StringWriter.INSTANCE; + } + + static ValueWriter uuids() { + return UUIDWriter.INSTANCE; + } + + static ValueWriter variants() { + return VariantWriter.INSTANCE; + } + + static ValueWriter decimal(int precision, int scale) { + return new DecimalWriter(precision, scale); + } + + static ValueWriter array(ValueWriter elementWriter, DataType elementType) { + return new ArrayWriter<>(elementWriter, elementType); + } + + static ValueWriter arrayMap( + ValueWriter keyWriter, DataType keyType, ValueWriter valueWriter, DataType valueType) { + return new ArrayMapWriter<>(keyWriter, keyType, valueWriter, valueType); + } + + static ValueWriter map( + ValueWriter keyWriter, DataType keyType, ValueWriter valueWriter, DataType valueType) { + return new MapWriter<>(keyWriter, keyType, valueWriter, valueType); + } + + static ValueWriter struct(List> writers, List types) { + return new StructWriter(writers, types); + } + + private static class StringWriter implements ValueWriter { + private static final StringWriter INSTANCE = new StringWriter(); + + private StringWriter() {} + + @Override + public void write(UTF8String s, Encoder encoder) throws IOException { + // use getBytes because it may return the backing byte array if available. + // otherwise, it copies to a new byte array, which is still cheaper than Avro + // calling toString, which incurs encoding costs + encoder.writeString(new Utf8(s.getBytes())); + } + } + + private static class UUIDWriter implements ValueWriter { + private static final ThreadLocal BUFFER = + ThreadLocal.withInitial( + () -> { + ByteBuffer buffer = ByteBuffer.allocate(16); + buffer.order(ByteOrder.BIG_ENDIAN); + return buffer; + }); + + private static final UUIDWriter INSTANCE = new UUIDWriter(); + + private UUIDWriter() {} + + @Override + @SuppressWarnings("ByteBufferBackingArray") + public void write(UTF8String s, Encoder encoder) throws IOException { + // TODO: direct conversion from string to byte buffer + UUID uuid = UUID.fromString(s.toString()); + // calling array() is safe because the buffer is always allocated by the thread-local + encoder.writeFixed(UUIDUtil.convertToByteBuffer(uuid, BUFFER.get()).array()); + } + } + + private static class VariantWriter implements ValueWriter { + private static final VariantWriter INSTANCE = new VariantWriter(); + + private final ValueWriter metadataWriter; + private final ValueWriter valueWriter; + + private VariantWriter() { + this.metadataWriter = ValueWriters.bytes(); + this.valueWriter = ValueWriters.bytes(); + } + + @Override + public void write(VariantVal variant, Encoder encoder) throws IOException { + metadataWriter.write(variant.getMetadata(), encoder); + valueWriter.write(variant.getValue(), encoder); + } + } + + private static class DecimalWriter implements ValueWriter { + private final int precision; + private final int scale; + private final ThreadLocal bytes; + + private DecimalWriter(int precision, int scale) { + this.precision = precision; + this.scale = scale; + this.bytes = + ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); + } + + @Override + public void write(Decimal d, Encoder encoder) throws IOException { + encoder.writeFixed( + DecimalUtil.toReusedFixLengthBytes(precision, scale, d.toJavaBigDecimal(), bytes.get())); + } + } + + private static class ArrayWriter implements ValueWriter { + private final ValueWriter elementWriter; + private final DataType elementType; + + private ArrayWriter(ValueWriter elementWriter, DataType elementType) { + this.elementWriter = elementWriter; + this.elementType = elementType; + } + + @Override + @SuppressWarnings("unchecked") + public void write(ArrayData array, Encoder encoder) throws IOException { + encoder.writeArrayStart(); + int numElements = array.numElements(); + encoder.setItemCount(numElements); + for (int i = 0; i < numElements; i += 1) { + encoder.startItem(); + elementWriter.write((T) array.get(i, elementType), encoder); + } + encoder.writeArrayEnd(); + } + } + + private static class ArrayMapWriter implements ValueWriter { + private final ValueWriter keyWriter; + private final ValueWriter valueWriter; + private final DataType keyType; + private final DataType valueType; + + private ArrayMapWriter( + ValueWriter keyWriter, + DataType keyType, + ValueWriter valueWriter, + DataType valueType) { + this.keyWriter = keyWriter; + this.keyType = keyType; + this.valueWriter = valueWriter; + this.valueType = valueType; + } + + @Override + @SuppressWarnings("unchecked") + public void write(MapData map, Encoder encoder) throws IOException { + encoder.writeArrayStart(); + int numElements = map.numElements(); + encoder.setItemCount(numElements); + ArrayData keyArray = map.keyArray(); + ArrayData valueArray = map.valueArray(); + for (int i = 0; i < numElements; i += 1) { + encoder.startItem(); + keyWriter.write((K) keyArray.get(i, keyType), encoder); + valueWriter.write((V) valueArray.get(i, valueType), encoder); + } + encoder.writeArrayEnd(); + } + } + + private static class MapWriter implements ValueWriter { + private final ValueWriter keyWriter; + private final ValueWriter valueWriter; + private final DataType keyType; + private final DataType valueType; + + private MapWriter( + ValueWriter keyWriter, + DataType keyType, + ValueWriter valueWriter, + DataType valueType) { + this.keyWriter = keyWriter; + this.keyType = keyType; + this.valueWriter = valueWriter; + this.valueType = valueType; + } + + @Override + @SuppressWarnings("unchecked") + public void write(MapData map, Encoder encoder) throws IOException { + encoder.writeMapStart(); + int numElements = map.numElements(); + encoder.setItemCount(numElements); + ArrayData keyArray = map.keyArray(); + ArrayData valueArray = map.valueArray(); + for (int i = 0; i < numElements; i += 1) { + encoder.startItem(); + keyWriter.write((K) keyArray.get(i, keyType), encoder); + valueWriter.write((V) valueArray.get(i, valueType), encoder); + } + encoder.writeMapEnd(); + } + } + + static class StructWriter implements ValueWriter { + private final ValueWriter[] writers; + private final DataType[] types; + + @SuppressWarnings("unchecked") + private StructWriter(List> writers, List types) { + this.writers = (ValueWriter[]) Array.newInstance(ValueWriter.class, writers.size()); + this.types = new DataType[writers.size()]; + for (int i = 0; i < writers.size(); i += 1) { + this.writers[i] = writers.get(i); + this.types[i] = types.get(i); + } + } + + ValueWriter[] writers() { + return writers; + } + + @Override + public void write(InternalRow row, Encoder encoder) throws IOException { + for (int i = 0; i < types.length; i += 1) { + if (row.isNullAt(i)) { + writers[i].write(null, encoder); + } else { + write(row, i, writers[i], encoder); + } + } + } + + @SuppressWarnings("unchecked") + private void write(InternalRow row, int pos, ValueWriter writer, Encoder encoder) + throws IOException { + writer.write((T) row.get(pos, types[pos]), encoder); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessorFactory.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessorFactory.java new file mode 100644 index 000000000000..b4bb9a918732 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessorFactory.java @@ -0,0 +1,140 @@ +/* + * 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.data.vectorized; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import org.apache.arrow.memory.ArrowBuf; +import org.apache.arrow.vector.FixedSizeBinaryVector; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.ValueVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.complex.ListVector; +import org.apache.iceberg.arrow.vectorized.GenericArrowVectorAccessorFactory; +import org.apache.iceberg.util.UUIDUtil; +import org.apache.parquet.column.Dictionary; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.vectorized.ArrowColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.unsafe.types.UTF8String; + +final class ArrowVectorAccessorFactory + extends GenericArrowVectorAccessorFactory< + Decimal, UTF8String, ColumnarArray, ArrowColumnVector> { + + ArrowVectorAccessorFactory() { + super( + DecimalFactoryImpl::new, + StringFactoryImpl::new, + StructChildFactoryImpl::new, + ArrayFactoryImpl::new); + } + + private static final class DecimalFactoryImpl implements DecimalFactory { + @Override + public Class getGenericClass() { + return Decimal.class; + } + + @Override + public Decimal ofLong(long value, int precision, int scale) { + return Decimal.apply(value, precision, scale); + } + + @Override + public Decimal ofBigDecimal(BigDecimal value, int precision, int scale) { + return Decimal.apply(value, precision, scale); + } + } + + private static final class StringFactoryImpl implements StringFactory { + @Override + public Class getGenericClass() { + return UTF8String.class; + } + + @Override + public UTF8String ofRow(VarCharVector vector, int rowId) { + int start = vector.getStartOffset(rowId); + int end = vector.getEndOffset(rowId); + + return UTF8String.fromAddress( + null, vector.getDataBuffer().memoryAddress() + start, end - start); + } + + @Override + public UTF8String ofRow(FixedSizeBinaryVector vector, int rowId) { + return UTF8String.fromString(UUIDUtil.convert(vector.get(rowId)).toString()); + } + + @Override + public UTF8String ofRow(IntVector offsetVector, Dictionary dictionary, int rowId) { + byte[] bytes = dictionary.decodeToBinary(offsetVector.get(rowId)).getBytes(); + return UTF8String.fromString(UUIDUtil.convert(bytes).toString()); + } + + @Override + public UTF8String ofBytes(byte[] bytes) { + return UTF8String.fromBytes(bytes); + } + + @Override + public UTF8String ofByteBuffer(ByteBuffer byteBuffer) { + if (byteBuffer.hasArray()) { + return UTF8String.fromBytes( + byteBuffer.array(), + byteBuffer.arrayOffset() + byteBuffer.position(), + byteBuffer.remaining()); + } + byte[] bytes = new byte[byteBuffer.remaining()]; + byteBuffer.get(bytes); + return UTF8String.fromBytes(bytes); + } + } + + private static final class ArrayFactoryImpl + implements ArrayFactory { + @Override + public ArrowColumnVector ofChild(ValueVector childVector) { + return new ArrowColumnVector(childVector); + } + + @Override + public ColumnarArray ofRow(ValueVector vector, ArrowColumnVector childData, int rowId) { + ArrowBuf offsets = vector.getOffsetBuffer(); + int index = rowId * ListVector.OFFSET_WIDTH; + int start = offsets.getInt(index); + int end = offsets.getInt(index + ListVector.OFFSET_WIDTH); + return new ColumnarArray(childData, start, end - start); + } + } + + private static final class StructChildFactoryImpl + implements StructChildFactory { + @Override + public Class getGenericClass() { + return ArrowColumnVector.class; + } + + @Override + public ArrowColumnVector of(ValueVector childVector) { + return new ArrowColumnVector(childVector); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java new file mode 100644 index 000000000000..4e02dafb3c13 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java @@ -0,0 +1,38 @@ +/* + * 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.data.vectorized; + +import org.apache.iceberg.arrow.vectorized.ArrowVectorAccessor; +import org.apache.iceberg.arrow.vectorized.VectorHolder; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.vectorized.ArrowColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.unsafe.types.UTF8String; + +public class ArrowVectorAccessors { + + private static final ArrowVectorAccessorFactory FACTORY = new ArrowVectorAccessorFactory(); + + static ArrowVectorAccessor + getVectorAccessor(VectorHolder holder) { + return FACTORY.getVectorAccessor(holder); + } + + private ArrowVectorAccessors() {} +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorBuilder.java new file mode 100644 index 000000000000..61616a9f233c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorBuilder.java @@ -0,0 +1,45 @@ +/* + * 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.data.vectorized; + +import org.apache.iceberg.arrow.vectorized.VectorHolder; +import org.apache.iceberg.arrow.vectorized.VectorHolder.ConstantVectorHolder; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.vectorized.ColumnVector; + +class ColumnVectorBuilder { + + public ColumnVector build(VectorHolder holder, int numRows) { + if (holder.isDummy()) { + if (holder instanceof VectorHolder.DeletedVectorHolder) { + return new DeletedColumnVector(Types.BooleanType.get()); + } else if (holder instanceof ConstantVectorHolder) { + ConstantVectorHolder constantHolder = (ConstantVectorHolder) holder; + Type icebergType = constantHolder.icebergType(); + Object value = constantHolder.getConstant(); + return new ConstantColumnVector(icebergType, numRows, value); + } else { + throw new IllegalStateException("Unknown dummy vector holder: " + holder); + } + } else { + return new IcebergArrowColumnVector(holder); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorWithFilter.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorWithFilter.java new file mode 100644 index 000000000000..edaaaeda2515 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnVectorWithFilter.java @@ -0,0 +1,155 @@ +/* + * 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.data.vectorized; + +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +/** + * A column vector implementation that applies row-level filtering. + * + *

    This class wraps an existing column vector and uses a row ID mapping array to remap row + * indices during data access. Each method that retrieves data for a specific row translates the + * provided row index using the mapping array, effectively filtering the original data to only + * expose the live subset of rows. This approach allows efficient row-level filtering without + * modifying the underlying data. + */ +public class ColumnVectorWithFilter extends ColumnVector { + private final ColumnVector delegate; + private final int[] rowIdMapping; + private volatile ColumnVectorWithFilter[] children = null; + + public ColumnVectorWithFilter(ColumnVector delegate, int[] rowIdMapping) { + super(delegate.dataType()); + this.delegate = delegate; + this.rowIdMapping = rowIdMapping; + } + + @Override + public void close() { + delegate.close(); + } + + @Override + public void closeIfFreeable() { + delegate.closeIfFreeable(); + } + + @Override + public boolean hasNull() { + return delegate.hasNull(); + } + + @Override + public int numNulls() { + // computing the actual number of nulls with rowIdMapping is expensive + // it is OK to overestimate and return the number of nulls in the original vector + return delegate.numNulls(); + } + + @Override + public boolean isNullAt(int rowId) { + return delegate.isNullAt(rowIdMapping[rowId]); + } + + @Override + public boolean getBoolean(int rowId) { + return delegate.getBoolean(rowIdMapping[rowId]); + } + + @Override + public byte getByte(int rowId) { + return delegate.getByte(rowIdMapping[rowId]); + } + + @Override + public short getShort(int rowId) { + return delegate.getShort(rowIdMapping[rowId]); + } + + @Override + public int getInt(int rowId) { + return delegate.getInt(rowIdMapping[rowId]); + } + + @Override + public long getLong(int rowId) { + return delegate.getLong(rowIdMapping[rowId]); + } + + @Override + public float getFloat(int rowId) { + return delegate.getFloat(rowIdMapping[rowId]); + } + + @Override + public double getDouble(int rowId) { + return delegate.getDouble(rowIdMapping[rowId]); + } + + @Override + public ColumnarArray getArray(int rowId) { + return delegate.getArray(rowIdMapping[rowId]); + } + + @Override + public ColumnarMap getMap(int rowId) { + return delegate.getMap(rowIdMapping[rowId]); + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + return delegate.getDecimal(rowIdMapping[rowId], precision, scale); + } + + @Override + public UTF8String getUTF8String(int rowId) { + return delegate.getUTF8String(rowIdMapping[rowId]); + } + + @Override + public byte[] getBinary(int rowId) { + return delegate.getBinary(rowIdMapping[rowId]); + } + + @Override + public ColumnVector getChild(int ordinal) { + if (children == null) { + synchronized (this) { + if (children == null) { + if (dataType() instanceof StructType) { + StructType structType = (StructType) dataType(); + this.children = new ColumnVectorWithFilter[structType.length()]; + for (int index = 0; index < structType.length(); index++) { + children[index] = new ColumnVectorWithFilter(delegate.getChild(index), rowIdMapping); + } + } else { + throw new UnsupportedOperationException("Unsupported nested type: " + dataType()); + } + } + } + } + + return children[ordinal]; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java new file mode 100644 index 000000000000..38d505d250d0 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchReader.java @@ -0,0 +1,94 @@ +/* + * 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.data.vectorized; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.arrow.vectorized.BaseBatchReader; +import org.apache.iceberg.arrow.vectorized.VectorizedArrowReader; +import org.apache.iceberg.parquet.VectorizedReader; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +/** + * {@link VectorizedReader} that returns Spark's {@link ColumnarBatch} to support Spark's vectorized + * read path. The {@link ColumnarBatch} returned is created by passing in the Arrow vectors + * populated via delegated read calls to {@linkplain VectorizedArrowReader VectorReader(s)}. + */ +public class ColumnarBatchReader extends BaseBatchReader { + + public ColumnarBatchReader(List> readers) { + super(readers); + } + + @Override + public void setRowGroupInfo( + PageReadStore pageStore, Map metaData) { + super.setRowGroupInfo(pageStore, metaData); + } + + @Override + public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) { + if (reuse == null) { + closeVectors(); + } + + return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); + } + + private class ColumnBatchLoader { + private final int batchSize; + + ColumnBatchLoader(int numRowsToRead) { + Preconditions.checkArgument( + numRowsToRead > 0, "Invalid number of rows to read: %s", numRowsToRead); + this.batchSize = numRowsToRead; + } + + ColumnarBatch loadDataToColumnBatch() { + ColumnVector[] vectors = readDataToColumnVectors(); + + ColumnarBatch batch = new ColumnarBatch(vectors); + batch.setNumRows(batchSize); + return batch; + } + + ColumnVector[] readDataToColumnVectors() { + ColumnVector[] arrowColumnVectors = new ColumnVector[readers.length]; + + ColumnVectorBuilder columnVectorBuilder = new ColumnVectorBuilder(); + for (int i = 0; i < readers.length; i += 1) { + vectorHolders[i] = readers[i].read(vectorHolders[i], batchSize); + int numRowsInVector = vectorHolders[i].numValues(); + Preconditions.checkState( + numRowsInVector == batchSize, + "Number of rows in the vector %s didn't match expected %s ", + numRowsInVector, + batchSize); + + arrowColumnVectors[i] = columnVectorBuilder.build(vectorHolders[i], numRowsInVector); + } + return arrowColumnVectors; + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchUtil.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchUtil.java new file mode 100644 index 000000000000..89fe4538bdc2 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ColumnarBatchUtil.java @@ -0,0 +1,181 @@ +/* + * 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.data.vectorized; + +import java.util.Arrays; +import java.util.function.Predicate; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarBatchRow; + +public class ColumnarBatchUtil { + + private ColumnarBatchUtil() {} + + /** + * Builds a row ID mapping inside a batch to skip deleted rows. + * + *

    +   * Initial state
    +   * Data values: [v0, v1, v2, v3, v4, v5, v6, v7]
    +   * Row ID mapping: [0, 1, 2, 3, 4, 5, 6, 7]
    +   *
    +   * Apply position deletes
    +   * Position deletes: 2, 6
    +   * Row ID mapping: [0, 1, 3, 4, 5, 7, -, -] (6 live records)
    +   *
    +   * Apply equality deletes
    +   * Equality deletes: v1, v2, v3
    +   * Row ID mapping: [0, 4, 5, 7, -, -, -, -] (4 live records)
    +   * 
    + * + * @param columnVectors the array of column vectors for the batch + * @param deletes the delete filter containing delete information + * @param rowStartPosInBatch the starting position of the row in the batch + * @param batchSize the size of the batch + * @return the mapping array and the number of live rows, or {@code null} if nothing is deleted + */ + public static Pair buildRowIdMapping( + ColumnVector[] columnVectors, + DeleteFilter deletes, + long rowStartPosInBatch, + int batchSize) { + if (deletes == null) { + return null; + } + + PositionDeleteIndex deletedPositions = deletes.deletedRowPositions(); + Predicate eqDeleteFilter = deletes.eqDeletedRowFilter(); + ColumnarBatchRow row = new ColumnarBatchRow(columnVectors); + int[] rowIdMapping = new int[batchSize]; + int liveRowId = 0; + + for (int rowId = 0; rowId < batchSize; rowId++) { + long pos = rowStartPosInBatch + rowId; + row.rowId = rowId; + if (isDeleted(pos, row, deletedPositions, eqDeleteFilter)) { + deletes.incrementDeleteCount(); + } else { + rowIdMapping[liveRowId] = rowId; + liveRowId++; + } + } + + return liveRowId == batchSize ? null : Pair.of(rowIdMapping, liveRowId); + } + + /** + * Builds a boolean array to indicate if a row is deleted or not. + * + *
    +   * Initial state
    +   * Data values: [v0, v1, v2, v3, v4, v5, v6, v7]
    +   * Is deleted array: [F, F, F, F, F, F, F, F]
    +   *
    +   * Apply position deletes
    +   * Position deletes: 2, 6
    +   * Is deleted array: [F, F, T, F, F, F, T, F] (6 live records)
    +   *
    +   * Apply equality deletes
    +   * Equality deletes: v1, v2, v3
    +   * Is deleted array: [F, T, T, T, F, F, T, F] (4 live records)
    +   * 
    + * + * @param columnVectors the array of column vectors for the batch. + * @param deletes the delete filter containing information about which rows should be deleted. + * @param rowStartPosInBatch the starting position of the row in the batch, used to calculate the + * absolute position of the rows in the context of the entire dataset. + * @param batchSize the number of rows in the current batch. + * @return an array of boolean values to indicate if a row is deleted or not + */ + public static boolean[] buildIsDeleted( + ColumnVector[] columnVectors, + DeleteFilter deletes, + long rowStartPosInBatch, + int batchSize) { + boolean[] isDeleted = new boolean[batchSize]; + + if (deletes == null) { + return isDeleted; + } + + PositionDeleteIndex deletedPositions = deletes.deletedRowPositions(); + Predicate eqDeleteFilter = deletes.eqDeletedRowFilter(); + ColumnarBatchRow row = new ColumnarBatchRow(columnVectors); + + for (int rowId = 0; rowId < batchSize; rowId++) { + long pos = rowStartPosInBatch + rowId; + row.rowId = rowId; + if (isDeleted(pos, row, deletedPositions, eqDeleteFilter)) { + deletes.incrementDeleteCount(); + isDeleted[rowId] = true; + } + } + + return isDeleted; + } + + private static boolean isDeleted( + long pos, + InternalRow row, + PositionDeleteIndex deletedPositions, + Predicate eqDeleteFilter) { + // use separate if statements to reduce the chance of speculative execution for equality tests + if (deletedPositions != null && deletedPositions.isDeleted(pos)) { + return true; + } + + if (eqDeleteFilter != null && !eqDeleteFilter.test(row)) { + return true; + } + + return false; + } + + /** + * Removes extra column vectors added for processing equality delete filters that are not part of + * the final query output. + * + *

    During query execution, additional columns may be included in the schema to evaluate + * equality delete filters. For example, if the table schema contains columns C1, C2, C3, C4, and + * C5, and the query is 'SELECT C5 FROM table'. While equality delete filters are applied on C3 + * and C4, the processing schema includes C5, C3, and C4. These extra columns (C3 and C4) are + * needed to identify rows to delete but are not included in the final result. + * + *

    This method removes the extra column vectors from the end of column vectors array, ensuring + * only the expected column vectors remain. + * + * @param deletes the delete filter containing delete information. + * @param columnVectors the array of column vectors representing query result data + * @return a new column vectors array with extra column vectors removed, or the original column + * vectors array if no extra column vectors are found + */ + public static ColumnVector[] removeExtraColumns( + DeleteFilter deletes, ColumnVector[] columnVectors) { + int expectedColumnSize = deletes.expectedSchema().columns().size(); + if (columnVectors.length > expectedColumnSize) { + return Arrays.copyOf(columnVectors, expectedColumnSize); + } else { + return columnVectors; + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java new file mode 100644 index 000000000000..81b7d83a7077 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java @@ -0,0 +1,140 @@ +/* + * 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.data.vectorized; + +import java.io.IOException; +import org.apache.comet.CometSchemaImporter; +import org.apache.comet.parquet.AbstractColumnReader; +import org.apache.comet.parquet.ColumnReader; +import org.apache.comet.parquet.TypeUtil; +import org.apache.comet.parquet.Utils; +import org.apache.comet.shaded.arrow.memory.RootAllocator; +import org.apache.iceberg.parquet.VectorizedReader; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.Types; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.page.PageReader; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.vectorized.ColumnVector; + +class CometColumnReader implements VectorizedReader { + // use the Comet default batch size + public static final int DEFAULT_BATCH_SIZE = 8192; + + private final ColumnDescriptor descriptor; + private final DataType sparkType; + + // The delegated ColumnReader from Comet side + private AbstractColumnReader delegate; + private boolean initialized = false; + private int batchSize = DEFAULT_BATCH_SIZE; + private CometSchemaImporter importer; + + CometColumnReader(DataType sparkType, ColumnDescriptor descriptor) { + this.sparkType = sparkType; + this.descriptor = descriptor; + } + + CometColumnReader(Types.NestedField field) { + DataType dataType = SparkSchemaUtil.convert(field.type()); + StructField structField = new StructField(field.name(), dataType, false, Metadata.empty()); + this.sparkType = dataType; + this.descriptor = TypeUtil.convertToParquet(structField); + } + + public AbstractColumnReader delegate() { + return delegate; + } + + void setDelegate(AbstractColumnReader delegate) { + this.delegate = delegate; + } + + void setInitialized(boolean initialized) { + this.initialized = initialized; + } + + public int batchSize() { + return batchSize; + } + + /** + * This method is to initialized/reset the CometColumnReader. This needs to be called for each row + * group after readNextRowGroup, so a new dictionary encoding can be set for each of the new row + * groups. + */ + public void reset() { + if (importer != null) { + importer.close(); + } + + if (delegate != null) { + delegate.close(); + } + + this.importer = new CometSchemaImporter(new RootAllocator()); + this.delegate = Utils.getColumnReader(sparkType, descriptor, importer, batchSize, false, false); + this.initialized = true; + } + + public ColumnDescriptor descriptor() { + return descriptor; + } + + /** Returns the Spark data type for this column. */ + public DataType sparkType() { + return sparkType; + } + + /** + * Set the page reader to be 'pageReader'. + * + *

    NOTE: this should be called before reading a new Parquet column chunk, and after {@link + * CometColumnReader#reset} is called. + */ + public void setPageReader(PageReader pageReader) throws IOException { + Preconditions.checkState(initialized, "Invalid state: 'reset' should be called first"); + ((ColumnReader) delegate).setPageReader(pageReader); + } + + @Override + public void close() { + // close resources on native side + if (importer != null) { + importer.close(); + } + + if (delegate != null) { + delegate.close(); + } + } + + @Override + public void setBatchSize(int size) { + this.batchSize = size; + } + + @Override + public ColumnVector read(ColumnVector reuse, int numRowsToRead) { + throw new UnsupportedOperationException("Not supported"); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java new file mode 100644 index 000000000000..3d3e9aca24de --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java @@ -0,0 +1,137 @@ +/* + * 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.data.vectorized; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import org.apache.comet.parquet.AbstractColumnReader; +import org.apache.comet.parquet.BatchReader; +import org.apache.iceberg.Schema; +import org.apache.iceberg.parquet.VectorizedReader; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.parquet.column.page.PageReadStore; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +/** + * {@link VectorizedReader} that returns Spark's {@link ColumnarBatch} to support Spark's vectorized + * read path. The {@link ColumnarBatch} returned is created by passing in the Arrow vectors + * populated via delegated read calls to {@link CometColumnReader VectorReader(s)}. + */ +@SuppressWarnings("checkstyle:VisibilityModifier") +class CometColumnarBatchReader implements VectorizedReader { + + private final CometColumnReader[] readers; + + // The delegated BatchReader on the Comet side does the real work of loading a batch of rows. + // The Comet BatchReader contains an array of ColumnReader. There is no need to explicitly call + // ColumnReader.readBatch; instead, BatchReader.nextBatch will be called, which underneath calls + // ColumnReader.readBatch. The only exception is DeleteColumnReader, because at the time of + // calling BatchReader.nextBatch, the isDeleted value is not yet available, so + // DeleteColumnReader.readBatch must be called explicitly later, after the isDeleted value is + // available. + private final BatchReader delegate; + + CometColumnarBatchReader(List> readers, Schema schema) { + this.readers = + readers.stream().map(CometColumnReader.class::cast).toArray(CometColumnReader[]::new); + + AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; + this.delegate = new BatchReader(abstractColumnReaders); + delegate.setSparkSchema(SparkSchemaUtil.convert(schema)); + } + + @Override + public void setRowGroupInfo( + PageReadStore pageStore, Map metaData) { + for (int i = 0; i < readers.length; i++) { + try { + if (!(readers[i] instanceof CometConstantColumnReader) + && !(readers[i] instanceof CometPositionColumnReader) + && !(readers[i] instanceof CometDeleteColumnReader)) { + readers[i].reset(); + readers[i].setPageReader(pageStore.getPageReader(readers[i].descriptor())); + } + } catch (IOException e) { + throw new UncheckedIOException("Failed to setRowGroupInfo for Comet vectorization", e); + } + } + + for (int i = 0; i < readers.length; i++) { + delegate.getColumnReaders()[i] = this.readers[i].delegate(); + } + } + + @Override + public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) { + return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); + } + + @Override + public void setBatchSize(int batchSize) { + for (CometColumnReader reader : readers) { + if (reader != null) { + reader.setBatchSize(batchSize); + } + } + } + + @Override + public void close() { + for (CometColumnReader reader : readers) { + if (reader != null) { + reader.close(); + } + } + } + + private class ColumnBatchLoader { + private final int batchSize; + + ColumnBatchLoader(int numRowsToRead) { + Preconditions.checkArgument( + numRowsToRead > 0, "Invalid number of rows to read: %s", numRowsToRead); + this.batchSize = numRowsToRead; + } + + ColumnarBatch loadDataToColumnBatch() { + ColumnVector[] vectors = readDataToColumnVectors(); + + ColumnarBatch batch = new ColumnarBatch(vectors); + batch.setNumRows(batchSize); + return batch; + } + + ColumnVector[] readDataToColumnVectors() { + ColumnVector[] columnVectors = new ColumnVector[readers.length]; + // Fetch rows for all readers in the delegate + delegate.nextBatch(batchSize); + for (int i = 0; i < readers.length; i++) { + columnVectors[i] = readers[i].delegate().currentBatch(); + } + + return columnVectors; + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java new file mode 100644 index 000000000000..047c96314b13 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java @@ -0,0 +1,65 @@ +/* + * 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.data.vectorized; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import org.apache.comet.parquet.ConstantColumnReader; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.unsafe.types.UTF8String; + +class CometConstantColumnReader extends CometColumnReader { + + CometConstantColumnReader(T value, Types.NestedField field) { + super(field); + // use delegate to set constant value on the native side to be consumed by native execution. + setDelegate( + new ConstantColumnReader(sparkType(), descriptor(), convertToSparkValue(value), false)); + } + + @Override + public void setBatchSize(int batchSize) { + super.setBatchSize(batchSize); + delegate().setBatchSize(batchSize); + setInitialized(true); + } + + private Object convertToSparkValue(T value) { + DataType dataType = sparkType(); + // Match the value to Spark internal type if necessary + if (dataType == DataTypes.StringType && value instanceof String) { + // the internal type for StringType is UTF8String + return UTF8String.fromString((String) value); + } else if (dataType instanceof DecimalType && value instanceof BigDecimal) { + // the internal type for DecimalType is Decimal + return Decimal.apply((BigDecimal) value); + } else if (dataType == DataTypes.BinaryType && value instanceof ByteBuffer) { + // the internal type for DecimalType is byte[] + // Iceberg default value should always use HeapBufferBuffer, so calling ByteBuffer.array() + // should be safe. + return ((ByteBuffer) value).array(); + } else { + return value; + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java new file mode 100644 index 000000000000..26219014f777 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java @@ -0,0 +1,74 @@ +/* + * 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.data.vectorized; + +import org.apache.comet.parquet.MetadataColumnReader; +import org.apache.comet.parquet.Native; +import org.apache.comet.parquet.TypeUtil; +import org.apache.comet.vector.CometVector; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; + +class CometDeleteColumnReader extends CometColumnReader { + CometDeleteColumnReader(Types.NestedField field) { + super(field); + setDelegate(new DeleteColumnReader()); + } + + @Override + public void setBatchSize(int batchSize) { + super.setBatchSize(batchSize); + delegate().setBatchSize(batchSize); + setInitialized(true); + } + + private static class DeleteColumnReader extends MetadataColumnReader { + private final CometDeletedColumnVector deletedVector; + + DeleteColumnReader() { + this(new boolean[0]); + } + + DeleteColumnReader(boolean[] isDeleted) { + super( + DataTypes.BooleanType, + TypeUtil.convertToParquet( + new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), + false /* useDecimal128 = false */, + false /* isConstant = false */); + this.deletedVector = new CometDeletedColumnVector(isDeleted); + } + + @Override + public void readBatch(int total) { + Native.resetBatch(nativeHandle); + // set isDeleted on the native side to be consumed by native execution + Native.setIsDeleted(nativeHandle, deletedVector.isDeleted()); + + super.readBatch(total); + } + + @Override + public CometVector currentBatch() { + return deletedVector; + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java new file mode 100644 index 000000000000..5817f2c20a4f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java @@ -0,0 +1,155 @@ +/* + * 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.data.vectorized; + +import org.apache.comet.shaded.arrow.vector.ValueVector; +import org.apache.comet.vector.CometVector; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +public class CometDeletedColumnVector extends CometVector implements UpdatableDeletedColumnVector { + private boolean[] isDeleted; + + public CometDeletedColumnVector(boolean[] isDeleted) { + super(SparkSchemaUtil.convert(Types.BooleanType.get()), false); + this.isDeleted = isDeleted; + } + + @Override + public void setValue(boolean[] deleted) { + this.isDeleted = deleted; + } + + boolean[] isDeleted() { + return isDeleted; + } + + @Override + public void setNumNulls(int numNulls) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void setNumValues(int numValues) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public int numValues() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public ValueVector getValueVector() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public CometVector slice(int offset, int length) { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public void close() {} + + @Override + public boolean hasNull() { + return false; + } + + @Override + public int numNulls() { + return 0; + } + + @Override + public boolean isNullAt(int rowId) { + return false; + } + + @Override + public boolean getBoolean(int rowId) { + return isDeleted[rowId]; + } + + @Override + public byte getByte(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public short getShort(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public int getInt(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public long getLong(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloat(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public double getDouble(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarArray getArray(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarMap getMap(int ordinal) { + throw new UnsupportedOperationException(); + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + throw new UnsupportedOperationException(); + } + + @Override + public UTF8String getUTF8String(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] getBinary(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnVector getChild(int ordinal) { + throw new UnsupportedOperationException(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java new file mode 100644 index 000000000000..bcc0e514c28d --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java @@ -0,0 +1,62 @@ +/* + * 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.data.vectorized; + +import org.apache.comet.parquet.MetadataColumnReader; +import org.apache.comet.parquet.Native; +import org.apache.iceberg.types.Types; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.spark.sql.types.DataTypes; + +class CometPositionColumnReader extends CometColumnReader { + CometPositionColumnReader(Types.NestedField field) { + super(field); + setDelegate(new PositionColumnReader(descriptor())); + } + + @Override + public void setBatchSize(int batchSize) { + super.setBatchSize(batchSize); + delegate().setBatchSize(batchSize); + setInitialized(true); + } + + private static class PositionColumnReader extends MetadataColumnReader { + /** The current position value of the column that are used to initialize this column reader. */ + private long position; + + PositionColumnReader(ColumnDescriptor descriptor) { + super( + DataTypes.LongType, + descriptor, + false /* useDecimal128 = false */, + false /* isConstant = false */); + } + + @Override + public void readBatch(int total) { + Native.resetBatch(nativeHandle); + // set position on the native side to be consumed by native execution + Native.setPosition(nativeHandle, position, total); + position += total; + + super.readBatch(total); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java new file mode 100644 index 000000000000..779dc240d4f6 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java @@ -0,0 +1,138 @@ +/* + * 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.data.vectorized; + +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.IntStream; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.parquet.TypeWithSchemaVisitor; +import org.apache.iceberg.parquet.VectorizedReader; +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.SparkSchemaUtil; +import org.apache.iceberg.types.Types; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor> { + + private final MessageType parquetSchema; + private final Schema icebergSchema; + private final Map idToConstant; + private final Function>, VectorizedReader> readerFactory; + + CometVectorizedReaderBuilder( + Schema expectedSchema, + MessageType parquetSchema, + Map idToConstant, + Function>, VectorizedReader> readerFactory) { + this.parquetSchema = parquetSchema; + this.icebergSchema = expectedSchema; + this.idToConstant = idToConstant; + this.readerFactory = readerFactory; + } + + @Override + public VectorizedReader message( + Types.StructType expected, MessageType message, List> fieldReaders) { + GroupType groupType = message.asGroupType(); + Map> readersById = Maps.newHashMap(); + List fields = groupType.getFields(); + + IntStream.range(0, fields.size()) + .filter(pos -> fields.get(pos).getId() != null) + .forEach(pos -> readersById.put(fields.get(pos).getId().intValue(), fieldReaders.get(pos))); + + List icebergFields = + expected != null ? expected.fields() : ImmutableList.of(); + + List> reorderedFields = + Lists.newArrayListWithExpectedSize(icebergFields.size()); + + for (Types.NestedField field : icebergFields) { + int id = field.fieldId(); + VectorizedReader reader = readersById.get(id); + if (idToConstant.containsKey(id)) { + CometConstantColumnReader constantReader = + new CometConstantColumnReader<>(idToConstant.get(id), field); + reorderedFields.add(constantReader); + } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { + reorderedFields.add(new CometPositionColumnReader(field)); + } else if (id == MetadataColumns.IS_DELETED.fieldId()) { + CometColumnReader deleteReader = new CometDeleteColumnReader<>(field); + reorderedFields.add(deleteReader); + } else if (reader != null) { + reorderedFields.add(reader); + } else if (field.initialDefault() != null) { + CometColumnReader constantReader = + new CometConstantColumnReader<>(field.initialDefault(), field); + reorderedFields.add(constantReader); + } else if (field.isOptional()) { + CometColumnReader constantReader = new CometConstantColumnReader<>(null, field); + reorderedFields.add(constantReader); + } else { + throw new IllegalArgumentException( + String.format("Missing required field: %s", field.name())); + } + } + return vectorizedReader(reorderedFields); + } + + protected VectorizedReader vectorizedReader(List> reorderedFields) { + return readerFactory.apply(reorderedFields); + } + + @Override + public VectorizedReader struct( + Types.StructType expected, GroupType groupType, List> fieldReaders) { + if (expected != null) { + throw new UnsupportedOperationException( + "Vectorized reads are not supported yet for struct fields"); + } + return null; + } + + @Override + public VectorizedReader primitive( + org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { + + if (primitive.getId() == null) { + return null; + } + int parquetFieldId = primitive.getId().intValue(); + ColumnDescriptor desc = parquetSchema.getColumnDescription(currentPath()); + // Nested types not yet supported for vectorized reads + if (desc.getMaxRepetitionLevel() > 0) { + return null; + } + Types.NestedField icebergField = icebergSchema.findField(parquetFieldId); + if (icebergField == null) { + return null; + } + + return new CometColumnReader(SparkSchemaUtil.convert(icebergField.type()), desc); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java new file mode 100644 index 000000000000..1398a137c1c0 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ConstantColumnVector.java @@ -0,0 +1,141 @@ +/* + * 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.data.vectorized; + +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +class ConstantColumnVector extends ColumnVector { + + private final Type icebergType; + private final Object constant; + private final int batchSize; + + ConstantColumnVector(Type icebergType, int batchSize, Object constant) { + // the type may be unknown for NULL vectors + super(icebergType != null ? SparkSchemaUtil.convert(icebergType) : null); + this.icebergType = icebergType; + this.constant = constant; + this.batchSize = batchSize; + } + + @Override + public void close() {} + + @Override + public boolean hasNull() { + return constant == null; + } + + @Override + public int numNulls() { + return constant == null ? batchSize : 0; + } + + @Override + public boolean isNullAt(int rowId) { + return constant == null; + } + + @Override + public boolean getBoolean(int rowId) { + return (boolean) constant; + } + + @Override + public byte getByte(int rowId) { + return (byte) constant; + } + + @Override + public short getShort(int rowId) { + return (short) constant; + } + + @Override + public int getInt(int rowId) { + return (int) constant; + } + + @Override + public long getLong(int rowId) { + return (long) constant; + } + + @Override + public float getFloat(int rowId) { + return (float) constant; + } + + @Override + public double getDouble(int rowId) { + return (double) constant; + } + + @Override + public ColumnarArray getArray(int rowId) { + throw new UnsupportedOperationException(this.getClass() + " does not implement getArray"); + } + + @Override + public ColumnarMap getMap(int ordinal) { + throw new UnsupportedOperationException(this.getClass() + " does not implement getMap"); + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + return (Decimal) constant; + } + + @Override + public UTF8String getUTF8String(int rowId) { + return (UTF8String) constant; + } + + @Override + public byte[] getBinary(int rowId) { + return (byte[]) constant; + } + + @Override + public ColumnVector getChild(int ordinal) { + InternalRow constantAsRow = (InternalRow) constant; + Object childConstant = constantAsRow.get(ordinal, childType(ordinal)); + return new ConstantColumnVector(childIcebergType(ordinal), batchSize, childConstant); + } + + private Type childIcebergType(int ordinal) { + Types.StructType icebergTypeAsStruct = (Types.StructType) icebergType; + return icebergTypeAsStruct.fields().get(ordinal).type(); + } + + private DataType childType(int ordinal) { + StructType typeAsStruct = (StructType) type; + return typeAsStruct.fields()[ordinal].dataType(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java new file mode 100644 index 000000000000..fa3bcfdd004e --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/DeletedColumnVector.java @@ -0,0 +1,123 @@ +/* + * 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.data.vectorized; + +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.Type; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +public class DeletedColumnVector extends ColumnVector implements UpdatableDeletedColumnVector { + private boolean[] isDeleted; + + public DeletedColumnVector(Type type) { + super(SparkSchemaUtil.convert(type)); + } + + @Override + public void setValue(boolean[] deleted) { + this.isDeleted = deleted; + } + + @Override + public void close() {} + + @Override + public boolean hasNull() { + return false; + } + + @Override + public int numNulls() { + return 0; + } + + @Override + public boolean isNullAt(int rowId) { + return false; + } + + @Override + public boolean getBoolean(int rowId) { + return isDeleted[rowId]; + } + + @Override + public byte getByte(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public short getShort(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public int getInt(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public long getLong(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloat(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public double getDouble(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarArray getArray(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarMap getMap(int ordinal) { + throw new UnsupportedOperationException(); + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + throw new UnsupportedOperationException(); + } + + @Override + public UTF8String getUTF8String(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] getBinary(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnVector getChild(int ordinal) { + throw new UnsupportedOperationException(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java new file mode 100644 index 000000000000..bf6de0e1391f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/IcebergArrowColumnVector.java @@ -0,0 +1,164 @@ +/* + * 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.data.vectorized; + +import org.apache.iceberg.arrow.vectorized.ArrowVectorAccessor; +import org.apache.iceberg.arrow.vectorized.NullabilityHolder; +import org.apache.iceberg.arrow.vectorized.VectorHolder; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.vectorized.ArrowColumnVector; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +/** + * Implementation of Spark's {@link ColumnVector} interface. The code for this class is heavily + * inspired from Spark's {@link ArrowColumnVector} The main difference is in how nullability checks + * are made in this class by relying on {@link NullabilityHolder} instead of the validity vector in + * the Arrow vector. + */ +public class IcebergArrowColumnVector extends ColumnVector { + + private final ArrowVectorAccessor accessor; + private final NullabilityHolder nullabilityHolder; + + public IcebergArrowColumnVector(VectorHolder holder) { + super(SparkSchemaUtil.convert(holder.icebergType())); + this.nullabilityHolder = holder.nullabilityHolder(); + this.accessor = ArrowVectorAccessors.getVectorAccessor(holder); + } + + protected ArrowVectorAccessor accessor() { + return accessor; + } + + protected NullabilityHolder nullabilityHolder() { + return nullabilityHolder; + } + + @Override + public void close() { + accessor.close(); + } + + @Override + public void closeIfFreeable() { + // If a column vector is writable or constant, it should override this method and do nothing. + // See more details at SPARK-50235, SPARK-50463 (Fixed in Spark 3.5.4) + } + + @Override + public boolean hasNull() { + return nullabilityHolder.hasNulls(); + } + + @Override + public int numNulls() { + return nullabilityHolder.numNulls(); + } + + @Override + public boolean isNullAt(int rowId) { + return nullabilityHolder.isNullAt(rowId) == 1; + } + + @Override + public boolean getBoolean(int rowId) { + return accessor.getBoolean(rowId); + } + + @Override + public byte getByte(int rowId) { + throw new UnsupportedOperationException("Unsupported type - byte"); + } + + @Override + public short getShort(int rowId) { + throw new UnsupportedOperationException("Unsupported type - short"); + } + + @Override + public int getInt(int rowId) { + return accessor.getInt(rowId); + } + + @Override + public long getLong(int rowId) { + return accessor.getLong(rowId); + } + + @Override + public float getFloat(int rowId) { + return accessor.getFloat(rowId); + } + + @Override + public double getDouble(int rowId) { + return accessor.getDouble(rowId); + } + + @Override + public ColumnarArray getArray(int rowId) { + if (isNullAt(rowId)) { + return null; + } + return accessor.getArray(rowId); + } + + @Override + public ColumnarMap getMap(int rowId) { + throw new UnsupportedOperationException("Unsupported type - map"); + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + if (isNullAt(rowId)) { + return null; + } + return accessor.getDecimal(rowId, precision, scale); + } + + @Override + public UTF8String getUTF8String(int rowId) { + if (isNullAt(rowId)) { + return null; + } + return accessor.getUTF8String(rowId); + } + + @Override + public byte[] getBinary(int rowId) { + if (isNullAt(rowId)) { + return null; + } + return accessor.getBinary(rowId); + } + + @Override + public ArrowColumnVector getChild(int ordinal) { + return accessor.childColumn(ordinal); + } + + public ArrowVectorAccessor + vectorAccessor() { + return accessor; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/RowPositionColumnVector.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/RowPositionColumnVector.java new file mode 100644 index 000000000000..a389cd8286e5 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/RowPositionColumnVector.java @@ -0,0 +1,120 @@ +/* + * 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.data.vectorized; + +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +public class RowPositionColumnVector extends ColumnVector { + + private final long batchOffsetInFile; + + RowPositionColumnVector(long batchOffsetInFile) { + super(SparkSchemaUtil.convert(Types.LongType.get())); + this.batchOffsetInFile = batchOffsetInFile; + } + + @Override + public void close() {} + + @Override + public boolean hasNull() { + return false; + } + + @Override + public int numNulls() { + return 0; + } + + @Override + public boolean isNullAt(int rowId) { + return false; + } + + @Override + public boolean getBoolean(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public byte getByte(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public short getShort(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public int getInt(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public long getLong(int rowId) { + return batchOffsetInFile + rowId; + } + + @Override + public float getFloat(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public double getDouble(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarArray getArray(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarMap getMap(int ordinal) { + throw new UnsupportedOperationException(); + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + throw new UnsupportedOperationException(); + } + + @Override + public UTF8String getUTF8String(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] getBinary(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnVector getChild(int ordinal) { + throw new UnsupportedOperationException(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java new file mode 100644 index 000000000000..99bedc42bf3c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/UpdatableDeletedColumnVector.java @@ -0,0 +1,23 @@ +/* + * 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.data.vectorized; + +public interface UpdatableDeletedColumnVector { + void setValue(boolean[] isDeleted); +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java new file mode 100644 index 000000000000..4f324239881e --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java @@ -0,0 +1,492 @@ +/* + * 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.data.vectorized; + +import java.util.List; +import java.util.Map; +import java.util.stream.IntStream; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.orc.OrcBatchReader; +import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor; +import org.apache.iceberg.orc.OrcValueReader; +import org.apache.iceberg.orc.OrcValueReaders; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.data.SparkOrcValueReaders; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.orc.TypeDescription; +import org.apache.orc.storage.ql.exec.vector.ListColumnVector; +import org.apache.orc.storage.ql.exec.vector.MapColumnVector; +import org.apache.orc.storage.ql.exec.vector.StructColumnVector; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +public class VectorizedSparkOrcReaders { + + private VectorizedSparkOrcReaders() {} + + public static OrcBatchReader buildReader( + Schema expectedSchema, TypeDescription fileSchema, Map idToConstant) { + Converter converter = + OrcSchemaWithTypeVisitor.visit(expectedSchema, fileSchema, new ReadBuilder(idToConstant)); + + return new OrcBatchReader() { + private long batchOffsetInFile; + + @Override + public ColumnarBatch read(VectorizedRowBatch batch) { + BaseOrcColumnVector cv = + (BaseOrcColumnVector) + converter.convert( + new StructColumnVector(batch.size, batch.cols), + batch.size, + batchOffsetInFile, + batch.selectedInUse, + batch.selected); + ColumnarBatch columnarBatch = + new ColumnarBatch( + IntStream.range(0, expectedSchema.columns().size()) + .mapToObj(cv::getChild) + .toArray(ColumnVector[]::new)); + columnarBatch.setNumRows(batch.size); + return columnarBatch; + } + + @Override + public void setBatchContext(long batchOffsetInFile) { + this.batchOffsetInFile = batchOffsetInFile; + } + }; + } + + private interface Converter { + ColumnVector convert( + org.apache.orc.storage.ql.exec.vector.ColumnVector columnVector, + int batchSize, + long batchOffsetInFile, + boolean isSelectedInUse, + int[] selected); + } + + private static class ReadBuilder extends OrcSchemaWithTypeVisitor { + private final Map idToConstant; + + private ReadBuilder(Map idToConstant) { + this.idToConstant = idToConstant; + } + + @Override + public Converter record( + Types.StructType iStruct, + TypeDescription record, + List names, + List fields) { + return new StructConverter(iStruct, fields, idToConstant); + } + + @Override + public Converter list(Types.ListType iList, TypeDescription array, Converter element) { + return new ArrayConverter(iList, element); + } + + @Override + public Converter map(Types.MapType iMap, TypeDescription map, Converter key, Converter value) { + return new MapConverter(iMap, key, value); + } + + @Override + public Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) { + final OrcValueReader primitiveValueReader; + switch (primitive.getCategory()) { + case BOOLEAN: + primitiveValueReader = OrcValueReaders.booleans(); + break; + case BYTE: + // Iceberg does not have a byte type. Use int + case SHORT: + // Iceberg does not have a short type. Use int + case DATE: + case INT: + primitiveValueReader = OrcValueReaders.ints(); + break; + case LONG: + primitiveValueReader = OrcValueReaders.longs(); + break; + case FLOAT: + primitiveValueReader = OrcValueReaders.floats(); + break; + case DOUBLE: + primitiveValueReader = OrcValueReaders.doubles(); + break; + case TIMESTAMP_INSTANT: + case TIMESTAMP: + primitiveValueReader = SparkOrcValueReaders.timestampTzs(); + break; + case DECIMAL: + primitiveValueReader = + SparkOrcValueReaders.decimals(primitive.getPrecision(), primitive.getScale()); + break; + case CHAR: + case VARCHAR: + case STRING: + primitiveValueReader = SparkOrcValueReaders.utf8String(); + break; + case BINARY: + primitiveValueReader = + Type.TypeID.UUID == iPrimitive.typeId() + ? SparkOrcValueReaders.uuids() + : OrcValueReaders.bytes(); + break; + default: + throw new IllegalArgumentException("Unhandled type " + primitive); + } + return (columnVector, batchSize, batchOffsetInFile, isSelectedInUse, selected) -> + new PrimitiveOrcColumnVector( + iPrimitive, batchSize, columnVector, primitiveValueReader, isSelectedInUse, selected); + } + } + + private abstract static class BaseOrcColumnVector extends ColumnVector { + private final org.apache.orc.storage.ql.exec.vector.ColumnVector vector; + private final int batchSize; + private final boolean isSelectedInUse; + private final int[] selected; + private Integer numNulls; + + BaseOrcColumnVector( + Type type, + int batchSize, + org.apache.orc.storage.ql.exec.vector.ColumnVector vector, + boolean isSelectedInUse, + int[] selected) { + super(SparkSchemaUtil.convert(type)); + this.vector = vector; + this.batchSize = batchSize; + this.isSelectedInUse = isSelectedInUse; + this.selected = selected; + } + + @Override + public void close() {} + + @Override + public boolean hasNull() { + return !vector.noNulls; + } + + @Override + public int numNulls() { + if (numNulls == null) { + numNulls = numNullsHelper(); + } + return numNulls; + } + + private int numNullsHelper() { + if (vector.isRepeating) { + if (vector.isNull[0]) { + return batchSize; + } else { + return 0; + } + } else if (vector.noNulls) { + return 0; + } else { + int count = 0; + for (int i = 0; i < batchSize; i++) { + if (vector.isNull[i]) { + count++; + } + } + return count; + } + } + + protected int getRowIndex(int rowId) { + int row = isSelectedInUse ? selected[rowId] : rowId; + return vector.isRepeating ? 0 : row; + } + + @Override + public boolean isNullAt(int rowId) { + return vector.isNull[getRowIndex(rowId)]; + } + + @Override + public boolean getBoolean(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public byte getByte(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public short getShort(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public int getInt(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public long getLong(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public float getFloat(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public double getDouble(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + throw new UnsupportedOperationException(); + } + + @Override + public UTF8String getUTF8String(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public byte[] getBinary(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarArray getArray(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarMap getMap(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnVector getChild(int ordinal) { + throw new UnsupportedOperationException(); + } + } + + private static class PrimitiveOrcColumnVector extends BaseOrcColumnVector { + private final org.apache.orc.storage.ql.exec.vector.ColumnVector vector; + private final OrcValueReader primitiveValueReader; + + PrimitiveOrcColumnVector( + Type type, + int batchSize, + org.apache.orc.storage.ql.exec.vector.ColumnVector vector, + OrcValueReader primitiveValueReader, + boolean isSelectedInUse, + int[] selected) { + super(type, batchSize, vector, isSelectedInUse, selected); + this.vector = vector; + this.primitiveValueReader = primitiveValueReader; + } + + @Override + public boolean getBoolean(int rowId) { + return (Boolean) primitiveValueReader.read(vector, getRowIndex(rowId)); + } + + @Override + public int getInt(int rowId) { + return (Integer) primitiveValueReader.read(vector, getRowIndex(rowId)); + } + + @Override + public long getLong(int rowId) { + return (Long) primitiveValueReader.read(vector, getRowIndex(rowId)); + } + + @Override + public float getFloat(int rowId) { + return (Float) primitiveValueReader.read(vector, getRowIndex(rowId)); + } + + @Override + public double getDouble(int rowId) { + return (Double) primitiveValueReader.read(vector, getRowIndex(rowId)); + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + // TODO: Is it okay to assume that (precision,scale) parameters == (precision,scale) of the + // decimal type + // and return a Decimal with (precision,scale) of the decimal type? + return (Decimal) primitiveValueReader.read(vector, getRowIndex(rowId)); + } + + @Override + public UTF8String getUTF8String(int rowId) { + return (UTF8String) primitiveValueReader.read(vector, getRowIndex(rowId)); + } + + @Override + public byte[] getBinary(int rowId) { + return (byte[]) primitiveValueReader.read(vector, getRowIndex(rowId)); + } + } + + private static class ArrayConverter implements Converter { + private final Types.ListType listType; + private final Converter elementConverter; + + private ArrayConverter(Types.ListType listType, Converter elementConverter) { + this.listType = listType; + this.elementConverter = elementConverter; + } + + @Override + public ColumnVector convert( + org.apache.orc.storage.ql.exec.vector.ColumnVector vector, + int batchSize, + long batchOffsetInFile, + boolean isSelectedInUse, + int[] selected) { + ListColumnVector listVector = (ListColumnVector) vector; + ColumnVector elementVector = + elementConverter.convert(listVector.child, batchSize, batchOffsetInFile, false, null); + + return new BaseOrcColumnVector(listType, batchSize, vector, isSelectedInUse, selected) { + @Override + public ColumnarArray getArray(int rowId) { + int index = getRowIndex(rowId); + return new ColumnarArray( + elementVector, (int) listVector.offsets[index], (int) listVector.lengths[index]); + } + }; + } + } + + private static class MapConverter implements Converter { + private final Types.MapType mapType; + private final Converter keyConverter; + private final Converter valueConverter; + + private MapConverter(Types.MapType mapType, Converter keyConverter, Converter valueConverter) { + this.mapType = mapType; + this.keyConverter = keyConverter; + this.valueConverter = valueConverter; + } + + @Override + public ColumnVector convert( + org.apache.orc.storage.ql.exec.vector.ColumnVector vector, + int batchSize, + long batchOffsetInFile, + boolean isSelectedInUse, + int[] selected) { + MapColumnVector mapVector = (MapColumnVector) vector; + ColumnVector keyVector = + keyConverter.convert(mapVector.keys, batchSize, batchOffsetInFile, false, null); + ColumnVector valueVector = + valueConverter.convert(mapVector.values, batchSize, batchOffsetInFile, false, null); + + return new BaseOrcColumnVector(mapType, batchSize, vector, isSelectedInUse, selected) { + @Override + public ColumnarMap getMap(int rowId) { + int index = getRowIndex(rowId); + return new ColumnarMap( + keyVector, + valueVector, + (int) mapVector.offsets[index], + (int) mapVector.lengths[index]); + } + }; + } + } + + private static class StructConverter implements Converter { + private final Types.StructType structType; + private final List fieldConverters; + private final Map idToConstant; + + private StructConverter( + Types.StructType structType, + List fieldConverters, + Map idToConstant) { + this.structType = structType; + this.fieldConverters = fieldConverters; + this.idToConstant = idToConstant; + } + + @Override + public ColumnVector convert( + org.apache.orc.storage.ql.exec.vector.ColumnVector vector, + int batchSize, + long batchOffsetInFile, + boolean isSelectedInUse, + int[] selected) { + StructColumnVector structVector = (StructColumnVector) vector; + List fields = structType.fields(); + List fieldVectors = Lists.newArrayListWithExpectedSize(fields.size()); + for (int pos = 0, vectorIndex = 0; pos < fields.size(); pos += 1) { + Types.NestedField field = fields.get(pos); + if (idToConstant.containsKey(field.fieldId())) { + fieldVectors.add( + new ConstantColumnVector(field.type(), batchSize, idToConstant.get(field.fieldId()))); + } else if (field.equals(MetadataColumns.ROW_POSITION)) { + fieldVectors.add(new RowPositionColumnVector(batchOffsetInFile)); + } else if (field.equals(MetadataColumns.IS_DELETED)) { + DeletedColumnVector deletedVector = new DeletedColumnVector(field.type()); + deletedVector.setValue(new boolean[batchSize]); + fieldVectors.add(deletedVector); + } else if (field.type().equals(Types.UnknownType.get())) { + fieldVectors.add(new ConstantColumnVector(field.type(), batchSize, null)); + } else { + fieldVectors.add( + fieldConverters + .get(vectorIndex) + .convert( + structVector.fields[vectorIndex], + batchSize, + batchOffsetInFile, + isSelectedInUse, + selected)); + vectorIndex++; + } + } + + return new BaseOrcColumnVector(structType, batchSize, vector, isSelectedInUse, selected) { + @Override + public ColumnVector getChild(int ordinal) { + return fieldVectors.get(ordinal); + } + }; + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java new file mode 100644 index 000000000000..8e25e81a05b2 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -0,0 +1,143 @@ +/* + * 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.data.vectorized; + +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.vector.NullCheckingForGet; +import org.apache.iceberg.Schema; +import org.apache.iceberg.arrow.ArrowAllocation; +import org.apache.iceberg.arrow.vectorized.VectorizedReaderBuilder; +import org.apache.iceberg.parquet.TypeWithSchemaVisitor; +import org.apache.iceberg.parquet.VectorizedReader; +import org.apache.iceberg.spark.SparkUtil; +import org.apache.parquet.schema.MessageType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class VectorizedSparkParquetReaders { + + private static final Logger LOG = LoggerFactory.getLogger(VectorizedSparkParquetReaders.class); + private static final String ENABLE_UNSAFE_MEMORY_ACCESS = "arrow.enable_unsafe_memory_access"; + private static final String ENABLE_UNSAFE_MEMORY_ACCESS_ENV = "ARROW_ENABLE_UNSAFE_MEMORY_ACCESS"; + private static final String ENABLE_NULL_CHECK_FOR_GET = "arrow.enable_null_check_for_get"; + private static final String ENABLE_NULL_CHECK_FOR_GET_ENV = "ARROW_ENABLE_NULL_CHECK_FOR_GET"; + + static { + try { + enableUnsafeMemoryAccess(); + disableNullCheckForGet(); + } catch (Exception e) { + LOG.warn("Couldn't set Arrow properties, which may impact read performance", e); + } + } + + private VectorizedSparkParquetReaders() {} + + public static ColumnarBatchReader buildReader( + Schema expectedSchema, + MessageType fileSchema, + Map idToConstant, + BufferAllocator bufferAllocator) { + return (ColumnarBatchReader) + TypeWithSchemaVisitor.visit( + expectedSchema.asStruct(), + fileSchema, + new ReaderBuilder( + expectedSchema, + fileSchema, + NullCheckingForGet.NULL_CHECKING_ENABLED, + idToConstant, + ColumnarBatchReader::new, + bufferAllocator)); + } + + public static ColumnarBatchReader buildReader( + Schema expectedSchema, MessageType fileSchema, Map idToConstant) { + return buildReader(expectedSchema, fileSchema, idToConstant, ArrowAllocation.rootAllocator()); + } + + public static CometColumnarBatchReader buildCometReader( + Schema expectedSchema, MessageType fileSchema, Map idToConstant) { + return (CometColumnarBatchReader) + TypeWithSchemaVisitor.visit( + expectedSchema.asStruct(), + fileSchema, + new CometVectorizedReaderBuilder( + expectedSchema, + fileSchema, + idToConstant, + readers -> new CometColumnarBatchReader(readers, expectedSchema))); + } + + // enables unsafe memory access to avoid costly checks to see if index is within bounds + // as long as it is not configured explicitly (see BoundsChecking in Arrow) + private static void enableUnsafeMemoryAccess() { + String value = confValue(ENABLE_UNSAFE_MEMORY_ACCESS, ENABLE_UNSAFE_MEMORY_ACCESS_ENV); + if (value == null) { + LOG.info("Enabling {}", ENABLE_UNSAFE_MEMORY_ACCESS); + System.setProperty(ENABLE_UNSAFE_MEMORY_ACCESS, "true"); + } else { + LOG.info("Unsafe memory access was configured explicitly: {}", value); + } + } + + // disables expensive null checks for every get call in favor of Iceberg nullability + // as long as it is not configured explicitly (see NullCheckingForGet in Arrow) + private static void disableNullCheckForGet() { + String value = confValue(ENABLE_NULL_CHECK_FOR_GET, ENABLE_NULL_CHECK_FOR_GET_ENV); + if (value == null) { + LOG.info("Disabling {}", ENABLE_NULL_CHECK_FOR_GET); + System.setProperty(ENABLE_NULL_CHECK_FOR_GET, "false"); + } else { + LOG.info("Null checking for get calls was configured explicitly: {}", value); + } + } + + private static String confValue(String propName, String envName) { + String propValue = System.getProperty(propName); + if (propValue != null) { + return propValue; + } + + return System.getenv(envName); + } + + private static class ReaderBuilder extends VectorizedReaderBuilder { + + ReaderBuilder( + Schema expectedSchema, + MessageType parquetSchema, + boolean setArrowValidityVector, + Map idToConstant, + Function>, VectorizedReader> readerFactory, + BufferAllocator bufferAllocator) { + super( + expectedSchema, + parquetSchema, + setArrowValidityVector, + idToConstant, + readerFactory, + SparkUtil::internalToSpark, + bufferAllocator); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java new file mode 100644 index 000000000000..5ec44f314180 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java @@ -0,0 +1,40 @@ +/* + * 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.functions; + +import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; + +abstract class BaseScalarFunction implements ScalarFunction { + @Override + public int hashCode() { + return canonicalName().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (!(other instanceof ScalarFunction)) { + return false; + } + + ScalarFunction that = (ScalarFunction) other; + return canonicalName().equals(that.canonicalName()); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java new file mode 100644 index 000000000000..f31de7370533 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java @@ -0,0 +1,366 @@ +/* + * 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.functions; + +import java.io.Serializable; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.util.BucketUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.functions.BoundFunction; +import org.apache.spark.sql.connector.catalog.functions.Reducer; +import org.apache.spark.sql.connector.catalog.functions.ReducibleFunction; +import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; +import org.apache.spark.sql.types.BinaryType; +import org.apache.spark.sql.types.ByteType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.DateType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.types.LongType; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.TimestampNTZType; +import org.apache.spark.sql.types.TimestampType; +import org.apache.spark.unsafe.types.UTF8String; + +/** + * A Spark function implementation for the Iceberg bucket transform. + * + *

    Example usage: {@code SELECT system.bucket(128, 'abc')}, which returns the bucket 122. + * + *

    Note that for performance reasons, the given input number of buckets is not validated in the + * implementations used in code-gen. The number of buckets must be positive to give meaningful + * results. + */ +public class BucketFunction implements UnboundFunction { + + private static final int NUM_BUCKETS_ORDINAL = 0; + private static final int VALUE_ORDINAL = 1; + + private static final Set SUPPORTED_NUM_BUCKETS_TYPES = + ImmutableSet.of(DataTypes.ByteType, DataTypes.ShortType, DataTypes.IntegerType); + + @Override + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public BoundFunction bind(StructType inputType) { + if (inputType.size() != 2) { + throw new UnsupportedOperationException( + "Wrong number of inputs (expected numBuckets and value)"); + } + + StructField numBucketsField = inputType.fields()[NUM_BUCKETS_ORDINAL]; + StructField valueField = inputType.fields()[VALUE_ORDINAL]; + + if (!SUPPORTED_NUM_BUCKETS_TYPES.contains(numBucketsField.dataType())) { + throw new UnsupportedOperationException( + "Expected number of buckets to be tinyint, shortint or int"); + } + + DataType type = valueField.dataType(); + if (type instanceof DateType) { + return new BucketInt(type); + } else if (type instanceof ByteType + || type instanceof ShortType + || type instanceof IntegerType) { + return new BucketInt(DataTypes.IntegerType); + } else if (type instanceof LongType) { + return new BucketLong(type); + } else if (type instanceof TimestampType) { + return new BucketLong(type); + } else if (type instanceof TimestampNTZType) { + return new BucketLong(type); + } else if (type instanceof DecimalType) { + return new BucketDecimal(type); + } else if (type instanceof StringType) { + return new BucketString(); + } else if (type instanceof BinaryType) { + return new BucketBinary(); + } else { + throw new UnsupportedOperationException( + "Expected column to be date, tinyint, smallint, int, bigint, decimal, timestamp, string, or binary"); + } + } + + @Override + public String description() { + return name() + + "(numBuckets, col) - Call Iceberg's bucket transform\n" + + " numBuckets :: number of buckets to divide the rows into, e.g. bucket(100, 34) -> 79 (must be a tinyint, smallint, or int)\n" + + " col :: column to bucket (must be a date, integer, long, timestamp, decimal, string, or binary)"; + } + + @Override + public String name() { + return "bucket"; + } + + public abstract static class BucketBase extends BaseScalarFunction + implements ReducibleFunction { + public static int apply(int numBuckets, int hashedValue) { + return (hashedValue & Integer.MAX_VALUE) % numBuckets; + } + + @Override + public String name() { + return "bucket"; + } + + @Override + public DataType resultType() { + return DataTypes.IntegerType; + } + + protected int gcd(int num1, int num2) { + return BigInteger.valueOf(num1).gcd(BigInteger.valueOf(num2)).intValue(); + } + + @Override + public Reducer reducer( + int thisNumBuckets, ReducibleFunction otherBucketFunction, int otherNumBuckets) { + + if (otherBucketFunction instanceof BucketBase) { + int commonDivisor = gcd(thisNumBuckets, otherNumBuckets); + if (commonDivisor > 1 && commonDivisor != thisNumBuckets) { + return new BucketReducer(commonDivisor); + } + } + + return null; + } + } + + // Used for both int and date - tinyint and smallint are upcasted to int by Spark. + public static class BucketInt extends BucketBase { + private final DataType sqlType; + + // magic method used in codegen + public static int invoke(int numBuckets, int value) { + return apply(numBuckets, hash(value)); + } + + // Visible for testing + public static int hash(int value) { + return BucketUtil.hash(value); + } + + public BucketInt(DataType sqlType) { + this.sqlType = sqlType; + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.IntegerType, sqlType}; + } + + @Override + public String canonicalName() { + return String.format("iceberg.bucket(%s)", sqlType.catalogString()); + } + + @Override + public Integer produceResult(InternalRow input) { + // return null for null input to match what Spark does in the code-generated versions. + if (input.isNullAt(NUM_BUCKETS_ORDINAL) || input.isNullAt(VALUE_ORDINAL)) { + return null; + } else { + return invoke(input.getInt(NUM_BUCKETS_ORDINAL), input.getInt(VALUE_ORDINAL)); + } + } + } + + // Used for both BigInt and Timestamp + public static class BucketLong extends BucketBase { + private final DataType sqlType; + + // magic function for usage with codegen - needs to be static + public static int invoke(int numBuckets, long value) { + return apply(numBuckets, hash(value)); + } + + // Visible for testing + public static int hash(long value) { + return BucketUtil.hash(value); + } + + public BucketLong(DataType sqlType) { + this.sqlType = sqlType; + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.IntegerType, sqlType}; + } + + @Override + public String canonicalName() { + return String.format("iceberg.bucket(%s)", sqlType.catalogString()); + } + + @Override + public Integer produceResult(InternalRow input) { + if (input.isNullAt(NUM_BUCKETS_ORDINAL) || input.isNullAt(VALUE_ORDINAL)) { + return null; + } else { + return invoke(input.getInt(NUM_BUCKETS_ORDINAL), input.getLong(VALUE_ORDINAL)); + } + } + } + + public static class BucketString extends BucketBase { + // magic function for usage with codegen + public static Integer invoke(int numBuckets, UTF8String value) { + if (value == null) { + return null; + } + + return apply(numBuckets, hash(value.getBytes())); + } + + // Visible for testing + public static int hash(byte[] value) { + return BucketUtil.hash(value); + } + + // Visible for testing + public static int hash(String value) { + return BucketUtil.hash(value); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.IntegerType, DataTypes.StringType}; + } + + @Override + public String canonicalName() { + return "iceberg.bucket(string)"; + } + + @Override + public Integer produceResult(InternalRow input) { + if (input.isNullAt(NUM_BUCKETS_ORDINAL) || input.isNullAt(VALUE_ORDINAL)) { + return null; + } else { + return invoke(input.getInt(NUM_BUCKETS_ORDINAL), input.getUTF8String(VALUE_ORDINAL)); + } + } + } + + public static class BucketBinary extends BucketBase { + public static Integer invoke(int numBuckets, byte[] value) { + if (value == null) { + return null; + } + + return apply(numBuckets, hash(ByteBuffer.wrap(value))); + } + + // Visible for testing + public static int hash(ByteBuffer value) { + return BucketUtil.hash(value); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.IntegerType, DataTypes.BinaryType}; + } + + @Override + public Integer produceResult(InternalRow input) { + if (input.isNullAt(NUM_BUCKETS_ORDINAL) || input.isNullAt(VALUE_ORDINAL)) { + return null; + } else { + return invoke(input.getInt(NUM_BUCKETS_ORDINAL), input.getBinary(VALUE_ORDINAL)); + } + } + + @Override + public String canonicalName() { + return "iceberg.bucket(binary)"; + } + } + + public static class BucketDecimal extends BucketBase { + private final DataType sqlType; + private final int precision; + private final int scale; + + // magic method used in codegen + public static Integer invoke(int numBuckets, Decimal value) { + if (value == null) { + return null; + } + + return apply(numBuckets, hash(value.toJavaBigDecimal())); + } + + // Visible for testing + public static int hash(BigDecimal value) { + return BucketUtil.hash(value); + } + + public BucketDecimal(DataType sqlType) { + this.sqlType = sqlType; + this.precision = ((DecimalType) sqlType).precision(); + this.scale = ((DecimalType) sqlType).scale(); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.IntegerType, sqlType}; + } + + @Override + public Integer produceResult(InternalRow input) { + if (input.isNullAt(NUM_BUCKETS_ORDINAL) || input.isNullAt(VALUE_ORDINAL)) { + return null; + } else { + int numBuckets = input.getInt(NUM_BUCKETS_ORDINAL); + Decimal value = input.getDecimal(VALUE_ORDINAL, precision, scale); + return invoke(numBuckets, value); + } + } + + @Override + public String canonicalName() { + return "iceberg.bucket(decimal)"; + } + } + + static class BucketReducer implements Reducer, Serializable { + private int commonDivisor; + + BucketReducer(int commonDivisor) { + this.commonDivisor = commonDivisor; + } + + @Override + public Integer reduce(Integer bucketNo) { + return bucketNo % this.commonDivisor; + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java new file mode 100644 index 000000000000..55d4a3755a9d --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.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.functions; + +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.functions.BoundFunction; +import org.apache.spark.sql.connector.catalog.functions.Reducer; +import org.apache.spark.sql.connector.catalog.functions.ReducibleFunction; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.DateType; +import org.apache.spark.sql.types.TimestampNTZType; +import org.apache.spark.sql.types.TimestampType; + +/** + * A Spark function implementation for the Iceberg day transform. + * + *

    Example usage: {@code SELECT system.days('source_col')}. + */ +public class DaysFunction extends UnaryUnboundFunction { + + @Override + protected BoundFunction doBind(DataType valueType) { + if (valueType instanceof DateType) { + return new DateToDaysFunction(); + } else if (valueType instanceof TimestampType) { + return new TimestampToDaysFunction(); + } else if (valueType instanceof TimestampNTZType) { + return new TimestampNtzToDaysFunction(); + } else { + throw new UnsupportedOperationException( + "Expected value to be date or timestamp: " + valueType.catalogString()); + } + } + + @Override + public String description() { + return name() + + "(col) - Call Iceberg's day transform\n" + + " col :: source column (must be date or timestamp)"; + } + + @Override + public String name() { + return "days"; + } + + protected abstract static class BaseToDaysFunction extends BaseScalarFunction + implements ReducibleFunction { + @Override + public String name() { + return "days"; + } + + @Override + public DataType resultType() { + return DataTypes.DateType; + } + + @Override + public Reducer reducer(ReducibleFunction otherFunction) { + return null; + } + } + + // Spark and Iceberg internal representations of dates match so no transformation is required + public static class DateToDaysFunction extends BaseToDaysFunction { + // magic method used in codegen + public static int invoke(int days) { + return days; + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.DateType}; + } + + @Override + public String canonicalName() { + return "iceberg.days(date)"; + } + + @Override + public Integer produceResult(InternalRow input) { + // return null for null input to match what Spark does in codegen + return input.isNullAt(0) ? null : input.getInt(0); + } + } + + public static class TimestampToDaysFunction extends BaseToDaysFunction { + // magic method used in codegen + public static int invoke(long micros) { + return DateTimeUtil.microsToDays(micros); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.TimestampType}; + } + + @Override + public String canonicalName() { + return "iceberg.days(timestamp)"; + } + + @Override + public Integer produceResult(InternalRow input) { + // return null for null input to match what Spark does in codegen + return input.isNullAt(0) ? null : invoke(input.getLong(0)); + } + } + + public static class TimestampNtzToDaysFunction extends BaseToDaysFunction { + // magic method used in codegen + public static int invoke(long micros) { + return DateTimeUtil.microsToDays(micros); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.TimestampNTZType}; + } + + @Override + public String canonicalName() { + return "iceberg.days(timestamp_ntz)"; + } + + @Override + public Integer produceResult(InternalRow input) { + // return null for null input to match what Spark does in codegen + return input.isNullAt(0) ? null : invoke(input.getLong(0)); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java new file mode 100644 index 000000000000..3261a908ec95 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java @@ -0,0 +1,146 @@ +/* + * 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.functions; + +import java.io.Serializable; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.functions.BoundFunction; +import org.apache.spark.sql.connector.catalog.functions.Reducer; +import org.apache.spark.sql.connector.catalog.functions.ReducibleFunction; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.TimestampNTZType; +import org.apache.spark.sql.types.TimestampType; + +/** + * A Spark function implementation for the Iceberg hour transform. + * + *

    Example usage: {@code SELECT system.hours('source_col')}. + */ +public class HoursFunction extends UnaryUnboundFunction { + + @Override + protected BoundFunction doBind(DataType valueType) { + if (valueType instanceof TimestampType) { + return new TimestampToHoursFunction(); + } else if (valueType instanceof TimestampNTZType) { + return new TimestampNtzToHoursFunction(); + } else { + throw new UnsupportedOperationException( + "Expected value to be timestamp: " + valueType.catalogString()); + } + } + + @Override + public String description() { + return name() + + "(col) - Call Iceberg's hour transform\n" + + " col :: source column (must be timestamp)"; + } + + @Override + public String name() { + return "hours"; + } + + public abstract static class BaseToHourFunction extends BaseScalarFunction + implements ReducibleFunction { + @Override + public Reducer reducer(ReducibleFunction otherBucketFunction) { + if (otherBucketFunction instanceof DaysFunction.BaseToDaysFunction) { + return new HourToDaysReducer(); + } + return null; + } + } + + public static class TimestampToHoursFunction extends BaseToHourFunction { + // magic method used in codegen + public static int invoke(long micros) { + return DateTimeUtil.microsToHours(micros); + } + + @Override + public String name() { + return "hours"; + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.TimestampType}; + } + + @Override + public DataType resultType() { + return DataTypes.IntegerType; + } + + @Override + public String canonicalName() { + return "iceberg.hours(timestamp)"; + } + + @Override + public Integer produceResult(InternalRow input) { + // return null for null input to match what Spark does in codegen + return input.isNullAt(0) ? null : invoke(input.getLong(0)); + } + } + + public static class TimestampNtzToHoursFunction extends BaseToHourFunction { + // magic method used in codegen + public static int invoke(long micros) { + return DateTimeUtil.microsToHours(micros); + } + + @Override + public String name() { + return "hours"; + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.TimestampNTZType}; + } + + @Override + public DataType resultType() { + return DataTypes.IntegerType; + } + + @Override + public String canonicalName() { + return "iceberg.hours(timestamp_ntz)"; + } + + @Override + public Integer produceResult(InternalRow input) { + // return null for null input to match what Spark does in codegen + return input.isNullAt(0) ? null : invoke(input.getLong(0)); + } + } + + public static class HourToDaysReducer implements Reducer, Serializable { + @Override + public Integer reduce(Integer hour) { + return DateTimeUtil.hoursToDays(hour); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java new file mode 100644 index 000000000000..689a0f4cb4df --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java @@ -0,0 +1,95 @@ +/* + * 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.functions; + +import org.apache.iceberg.IcebergBuild; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.functions.BoundFunction; +import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +/** + * A function for use in SQL that returns the current Iceberg version, e.g. {@code SELECT + * system.iceberg_version()} will return a String such as "0.14.0" or "0.15.0-SNAPSHOT" + */ +public class IcebergVersionFunction implements UnboundFunction { + @Override + public BoundFunction bind(StructType inputType) { + if (inputType.fields().length > 0) { + throw new UnsupportedOperationException( + String.format("Cannot bind: %s does not accept arguments", name())); + } + + return new IcebergVersionFunctionImpl(); + } + + @Override + public String description() { + return name() + " - Returns the runtime Iceberg version"; + } + + @Override + public String name() { + return "iceberg_version"; + } + + // Implementing class cannot be private, otherwise Spark is unable to access the static invoke + // function during code-gen and calling the function fails + static class IcebergVersionFunctionImpl extends BaseScalarFunction { + private static final UTF8String VERSION = UTF8String.fromString(IcebergBuild.version()); + + // magic function used in code-gen. must be named `invoke`. + public static UTF8String invoke() { + return VERSION; + } + + @Override + public DataType[] inputTypes() { + return new DataType[0]; + } + + @Override + public DataType resultType() { + return DataTypes.StringType; + } + + @Override + public boolean isResultNullable() { + return false; + } + + @Override + public String canonicalName() { + return "iceberg." + name(); + } + + @Override + public String name() { + return "iceberg_version"; + } + + @Override + public UTF8String produceResult(InternalRow input) { + return invoke(); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java new file mode 100644 index 000000000000..353d850f86e2 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java @@ -0,0 +1,143 @@ +/* + * 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.functions; + +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.functions.BoundFunction; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.DateType; +import org.apache.spark.sql.types.TimestampNTZType; +import org.apache.spark.sql.types.TimestampType; + +/** + * A Spark function implementation for the Iceberg month transform. + * + *

    Example usage: {@code SELECT system.months('source_col')}. + */ +public class MonthsFunction extends UnaryUnboundFunction { + + @Override + protected BoundFunction doBind(DataType valueType) { + if (valueType instanceof DateType) { + return new DateToMonthsFunction(); + } else if (valueType instanceof TimestampType) { + return new TimestampToMonthsFunction(); + } else if (valueType instanceof TimestampNTZType) { + return new TimestampNtzToMonthsFunction(); + } else { + throw new UnsupportedOperationException( + "Expected value to be date or timestamp: " + valueType.catalogString()); + } + } + + @Override + public String description() { + return name() + + "(col) - Call Iceberg's month transform\n" + + " col :: source column (must be date or timestamp)"; + } + + @Override + public String name() { + return "months"; + } + + private abstract static class BaseToMonthsFunction extends BaseScalarFunction { + @Override + public String name() { + return "months"; + } + + @Override + public DataType resultType() { + return DataTypes.IntegerType; + } + } + + public static class DateToMonthsFunction extends BaseToMonthsFunction { + // magic method used in codegen + public static int invoke(int days) { + return DateTimeUtil.daysToMonths(days); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.DateType}; + } + + @Override + public String canonicalName() { + return "iceberg.months(date)"; + } + + @Override + public Integer produceResult(InternalRow input) { + // return null for null input to match what Spark does in codegen + return input.isNullAt(0) ? null : invoke(input.getInt(0)); + } + } + + public static class TimestampToMonthsFunction extends BaseToMonthsFunction { + // magic method used in codegen + public static int invoke(long micros) { + return DateTimeUtil.microsToMonths(micros); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.TimestampType}; + } + + @Override + public String canonicalName() { + return "iceberg.months(timestamp)"; + } + + @Override + public Integer produceResult(InternalRow input) { + // return null for null input to match what Spark does in codegen + return input.isNullAt(0) ? null : invoke(input.getLong(0)); + } + } + + public static class TimestampNtzToMonthsFunction extends BaseToMonthsFunction { + // magic method used in codegen + public static int invoke(long micros) { + return DateTimeUtil.microsToMonths(micros); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.TimestampNTZType}; + } + + @Override + public String canonicalName() { + return "iceberg.months(timestamp_ntz)"; + } + + @Override + public Integer produceResult(InternalRow input) { + // return null for null input to match what Spark does in codegen + return input.isNullAt(0) ? null : invoke(input.getLong(0)); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/SparkFunctions.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/SparkFunctions.java new file mode 100644 index 000000000000..6d9cadec576d --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/SparkFunctions.java @@ -0,0 +1,75 @@ +/* + * 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.functions; + +import java.util.List; +import java.util.Locale; +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; + +public class SparkFunctions { + + private SparkFunctions() {} + + private static final Map FUNCTIONS = + ImmutableMap.of( + "iceberg_version", new IcebergVersionFunction(), + "years", new YearsFunction(), + "months", new MonthsFunction(), + "days", new DaysFunction(), + "hours", new HoursFunction(), + "bucket", new BucketFunction(), + "truncate", new TruncateFunction()); + + private static final Map, UnboundFunction> CLASS_TO_FUNCTIONS = + ImmutableMap.of( + YearsFunction.class, new YearsFunction(), + MonthsFunction.class, new MonthsFunction(), + DaysFunction.class, new DaysFunction(), + HoursFunction.class, new HoursFunction(), + BucketFunction.class, new BucketFunction(), + TruncateFunction.class, new TruncateFunction()); + + private static final List FUNCTION_NAMES = ImmutableList.copyOf(FUNCTIONS.keySet()); + + // Functions that are added to all Iceberg catalogs should be accessed with the `system` + // namespace. They can also be accessed with no namespace at all if qualified with the + // catalog name, e.g. my_hadoop_catalog.iceberg_version(). + // As namespace resolution is handled by those rules in BaseCatalog, a list of names + // alone is returned. + public static List list() { + return FUNCTION_NAMES; + } + + public static UnboundFunction load(String name) { + // function resolution is case-insensitive to match the existing Spark behavior for functions + return FUNCTIONS.get(name.toLowerCase(Locale.ROOT)); + } + + public static UnboundFunction loadFunctionByClass(Class functionClass) { + Class declaringClass = functionClass.getDeclaringClass(); + if (declaringClass == null) { + return null; + } + + return CLASS_TO_FUNCTIONS.get(declaringClass); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java new file mode 100644 index 000000000000..662033718fb1 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java @@ -0,0 +1,357 @@ +/* + * 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.functions; + +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.Locale; +import java.util.Set; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.util.BinaryUtil; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.util.TruncateUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.functions.BoundFunction; +import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; +import org.apache.spark.sql.types.BinaryType; +import org.apache.spark.sql.types.ByteType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.types.LongType; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +/** + * A Spark function implementation for the Iceberg truncate transform. + * + *

    Example usage: {@code SELECT system.truncate(1, 'abc')}, which returns the String 'a'. + * + *

    Note that for performance reasons, the given input width is not validated in the + * implementations used in code-gen. The width must remain non-negative to give meaningful results. + */ +public class TruncateFunction implements UnboundFunction { + + private static final int WIDTH_ORDINAL = 0; + private static final int VALUE_ORDINAL = 1; + + private static final Set SUPPORTED_WIDTH_TYPES = + ImmutableSet.of(DataTypes.ByteType, DataTypes.ShortType, DataTypes.IntegerType); + + @Override + public BoundFunction bind(StructType inputType) { + if (inputType.size() != 2) { + throw new UnsupportedOperationException("Wrong number of inputs (expected width and value)"); + } + + StructField widthField = inputType.fields()[WIDTH_ORDINAL]; + StructField valueField = inputType.fields()[VALUE_ORDINAL]; + + if (!SUPPORTED_WIDTH_TYPES.contains(widthField.dataType())) { + throw new UnsupportedOperationException( + "Expected truncation width to be tinyint, shortint or int"); + } + + DataType valueType = valueField.dataType(); + if (valueType instanceof ByteType) { + return new TruncateTinyInt(); + } else if (valueType instanceof ShortType) { + return new TruncateSmallInt(); + } else if (valueType instanceof IntegerType) { + return new TruncateInt(); + } else if (valueType instanceof LongType) { + return new TruncateBigInt(); + } else if (valueType instanceof DecimalType) { + return new TruncateDecimal( + ((DecimalType) valueType).precision(), ((DecimalType) valueType).scale()); + } else if (valueType instanceof StringType) { + return new TruncateString(); + } else if (valueType instanceof BinaryType) { + return new TruncateBinary(); + } else { + throw new UnsupportedOperationException( + "Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + } + } + + @Override + public String description() { + return name() + + "(width, col) - Call Iceberg's truncate transform\n" + + " width :: width for truncation, e.g. truncate(10, 255) -> 250 (must be an integer)\n" + + " col :: column to truncate (must be an integer, decimal, string, or binary)"; + } + + @Override + public String name() { + return "truncate"; + } + + public abstract static class TruncateBase extends BaseScalarFunction { + @Override + public String name() { + return "truncate"; + } + } + + public static class TruncateTinyInt extends TruncateBase { + public static byte invoke(int width, byte value) { + return TruncateUtil.truncateByte(width, value); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.IntegerType, DataTypes.ByteType}; + } + + @Override + public DataType resultType() { + return DataTypes.ByteType; + } + + @Override + public String canonicalName() { + return "iceberg.truncate(tinyint)"; + } + + @Override + public Byte produceResult(InternalRow input) { + if (input.isNullAt(WIDTH_ORDINAL) || input.isNullAt(VALUE_ORDINAL)) { + return null; + } else { + return invoke(input.getInt(WIDTH_ORDINAL), input.getByte(VALUE_ORDINAL)); + } + } + } + + public static class TruncateSmallInt extends TruncateBase { + // magic method used in codegen + public static short invoke(int width, short value) { + return TruncateUtil.truncateShort(width, value); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.IntegerType, DataTypes.ShortType}; + } + + @Override + public DataType resultType() { + return DataTypes.ShortType; + } + + @Override + public String canonicalName() { + return "iceberg.truncate(smallint)"; + } + + @Override + public Short produceResult(InternalRow input) { + if (input.isNullAt(WIDTH_ORDINAL) || input.isNullAt(VALUE_ORDINAL)) { + return null; + } else { + return invoke(input.getInt(WIDTH_ORDINAL), input.getShort(VALUE_ORDINAL)); + } + } + } + + public static class TruncateInt extends TruncateBase { + // magic method used in codegen + public static int invoke(int width, int value) { + return TruncateUtil.truncateInt(width, value); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.IntegerType, DataTypes.IntegerType}; + } + + @Override + public DataType resultType() { + return DataTypes.IntegerType; + } + + @Override + public String canonicalName() { + return "iceberg.truncate(int)"; + } + + @Override + public Integer produceResult(InternalRow input) { + if (input.isNullAt(WIDTH_ORDINAL) || input.isNullAt(VALUE_ORDINAL)) { + return null; + } else { + return invoke(input.getInt(WIDTH_ORDINAL), input.getInt(VALUE_ORDINAL)); + } + } + } + + public static class TruncateBigInt extends TruncateBase { + // magic function for usage with codegen + public static long invoke(int width, long value) { + return TruncateUtil.truncateLong(width, value); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.IntegerType, DataTypes.LongType}; + } + + @Override + public DataType resultType() { + return DataTypes.LongType; + } + + @Override + public String canonicalName() { + return "iceberg.truncate(bigint)"; + } + + @Override + public Long produceResult(InternalRow input) { + if (input.isNullAt(WIDTH_ORDINAL) || input.isNullAt(VALUE_ORDINAL)) { + return null; + } else { + return invoke(input.getInt(WIDTH_ORDINAL), input.getLong(VALUE_ORDINAL)); + } + } + } + + public static class TruncateString extends TruncateBase { + // magic function for usage with codegen + public static UTF8String invoke(int width, UTF8String value) { + if (value == null) { + return null; + } + + return value.substring(0, width); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.IntegerType, DataTypes.StringType}; + } + + @Override + public DataType resultType() { + return DataTypes.StringType; + } + + @Override + public String canonicalName() { + return "iceberg.truncate(string)"; + } + + @Override + public UTF8String produceResult(InternalRow input) { + if (input.isNullAt(WIDTH_ORDINAL) || input.isNullAt(VALUE_ORDINAL)) { + return null; + } else { + return invoke(input.getInt(WIDTH_ORDINAL), input.getUTF8String(VALUE_ORDINAL)); + } + } + } + + public static class TruncateBinary extends TruncateBase { + // magic method used in codegen + public static byte[] invoke(int width, byte[] value) { + if (value == null) { + return null; + } + + return ByteBuffers.toByteArray( + BinaryUtil.truncateBinaryUnsafe(ByteBuffer.wrap(value), width)); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.IntegerType, DataTypes.BinaryType}; + } + + @Override + public DataType resultType() { + return DataTypes.BinaryType; + } + + @Override + public String canonicalName() { + return "iceberg.truncate(binary)"; + } + + @Override + public byte[] produceResult(InternalRow input) { + if (input.isNullAt(WIDTH_ORDINAL) || input.isNullAt(VALUE_ORDINAL)) { + return null; + } else { + return invoke(input.getInt(WIDTH_ORDINAL), input.getBinary(VALUE_ORDINAL)); + } + } + } + + public static class TruncateDecimal extends TruncateBase { + private final int precision; + private final int scale; + + public TruncateDecimal(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + // magic method used in codegen + public static Decimal invoke(int width, Decimal value) { + if (value == null) { + return null; + } + + return Decimal.apply( + TruncateUtil.truncateDecimal(BigInteger.valueOf(width), value.toJavaBigDecimal())); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.IntegerType, DataTypes.createDecimalType(precision, scale)}; + } + + @Override + public DataType resultType() { + return DataTypes.createDecimalType(precision, scale); + } + + @Override + public String canonicalName() { + return String.format(Locale.ROOT, "iceberg.truncate(decimal(%d,%d))", precision, scale); + } + + @Override + public Decimal produceResult(InternalRow input) { + if (input.isNullAt(WIDTH_ORDINAL) || input.isNullAt(VALUE_ORDINAL)) { + return null; + } else { + int width = input.getInt(WIDTH_ORDINAL); + Decimal value = input.getDecimal(VALUE_ORDINAL, precision, scale); + return invoke(width, value); + } + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/UnaryUnboundFunction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/UnaryUnboundFunction.java new file mode 100644 index 000000000000..9003c68919dc --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/UnaryUnboundFunction.java @@ -0,0 +1,44 @@ +/* + * 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.functions; + +import org.apache.spark.sql.connector.catalog.functions.BoundFunction; +import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.StructType; + +/** An unbound function that accepts only one argument */ +abstract class UnaryUnboundFunction implements UnboundFunction { + + @Override + public BoundFunction bind(StructType inputType) { + DataType valueType = valueType(inputType); + return doBind(valueType); + } + + protected abstract BoundFunction doBind(DataType valueType); + + private DataType valueType(StructType inputType) { + if (inputType.size() != 1) { + throw new UnsupportedOperationException("Wrong number of inputs (expected value)"); + } + + return inputType.fields()[0].dataType(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java new file mode 100644 index 000000000000..cfd1b0e8d002 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java @@ -0,0 +1,143 @@ +/* + * 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.functions; + +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.functions.BoundFunction; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.DateType; +import org.apache.spark.sql.types.TimestampNTZType; +import org.apache.spark.sql.types.TimestampType; + +/** + * A Spark function implementation for the Iceberg year transform. + * + *

    Example usage: {@code SELECT system.years('source_col')}. + */ +public class YearsFunction extends UnaryUnboundFunction { + + @Override + protected BoundFunction doBind(DataType valueType) { + if (valueType instanceof DateType) { + return new DateToYearsFunction(); + } else if (valueType instanceof TimestampType) { + return new TimestampToYearsFunction(); + } else if (valueType instanceof TimestampNTZType) { + return new TimestampNtzToYearsFunction(); + } else { + throw new UnsupportedOperationException( + "Expected value to be date or timestamp: " + valueType.catalogString()); + } + } + + @Override + public String description() { + return name() + + "(col) - Call Iceberg's year transform\n" + + " col :: source column (must be date or timestamp)"; + } + + @Override + public String name() { + return "years"; + } + + private abstract static class BaseToYearsFunction extends BaseScalarFunction { + @Override + public String name() { + return "years"; + } + + @Override + public DataType resultType() { + return DataTypes.IntegerType; + } + } + + public static class DateToYearsFunction extends BaseToYearsFunction { + // magic method used in codegen + public static int invoke(int days) { + return DateTimeUtil.daysToYears(days); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.DateType}; + } + + @Override + public String canonicalName() { + return "iceberg.years(date)"; + } + + @Override + public Integer produceResult(InternalRow input) { + // return null for null input to match what Spark does in codegen + return input.isNullAt(0) ? null : invoke(input.getInt(0)); + } + } + + public static class TimestampToYearsFunction extends BaseToYearsFunction { + // magic method used in codegen + public static int invoke(long micros) { + return DateTimeUtil.microsToYears(micros); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.TimestampType}; + } + + @Override + public String canonicalName() { + return "iceberg.years(timestamp)"; + } + + @Override + public Integer produceResult(InternalRow input) { + // return null for null input to match what Spark does in codegen + return input.isNullAt(0) ? null : invoke(input.getLong(0)); + } + } + + public static class TimestampNtzToYearsFunction extends BaseToYearsFunction { + // magic method used in codegen + public static int invoke(long micros) { + return DateTimeUtil.microsToYears(micros); + } + + @Override + public DataType[] inputTypes() { + return new DataType[] {DataTypes.TimestampNTZType}; + } + + @Override + public String canonicalName() { + return "iceberg.years(timestamp_ntz)"; + } + + @Override + public Integer produceResult(InternalRow input) { + // return null for null input to match what Spark does in codegen + return input.isNullAt(0) ? null : invoke(input.getLong(0)); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java new file mode 100644 index 000000000000..e5e369f32701 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/AddFilesProcedure.java @@ -0,0 +1,278 @@ +/* + * 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.procedures; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.mapping.MappingUtil; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +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.spark.Spark3Util; +import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.spark.SparkTableUtil.SparkPartition; +import org.apache.iceberg.util.LocationUtil; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import scala.collection.JavaConverters; + +class AddFilesProcedure extends BaseProcedure { + + static final String NAME = "add_files"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter SOURCE_TABLE_PARAM = + requiredInParameter("source_table", DataTypes.StringType); + private static final ProcedureParameter PARTITION_FILTER_PARAM = + optionalInParameter("partition_filter", STRING_MAP); + private static final ProcedureParameter CHECK_DUPLICATE_FILES_PARAM = + optionalInParameter("check_duplicate_files", DataTypes.BooleanType); + private static final ProcedureParameter PARALLELISM = + optionalInParameter("parallelism", DataTypes.IntegerType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + TABLE_PARAM, + SOURCE_TABLE_PARAM, + PARTITION_FILTER_PARAM, + CHECK_DUPLICATE_FILES_PARAM, + PARALLELISM + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("added_files_count", DataTypes.LongType, false, Metadata.empty()), + new StructField("changed_partition_count", DataTypes.LongType, true, Metadata.empty()), + }); + + private AddFilesProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + public static SparkProcedures.ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + protected AddFilesProcedure doBuild() { + return new AddFilesProcedure(tableCatalog()); + } + }; + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + + Identifier tableIdent = input.ident(TABLE_PARAM); + + CatalogPlugin sessionCat = spark().sessionState().catalogManager().v2SessionCatalog(); + Identifier sourceIdent = input.ident(SOURCE_TABLE_PARAM, sessionCat); + + Map partitionFilter = + input.asStringMap(PARTITION_FILTER_PARAM, ImmutableMap.of()); + + boolean checkDuplicateFiles = input.asBoolean(CHECK_DUPLICATE_FILES_PARAM, true); + + int parallelism = input.asInt(PARALLELISM, 1); + Preconditions.checkArgument(parallelism > 0, "Parallelism should be larger than 0"); + + return asScanIterator( + OUTPUT_TYPE, + importToIceberg( + tableIdent, sourceIdent, partitionFilter, checkDuplicateFiles, parallelism)); + } + + private InternalRow[] toOutputRows(Snapshot snapshot) { + Map summary = snapshot.summary(); + return new InternalRow[] { + newInternalRow(addedFilesCount(summary), changedPartitionCount(summary)) + }; + } + + private long addedFilesCount(Map stats) { + return PropertyUtil.propertyAsLong(stats, SnapshotSummary.ADDED_FILES_PROP, 0L); + } + + private Long changedPartitionCount(Map stats) { + return PropertyUtil.propertyAsNullableLong(stats, SnapshotSummary.CHANGED_PARTITION_COUNT_PROP); + } + + private boolean isFileIdentifier(Identifier ident) { + String[] namespace = ident.namespace(); + return namespace.length == 1 + && (namespace[0].equalsIgnoreCase("orc") + || namespace[0].equalsIgnoreCase("parquet") + || namespace[0].equalsIgnoreCase("avro")); + } + + private InternalRow[] importToIceberg( + Identifier destIdent, + Identifier sourceIdent, + Map partitionFilter, + boolean checkDuplicateFiles, + int parallelism) { + return modifyIcebergTable( + destIdent, + table -> { + ensureNameMappingPresent(table); + + if (isFileIdentifier(sourceIdent)) { + Path sourcePath = new Path(sourceIdent.name()); + String format = sourceIdent.namespace()[0]; + importFileTable( + table, sourcePath, format, partitionFilter, checkDuplicateFiles, parallelism); + } else { + importCatalogTable( + table, sourceIdent, partitionFilter, checkDuplicateFiles, parallelism); + } + + Snapshot snapshot = table.currentSnapshot(); + return toOutputRows(snapshot); + }); + } + + private static void ensureNameMappingPresent(Table table) { + if (table.properties().get(TableProperties.DEFAULT_NAME_MAPPING) == null) { + // Forces Name based resolution instead of position based resolution + NameMapping mapping = MappingUtil.create(table.schema()); + String mappingJson = NameMappingParser.toJson(mapping); + table.updateProperties().set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson).commit(); + } + } + + private void importFileTable( + Table table, + Path tableLocation, + String format, + Map partitionFilter, + boolean checkDuplicateFiles, + int parallelism) { + + org.apache.spark.sql.execution.datasources.PartitionSpec inferredSpec = + Spark3Util.getInferredSpec(spark(), tableLocation); + + List sparkPartNames = + JavaConverters.seqAsJavaList(inferredSpec.partitionColumns()).stream() + .map(StructField::name) + .collect(Collectors.toList()); + PartitionSpec compatibleSpec = SparkTableUtil.findCompatibleSpec(sparkPartNames, table); + + SparkTableUtil.validatePartitionFilter(compatibleSpec, partitionFilter, table.name()); + + // List Partitions via Spark InMemory file search interface + List partitions = + Spark3Util.getPartitions(spark(), tableLocation, format, partitionFilter, compatibleSpec); + + if (table.spec().isUnpartitioned()) { + Preconditions.checkArgument( + partitions.isEmpty(), "Cannot add partitioned files to an unpartitioned table"); + Preconditions.checkArgument( + partitionFilter.isEmpty(), + "Cannot use a partition filter when importing" + "to an unpartitioned table"); + + // Build a Global Partition for the source + SparkPartition partition = + new SparkPartition(Collections.emptyMap(), tableLocation.toString(), format); + importPartitions( + table, ImmutableList.of(partition), checkDuplicateFiles, compatibleSpec, parallelism); + } else { + Preconditions.checkArgument( + !partitions.isEmpty(), "Cannot find any matching partitions in table %s", table.name()); + importPartitions(table, partitions, checkDuplicateFiles, compatibleSpec, parallelism); + } + } + + private void importCatalogTable( + Table table, + Identifier sourceIdent, + Map partitionFilter, + boolean checkDuplicateFiles, + int parallelism) { + String stagingLocation = getMetadataLocation(table); + TableIdentifier sourceTableIdentifier = Spark3Util.toV1TableIdentifier(sourceIdent); + SparkTableUtil.importSparkTable( + spark(), + sourceTableIdentifier, + table, + stagingLocation, + partitionFilter, + checkDuplicateFiles, + parallelism); + } + + private void importPartitions( + Table table, + List partitions, + boolean checkDuplicateFiles, + PartitionSpec spec, + int parallelism) { + String stagingLocation = getMetadataLocation(table); + SparkTableUtil.importSparkPartitions( + spark(), partitions, table, spec, stagingLocation, checkDuplicateFiles, parallelism); + } + + private String getMetadataLocation(Table table) { + String defaultValue = LocationUtil.stripTrailingSlash(table.location()) + "/metadata"; + return LocationUtil.stripTrailingSlash( + table.properties().getOrDefault(TableProperties.WRITE_METADATA_LOCATION, defaultValue)); + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "AddFiles"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/AncestorsOfProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/AncestorsOfProcedure.java new file mode 100644 index 000000000000..bdbe727bffad --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/AncestorsOfProcedure.java @@ -0,0 +1,125 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import java.util.List; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.source.SparkTable; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +public class AncestorsOfProcedure extends BaseProcedure { + + static final String NAME = "ancestors_of"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter SNAPSHOT_ID_PARAM = + optionalInParameter("snapshot_id", DataTypes.LongType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, SNAPSHOT_ID_PARAM}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("snapshot_id", DataTypes.LongType, true, Metadata.empty()), + new StructField("timestamp", DataTypes.LongType, true, Metadata.empty()) + }); + + private AncestorsOfProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + public static SparkProcedures.ProcedureBuilder builder() { + return new Builder() { + @Override + protected AncestorsOfProcedure doBuild() { + return new AncestorsOfProcedure(tableCatalog()); + } + }; + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + + Identifier tableIdent = input.ident(TABLE_PARAM); + Long toSnapshotId = input.asLong(SNAPSHOT_ID_PARAM, null); + + SparkTable sparkTable = loadSparkTable(tableIdent); + Table icebergTable = sparkTable.table(); + + if (toSnapshotId == null) { + toSnapshotId = + icebergTable.currentSnapshot() != null ? icebergTable.currentSnapshot().snapshotId() : -1; + } + + List snapshotIds = + Lists.newArrayList( + SnapshotUtil.ancestorIdsBetween(toSnapshotId, null, icebergTable::snapshot)); + + return asScanIterator(OUTPUT_TYPE, toOutputRow(icebergTable, snapshotIds)); + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "AncestorsOf"; + } + + private InternalRow[] toOutputRow(Table table, List snapshotIds) { + if (snapshotIds.isEmpty()) { + return new InternalRow[0]; + } + + InternalRow[] internalRows = new InternalRow[snapshotIds.size()]; + for (int i = 0; i < snapshotIds.size(); i++) { + Long snapshotId = snapshotIds.get(i); + internalRows[i] = newInternalRow(snapshotId, table.snapshot(snapshotId).timestampMillis()); + } + + return internalRows; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java new file mode 100644 index 000000000000..858579e5dc3f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java @@ -0,0 +1,289 @@ +/* + * 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.procedures; + +import java.util.Collections; +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.function.Function; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.iceberg.spark.source.SparkTable; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.classic.SparkSession; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.catalog.procedures.UnboundProcedure; +import org.apache.spark.sql.connector.read.LocalScan; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.execution.CacheManager; +import org.apache.spark.sql.execution.datasources.SparkExpressionConverter; +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructType; +import scala.Option; + +abstract class BaseProcedure implements BoundProcedure, UnboundProcedure { + protected static final DataType STRING_MAP = + DataTypes.createMapType(DataTypes.StringType, DataTypes.StringType); + protected static final DataType STRING_ARRAY = DataTypes.createArrayType(DataTypes.StringType); + + protected static ProcedureParameter requiredInParameter(String name, DataType dataType) { + return ProcedureParameter.in(name, dataType).build(); + } + + protected static ProcedureParameter optionalInParameter(String name, DataType dataType) { + return optionalInParameter(name, dataType, "NULL"); + } + + protected static ProcedureParameter optionalInParameter( + String name, DataType dataType, String defaultValue) { + return ProcedureParameter.in(name, dataType).defaultValue(defaultValue).build(); + } + + private final SparkSession spark; + private final TableCatalog tableCatalog; + + private SparkActions actions; + private ExecutorService executorService = null; + + protected BaseProcedure(TableCatalog tableCatalog) { + this.spark = SparkSession.active(); + this.tableCatalog = tableCatalog; + } + + @Override + public boolean isDeterministic() { + return false; + } + + protected SparkSession spark() { + return this.spark; + } + + protected SparkActions actions() { + if (actions == null) { + this.actions = SparkActions.get(spark); + } + return actions; + } + + protected TableCatalog tableCatalog() { + return this.tableCatalog; + } + + protected T modifyIcebergTable(Identifier ident, Function func) { + try { + return execute(ident, true, func); + } finally { + closeService(); + } + } + + protected T withIcebergTable(Identifier ident, Function func) { + try { + return execute(ident, false, func); + } finally { + closeService(); + } + } + + private T execute( + Identifier ident, boolean refreshSparkCache, Function func) { + SparkTable sparkTable = loadSparkTable(ident); + org.apache.iceberg.Table icebergTable = sparkTable.table(); + + T result = func.apply(icebergTable); + + if (refreshSparkCache) { + refreshSparkCache(ident, sparkTable); + } + + return result; + } + + protected Identifier toIdentifier(String identifierAsString, String argName) { + CatalogAndIdentifier catalogAndIdentifier = + toCatalogAndIdentifier(identifierAsString, argName, tableCatalog); + + Preconditions.checkArgument( + catalogAndIdentifier.catalog().equals(tableCatalog), + "Cannot run procedure in catalog '%s': '%s' is a table in catalog '%s'", + tableCatalog.name(), + identifierAsString, + catalogAndIdentifier.catalog().name()); + + return catalogAndIdentifier.identifier(); + } + + protected CatalogAndIdentifier toCatalogAndIdentifier( + String identifierAsString, String argName, CatalogPlugin catalog) { + Preconditions.checkArgument( + identifierAsString != null && !identifierAsString.isEmpty(), + "Cannot handle an empty identifier for argument %s", + argName); + + return Spark3Util.catalogAndIdentifier( + "identifier for arg " + argName, spark, identifierAsString, catalog); + } + + protected SparkTable loadSparkTable(Identifier ident) { + try { + Table table = tableCatalog.loadTable(ident); + ValidationException.check( + table instanceof SparkTable, "%s is not %s", ident, SparkTable.class.getName()); + return (SparkTable) table; + } catch (NoSuchTableException e) { + String errMsg = + String.format("Couldn't load table '%s' in catalog '%s'", ident, tableCatalog.name()); + throw new RuntimeException(errMsg, e); + } + } + + protected Dataset loadRows(Identifier tableIdent, Map options) { + String tableName = Spark3Util.quotedFullIdentifier(tableCatalog().name(), tableIdent); + return spark().read().options(options).table(tableName); + } + + protected void refreshSparkCache(Identifier ident, Table table) { + CacheManager cacheManager = spark.sharedState().cacheManager(); + DataSourceV2Relation relation = + DataSourceV2Relation.create(table, Option.apply(tableCatalog), Option.apply(ident)); + cacheManager.recacheByPlan(spark, relation); + } + + protected Expression filterExpression(Identifier ident, String where) { + try { + String name = Spark3Util.quotedFullIdentifier(tableCatalog.name(), ident); + org.apache.spark.sql.catalyst.expressions.Expression expression = + SparkExpressionConverter.collectResolvedSparkExpression(spark, name, where); + return SparkExpressionConverter.convertToIcebergExpression(expression); + } catch (AnalysisException e) { + throw new IllegalArgumentException("Cannot parse predicates in where option: " + where, e); + } + } + + protected InternalRow newInternalRow(Object... values) { + return new GenericInternalRow(values); + } + + protected static class Result implements LocalScan { + private final StructType readSchema; + private final InternalRow[] rows; + + public Result(StructType readSchema, InternalRow[] rows) { + this.readSchema = readSchema; + this.rows = rows; + } + + @Override + public StructType readSchema() { + return this.readSchema; + } + + @Override + public InternalRow[] rows() { + return this.rows; + } + } + + protected Iterator asScanIterator(StructType readSchema, InternalRow... rows) { + return Collections.singleton(new Result(readSchema, rows)).iterator(); + } + + protected abstract static class Builder implements ProcedureBuilder { + private TableCatalog tableCatalog; + + @Override + public Builder withTableCatalog(TableCatalog newTableCatalog) { + this.tableCatalog = newTableCatalog; + return this; + } + + @Override + public T build() { + return doBuild(); + } + + protected abstract T doBuild(); + + TableCatalog tableCatalog() { + return tableCatalog; + } + } + + /** + * Closes this procedure's executor service if a new one was created with {@link + * BaseProcedure#executorService(int, String)}. Does not block for any remaining tasks. + */ + protected void closeService() { + if (executorService != null) { + executorService.shutdown(); + } + } + + /** + * Starts a new executor service which can be used by this procedure in its work. The pool will be + * automatically shut down if {@link #withIcebergTable(Identifier, Function)} or {@link + * #modifyIcebergTable(Identifier, Function)} are called. If these methods are not used then the + * service can be shut down with {@link #closeService()} or left to be closed when this class is + * finalized. + * + * @param threadPoolSize number of threads in the service + * @param nameFormat name prefix for threads created in this service + * @return the new executor service owned by this procedure + */ + protected ExecutorService executorService(int threadPoolSize, String nameFormat) { + Preconditions.checkArgument( + executorService == null, "Cannot create a new executor service, one already exists."); + Preconditions.checkArgument( + nameFormat != null, "Cannot create a service with null nameFormat arg"); + this.executorService = + MoreExecutors.getExitingExecutorService( + (ThreadPoolExecutor) + Executors.newFixedThreadPool( + threadPoolSize, + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat(nameFormat + "-%d") + .build())); + + return executorService; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/CherrypickSnapshotProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/CherrypickSnapshotProcedure.java new file mode 100644 index 000000000000..31043b953b75 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/CherrypickSnapshotProcedure.java @@ -0,0 +1,116 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +/** + * A procedure that applies changes in a given snapshot and creates a new snapshot which will be set + * as the current snapshot in a table. + * + *

    Note: this procedure invalidates all cached Spark plans that reference the affected + * table. + * + * @see org.apache.iceberg.ManageSnapshots#cherrypick(long) + */ +class CherrypickSnapshotProcedure extends BaseProcedure { + + static final String NAME = "cherrypick_snapshot"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter SNAPSHOT_ID_PARAM = + requiredInParameter("snapshot_id", DataTypes.LongType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, SNAPSHOT_ID_PARAM}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("source_snapshot_id", DataTypes.LongType, false, Metadata.empty()), + new StructField("current_snapshot_id", DataTypes.LongType, false, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + protected CherrypickSnapshotProcedure doBuild() { + return new CherrypickSnapshotProcedure(tableCatalog()); + } + }; + } + + private CherrypickSnapshotProcedure(TableCatalog catalog) { + super(catalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + + Identifier tableIdent = input.ident(TABLE_PARAM); + long snapshotId = input.asLong(SNAPSHOT_ID_PARAM); + + return asScanIterator( + OUTPUT_TYPE, + modifyIcebergTable( + tableIdent, + table -> { + table.manageSnapshots().cherrypick(snapshotId).commit(); + + Snapshot currentSnapshot = table.currentSnapshot(); + + InternalRow outputRow = newInternalRow(snapshotId, currentSnapshot.snapshotId()); + return new InternalRow[] {outputRow}; + })); + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "CherrypickSnapshotProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputePartitionStatsProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputePartitionStatsProcedure.java new file mode 100644 index 000000000000..c81157496e5f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputePartitionStatsProcedure.java @@ -0,0 +1,127 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import org.apache.iceberg.PartitionStatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ComputePartitionStats; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +/** + * A procedure that computes the stats incrementally from the last snapshot that has partition stats + * file until the given snapshot (uses current snapshot if not specified) and writes the combined + * result into a {@link PartitionStatisticsFile} after merging the partition stats. Does a full + * compute if previous statistics file does not exist. Also registers the {@link + * PartitionStatisticsFile} to table metadata. + * + * @see SparkActions#computePartitionStats(Table) + */ +public class ComputePartitionStatsProcedure extends BaseProcedure { + + static final String NAME = "compute_partition_stats"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter SNAPSHOT_ID_PARAM = + optionalInParameter("snapshot_id", DataTypes.LongType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, SNAPSHOT_ID_PARAM}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField( + "partition_statistics_file", DataTypes.StringType, true, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new Builder() { + @Override + protected ComputePartitionStatsProcedure doBuild() { + return new ComputePartitionStatsProcedure(tableCatalog()); + } + }; + } + + private ComputePartitionStatsProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + Identifier tableIdent = input.ident(TABLE_PARAM); + Long snapshotId = input.asLong(SNAPSHOT_ID_PARAM, null); + + return modifyIcebergTable( + tableIdent, + table -> { + ComputePartitionStats action = actions().computePartitionStats(table); + if (snapshotId != null) { + action.snapshot(snapshotId); + } + + return asScanIterator(OUTPUT_TYPE, toOutputRows(action.execute())); + }); + } + + private InternalRow[] toOutputRows(ComputePartitionStats.Result result) { + PartitionStatisticsFile statisticsFile = result.statisticsFile(); + if (statisticsFile != null) { + InternalRow row = newInternalRow(UTF8String.fromString(statisticsFile.path())); + return new InternalRow[] {row}; + } else { + return new InternalRow[0]; + } + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "ComputePartitionStatsProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java new file mode 100644 index 000000000000..382fda706b79 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ComputeTableStatsProcedure.java @@ -0,0 +1,131 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +/** + * A procedure that computes statistics of a table. + * + * @see SparkActions#computeTableStats(Table) + */ +public class ComputeTableStatsProcedure extends BaseProcedure { + + static final String NAME = "compute_table_stats"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter SNAPSHOT_ID_PARAM = + optionalInParameter("snapshot_id", DataTypes.LongType); + private static final ProcedureParameter COLUMNS_PARAM = + optionalInParameter("columns", STRING_ARRAY); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, SNAPSHOT_ID_PARAM, COLUMNS_PARAM}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("statistics_file", DataTypes.StringType, true, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new Builder() { + @Override + protected ComputeTableStatsProcedure doBuild() { + return new ComputeTableStatsProcedure(tableCatalog()); + } + }; + } + + private ComputeTableStatsProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + Identifier tableIdent = input.ident(TABLE_PARAM); + Long snapshotId = input.asLong(SNAPSHOT_ID_PARAM, null); + String[] columns = input.asStringArray(COLUMNS_PARAM, null); + + return modifyIcebergTable( + tableIdent, + table -> { + ComputeTableStats action = actions().computeTableStats(table); + + if (snapshotId != null) { + action.snapshot(snapshotId); + } + + if (columns != null) { + action.columns(columns); + } + + ComputeTableStats.Result result = action.execute(); + return asScanIterator(OUTPUT_TYPE, toOutputRows(result)); + }); + } + + private InternalRow[] toOutputRows(ComputeTableStats.Result result) { + StatisticsFile statisticsFile = result.statisticsFile(); + if (statisticsFile != null) { + InternalRow row = newInternalRow(UTF8String.fromString(statisticsFile.path())); + return new InternalRow[] {row}; + } else { + return new InternalRow[0]; + } + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "ComputeTableStatsProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java new file mode 100644 index 000000000000..73c94f778510 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java @@ -0,0 +1,329 @@ +/* + * 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.procedures; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Table; +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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.ChangelogIterator; +import org.apache.iceberg.spark.source.SparkChangelogTable; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.spark.api.java.function.MapPartitionsFunction; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.OrderUtils; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +/** + * A procedure that creates a view for changed rows. + * + *

    The procedure always removes the carry-over rows. Please query {@link SparkChangelogTable} + * instead when carry-over rows are required. + * + *

    The procedure doesn't compute the pre/post update images by default. If you want to compute + * them, you can set "compute_updates" to be true in the options. + * + *

    Carry-over rows are the result of a removal and insertion of the same row within an operation + * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1, + * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this + * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a', + * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the + * table. The procedure finds the carry-over rows and removes them from the result. + * + *

    Pre/post update images are converted from a pair of a delete row and an insert row. Identifier + * columns are used for determining whether an insert and a delete record refer to the same row. If + * the two records share the same values for the identity columns they are considered to be before + * and after states of the same row. You can either set identifier fields in the table schema or + * input them as the procedure parameters. Here is an example of pre/post update images with an + * identifier column(id). A pair of a delete row and an insert row with the same id: + * + *

      + *
    • (id=1, data='a', op='DELETE') + *
    • (id=1, data='b', op='INSERT') + *
    + * + *

    will be marked as pre/post update images: + * + *

      + *
    • (id=1, data='a', op='UPDATE_BEFORE') + *
    • (id=1, data='b', op='UPDATE_AFTER') + *
    + */ +public class CreateChangelogViewProcedure extends BaseProcedure { + + static final String NAME = "create_changelog_view"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter CHANGELOG_VIEW_PARAM = + optionalInParameter("changelog_view", DataTypes.StringType); + private static final ProcedureParameter OPTIONS_PARAM = + optionalInParameter("options", STRING_MAP); + private static final ProcedureParameter COMPUTE_UPDATES_PARAM = + optionalInParameter("compute_updates", DataTypes.BooleanType); + private static final ProcedureParameter IDENTIFIER_COLUMNS_PARAM = + optionalInParameter("identifier_columns", STRING_ARRAY); + private static final ProcedureParameter NET_CHANGES = + optionalInParameter("net_changes", DataTypes.BooleanType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + TABLE_PARAM, + CHANGELOG_VIEW_PARAM, + OPTIONS_PARAM, + COMPUTE_UPDATES_PARAM, + IDENTIFIER_COLUMNS_PARAM, + NET_CHANGES, + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty()) + }); + + public static SparkProcedures.ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + protected CreateChangelogViewProcedure doBuild() { + return new CreateChangelogViewProcedure(tableCatalog()); + } + }; + } + + private CreateChangelogViewProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + + Identifier tableIdent = input.ident(TABLE_PARAM); + + // load insert and deletes from the changelog table + Identifier changelogTableIdent = changelogTableIdent(tableIdent); + Dataset df = loadRows(changelogTableIdent, options(input)); + + boolean netChanges = input.asBoolean(NET_CHANGES, false); + String[] identifierColumns = identifierColumns(input, tableIdent); + Set unorderableColumnNames = + Arrays.stream(df.schema().fields()) + .filter(field -> !OrderUtils.isOrderable(field.dataType())) + .map(StructField::name) + .collect(Collectors.toSet()); + + Preconditions.checkArgument( + identifierColumns.length > 0 || unorderableColumnNames.isEmpty(), + "Identifier field is required as table contains unorderable columns: %s", + unorderableColumnNames); + + if (shouldComputeUpdateImages(input)) { + Preconditions.checkArgument(!netChanges, "Not support net changes with update images"); + df = computeUpdateImages(identifierColumns, df); + } else { + df = removeCarryoverRows(df, netChanges); + } + + String viewName = viewName(input, tableIdent.name()); + + df.createOrReplaceTempView(viewName); + + return asScanIterator(OUTPUT_TYPE, toOutputRows(viewName)); + } + + private Dataset computeUpdateImages(String[] identifierColumns, Dataset df) { + Preconditions.checkArgument( + identifierColumns.length > 0, + "Cannot compute the update images because identifier columns are not set"); + + String[] identifierFields = + ArrayUtil.add(identifierColumns, MetadataColumns.CHANGE_ORDINAL.name()); + Column[] repartitionSpec = + Arrays.stream(identifierFields) + .map(CreateChangelogViewProcedure::delimitedName) + .map(df::col) + .toArray(Column[]::new); + return applyChangelogIterator(df, repartitionSpec, identifierFields); + } + + private boolean shouldComputeUpdateImages(ProcedureInput input) { + // If the identifier columns are set, we compute pre/post update images by default. + boolean defaultValue = input.isProvided(IDENTIFIER_COLUMNS_PARAM); + return input.asBoolean(COMPUTE_UPDATES_PARAM, defaultValue); + } + + private Dataset removeCarryoverRows(Dataset df, boolean netChanges) { + Predicate columnsToKeep; + if (netChanges) { + Set metadataColumn = + Sets.newHashSet( + MetadataColumns.CHANGE_TYPE.name(), + MetadataColumns.CHANGE_ORDINAL.name(), + MetadataColumns.COMMIT_SNAPSHOT_ID.name()); + + columnsToKeep = column -> !metadataColumn.contains(column); + } else { + columnsToKeep = column -> !column.equals(MetadataColumns.CHANGE_TYPE.name()); + } + + Column[] repartitionSpec = + Arrays.stream(df.columns()) + .filter(columnsToKeep) + .map(CreateChangelogViewProcedure::delimitedName) + .map(df::col) + .toArray(Column[]::new); + + return applyCarryoverRemoveIterator(df, repartitionSpec, netChanges); + } + + private String[] identifierColumns(ProcedureInput input, Identifier tableIdent) { + if (input.isProvided(IDENTIFIER_COLUMNS_PARAM)) { + return input.asStringArray(IDENTIFIER_COLUMNS_PARAM); + } else { + Table table = loadSparkTable(tableIdent).table(); + return table.schema().identifierFieldNames().stream().toArray(String[]::new); + } + } + + private Identifier changelogTableIdent(Identifier tableIdent) { + List namespace = Lists.newArrayList(); + namespace.addAll(Arrays.asList(tableIdent.namespace())); + namespace.add(tableIdent.name()); + return Identifier.of(namespace.toArray(new String[0]), SparkChangelogTable.TABLE_NAME); + } + + private Map options(ProcedureInput input) { + return input.asStringMap(OPTIONS_PARAM, ImmutableMap.of()); + } + + private String viewName(ProcedureInput input, String tableName) { + String defaultValue = String.format("`%s_changes`", tableName); + return input.asString(CHANGELOG_VIEW_PARAM, defaultValue); + } + + private Dataset applyChangelogIterator( + Dataset df, Column[] repartitionSpec, String[] identifierFields) { + Column[] sortSpec = sortSpec(df, repartitionSpec, false); + StructType schema = df.schema(); + + return df.repartition(repartitionSpec) + .sortWithinPartitions(sortSpec) + .mapPartitions( + (MapPartitionsFunction) + rowIterator -> + ChangelogIterator.computeUpdates(rowIterator, schema, identifierFields), + Encoders.row(schema)); + } + + private Dataset applyCarryoverRemoveIterator( + Dataset df, Column[] repartitionSpec, boolean netChanges) { + Column[] sortSpec = sortSpec(df, repartitionSpec, netChanges); + StructType schema = df.schema(); + + return df.repartition(repartitionSpec) + .sortWithinPartitions(sortSpec) + .mapPartitions( + (MapPartitionsFunction) + rowIterator -> + netChanges + ? ChangelogIterator.removeNetCarryovers(rowIterator, schema) + : ChangelogIterator.removeCarryovers(rowIterator, schema), + Encoders.row(schema)); + } + + /** + * Ensure that column can be referenced using this name. Issues may come from field names that + * contain non-standard characters. In Spark, this can be fixed by using backtick + * quotes. + * + * @param columnName Column name that potentially can contain non-standard characters. + * @return A name that can be safely used within Spark to reference a column by its name. + */ + private static String delimitedName(String columnName) { + boolean delimited = columnName.startsWith("`") && columnName.endsWith("`"); + if (delimited) { + return columnName; + } else { + return "`" + columnName.replaceAll("`", "``") + "`"; + } + } + + private static Column[] sortSpec(Dataset df, Column[] repartitionSpec, boolean netChanges) { + Column changeType = df.col(MetadataColumns.CHANGE_TYPE.name()); + Column changeOrdinal = df.col(MetadataColumns.CHANGE_ORDINAL.name()); + Column[] extraColumns = + netChanges ? new Column[] {changeOrdinal, changeType} : new Column[] {changeType}; + + Column[] sortSpec = new Column[repartitionSpec.length + extraColumns.length]; + + System.arraycopy(repartitionSpec, 0, sortSpec, 0, repartitionSpec.length); + System.arraycopy(extraColumns, 0, sortSpec, repartitionSpec.length, extraColumns.length); + + return sortSpec; + } + + private InternalRow[] toOutputRows(String viewName) { + InternalRow row = newInternalRow(UTF8String.fromString(viewName)); + return new InternalRow[] {row}; + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "CreateChangelogViewProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java new file mode 100644 index 000000000000..2b771914c3fe --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ExpireSnapshotsProcedure.java @@ -0,0 +1,204 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ExpireSnapshots; +import org.apache.iceberg.io.SupportsBulkOperations; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.actions.ExpireSnapshotsSparkAction; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A procedure that expires snapshots in a table. + * + * @see SparkActions#expireSnapshots(Table) + */ +public class ExpireSnapshotsProcedure extends BaseProcedure { + + static final String NAME = "expire_snapshots"; + + private static final Logger LOG = LoggerFactory.getLogger(ExpireSnapshotsProcedure.class); + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter OLDER_THAN_PARAM = + optionalInParameter("older_than", DataTypes.TimestampType); + private static final ProcedureParameter RETAIN_LAST_PARAM = + optionalInParameter("retain_last", DataTypes.IntegerType); + private static final ProcedureParameter MAX_CONCURRENT_DELETES_PARAM = + optionalInParameter("max_concurrent_deletes", DataTypes.IntegerType); + private static final ProcedureParameter STREAM_RESULTS_PARAM = + optionalInParameter("stream_results", DataTypes.BooleanType); + private static final ProcedureParameter SNAPSHOT_IDS_PARAM = + optionalInParameter("snapshot_ids", DataTypes.createArrayType(DataTypes.LongType)); + private static final ProcedureParameter CLEAN_EXPIRED_METADATA_PARAM = + optionalInParameter("clean_expired_metadata", DataTypes.BooleanType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + TABLE_PARAM, + OLDER_THAN_PARAM, + RETAIN_LAST_PARAM, + MAX_CONCURRENT_DELETES_PARAM, + STREAM_RESULTS_PARAM, + SNAPSHOT_IDS_PARAM, + CLEAN_EXPIRED_METADATA_PARAM + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("deleted_data_files_count", DataTypes.LongType, true, Metadata.empty()), + new StructField( + "deleted_position_delete_files_count", DataTypes.LongType, true, Metadata.empty()), + new StructField( + "deleted_equality_delete_files_count", DataTypes.LongType, true, Metadata.empty()), + new StructField( + "deleted_manifest_files_count", DataTypes.LongType, true, Metadata.empty()), + new StructField( + "deleted_manifest_lists_count", DataTypes.LongType, true, Metadata.empty()), + new StructField( + "deleted_statistics_files_count", DataTypes.LongType, true, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + protected ExpireSnapshotsProcedure doBuild() { + return new ExpireSnapshotsProcedure(tableCatalog()); + } + }; + } + + private ExpireSnapshotsProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + Identifier tableIdent = input.ident(TABLE_PARAM); + Long olderThanMillis = input.asTimestampMillis(OLDER_THAN_PARAM, null); + Integer retainLastNum = input.asInt(RETAIN_LAST_PARAM, null); + Integer maxConcurrentDeletes = input.asInt(MAX_CONCURRENT_DELETES_PARAM, null); + Boolean streamResult = input.asBoolean(STREAM_RESULTS_PARAM, null); + long[] snapshotIds = input.asLongArray(SNAPSHOT_IDS_PARAM, null); + Boolean cleanExpiredMetadata = input.asBoolean(CLEAN_EXPIRED_METADATA_PARAM, null); + + Preconditions.checkArgument( + maxConcurrentDeletes == null || maxConcurrentDeletes > 0, + "max_concurrent_deletes should have value > 0, value: %s", + maxConcurrentDeletes); + + return modifyIcebergTable( + tableIdent, + table -> { + ExpireSnapshots action = actions().expireSnapshots(table); + + if (olderThanMillis != null) { + action.expireOlderThan(olderThanMillis); + } + + if (retainLastNum != null) { + action.retainLast(retainLastNum); + } + + if (maxConcurrentDeletes != null) { + if (table.io() instanceof SupportsBulkOperations) { + LOG.warn( + "max_concurrent_deletes only works with FileIOs that do not support bulk deletes. This " + + "table is currently using {} which supports bulk deletes so the parameter will be ignored. " + + "See that IO's documentation to learn how to adjust parallelism for that particular " + + "IO's bulk delete.", + table.io().getClass().getName()); + } else { + + action.executeDeleteWith(executorService(maxConcurrentDeletes, "expire-snapshots")); + } + } + + if (snapshotIds != null) { + for (long snapshotId : snapshotIds) { + action.expireSnapshotId(snapshotId); + } + } + + if (streamResult != null) { + action.option( + ExpireSnapshotsSparkAction.STREAM_RESULTS, Boolean.toString(streamResult)); + } + + if (cleanExpiredMetadata != null) { + action.cleanExpiredMetadata(cleanExpiredMetadata); + } + + ExpireSnapshots.Result result = action.execute(); + + return asScanIterator(OUTPUT_TYPE, toOutputRows(result)); + }); + } + + private InternalRow[] toOutputRows(ExpireSnapshots.Result result) { + InternalRow row = + newInternalRow( + result.deletedDataFilesCount(), + result.deletedPositionDeleteFilesCount(), + result.deletedEqualityDeleteFilesCount(), + result.deletedManifestsCount(), + result.deletedManifestListsCount(), + result.deletedStatisticsFilesCount()); + return new InternalRow[] {row}; + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "ExpireSnapshotProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/FastForwardBranchProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/FastForwardBranchProcedure.java new file mode 100644 index 000000000000..d7531e759408 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/FastForwardBranchProcedure.java @@ -0,0 +1,109 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +public class FastForwardBranchProcedure extends BaseProcedure { + + static final String NAME = "fast_forward"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter BRANCH_PARAM = + requiredInParameter("branch", DataTypes.StringType); + private static final ProcedureParameter TO_PARAM = + requiredInParameter("to", DataTypes.StringType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, BRANCH_PARAM, TO_PARAM}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("branch_updated", DataTypes.StringType, false, Metadata.empty()), + new StructField("previous_ref", DataTypes.LongType, true, Metadata.empty()), + new StructField("updated_ref", DataTypes.LongType, false, Metadata.empty()) + }); + + public static SparkProcedures.ProcedureBuilder builder() { + return new Builder() { + @Override + protected FastForwardBranchProcedure doBuild() { + return new FastForwardBranchProcedure(tableCatalog()); + } + }; + } + + private FastForwardBranchProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + + Identifier tableIdent = input.ident(TABLE_PARAM); + String from = input.asString(BRANCH_PARAM); + String to = input.asString(TO_PARAM); + + return modifyIcebergTable( + tableIdent, + table -> { + Long snapshotBefore = + table.snapshot(from) != null ? table.snapshot(from).snapshotId() : null; + table.manageSnapshots().fastForwardBranch(from, to).commit(); + long snapshotAfter = table.snapshot(from).snapshotId(); + InternalRow outputRow = + newInternalRow(UTF8String.fromString(from), snapshotBefore, snapshotAfter); + return asScanIterator(OUTPUT_TYPE, outputRow); + }); + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "FastForwardBranchProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java new file mode 100644 index 000000000000..1a01faf38a4f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java @@ -0,0 +1,133 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import java.util.Map; +import org.apache.iceberg.actions.MigrateTable; +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.SparkTableUtil; +import org.apache.iceberg.spark.actions.MigrateTableSparkAction; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +class MigrateTableProcedure extends BaseProcedure { + + static final String NAME = "migrate"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter PROPERTIES_PARAM = + optionalInParameter("properties", STRING_MAP); + private static final ProcedureParameter DROP_BACKUP_PARAM = + optionalInParameter("drop_backup", DataTypes.BooleanType); + private static final ProcedureParameter BACKUP_TABLE_NAME_PARAM = + optionalInParameter("backup_table_name", DataTypes.StringType); + private static final ProcedureParameter PARALLELISM_PARAM = + optionalInParameter("parallelism", DataTypes.IntegerType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + TABLE_PARAM, PROPERTIES_PARAM, DROP_BACKUP_PARAM, BACKUP_TABLE_NAME_PARAM, PARALLELISM_PARAM + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("migrated_files_count", DataTypes.LongType, false, Metadata.empty()) + }); + + private MigrateTableProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + protected MigrateTableProcedure doBuild() { + return new MigrateTableProcedure(tableCatalog()); + } + }; + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + String tableName = input.asString(TABLE_PARAM, null); + Preconditions.checkArgument( + tableName != null && !tableName.isEmpty(), + "Cannot handle an empty identifier for argument table"); + + Map properties = input.asStringMap(PROPERTIES_PARAM, ImmutableMap.of()); + + boolean dropBackup = input.asBoolean(DROP_BACKUP_PARAM, false); + String backupTableName = input.asString(BACKUP_TABLE_NAME_PARAM, null); + + MigrateTableSparkAction migrateTableSparkAction = + SparkActions.get().migrateTable(tableName).tableProperties(properties); + + if (dropBackup) { + migrateTableSparkAction = migrateTableSparkAction.dropBackup(); + } + + if (backupTableName != null) { + migrateTableSparkAction = migrateTableSparkAction.backupTableName(backupTableName); + } + + if (input.isProvided(PARALLELISM_PARAM)) { + int parallelism = input.asInt(PARALLELISM_PARAM); + Preconditions.checkArgument(parallelism > 0, "Parallelism should be larger than 0"); + migrateTableSparkAction = + migrateTableSparkAction.executeWith(SparkTableUtil.migrationService(parallelism)); + } + + MigrateTable.Result result = migrateTableSparkAction.execute(); + return asScanIterator(OUTPUT_TYPE, newInternalRow(result.migratedDataFilesCount())); + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "MigrateTableProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java new file mode 100644 index 000000000000..8f6dbdcf5a62 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java @@ -0,0 +1,274 @@ +/* + * 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.procedures; + +import java.lang.reflect.Array; +import java.util.Map; +import java.util.function.BiFunction; +import org.apache.commons.lang3.StringUtils; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; + +/** A class that abstracts common logic for working with input to a procedure. */ +class ProcedureInput { + + private static final DataType STRING_ARRAY = DataTypes.createArrayType(DataTypes.StringType); + private static final DataType STRING_MAP = + DataTypes.createMapType(DataTypes.StringType, DataTypes.StringType); + + private final SparkSession spark; + private final TableCatalog catalog; + private final Map paramOrdinals; + private final InternalRow args; + + ProcedureInput( + SparkSession spark, TableCatalog catalog, ProcedureParameter[] params, InternalRow args) { + this.spark = spark; + this.catalog = catalog; + this.paramOrdinals = computeParamOrdinals(params); + this.args = args; + } + + public boolean isProvided(ProcedureParameter param) { + int ordinal = ordinal(param); + return !args.isNullAt(ordinal); + } + + public Boolean asBoolean(ProcedureParameter param, Boolean defaultValue) { + validateParamType(param, DataTypes.BooleanType); + int ordinal = ordinal(param); + return args.isNullAt(ordinal) ? defaultValue : (Boolean) args.getBoolean(ordinal); + } + + public Integer asInt(ProcedureParameter param) { + Integer value = asInt(param, null); + Preconditions.checkArgument(value != null, "Parameter '%s' is not set", param.name()); + return value; + } + + public Integer asInt(ProcedureParameter param, Integer defaultValue) { + validateParamType(param, DataTypes.IntegerType); + int ordinal = ordinal(param); + return args.isNullAt(ordinal) ? defaultValue : (Integer) args.getInt(ordinal); + } + + public long asTimestampMillis(ProcedureParameter param) { + Long value = asTimestampMillis(param, null); + Preconditions.checkArgument(value != null, "Parameter '%s' is not set", param.name()); + return value; + } + + public Long asTimestampMillis(ProcedureParameter param, Long defaultValue) { + validateParamType(param, DataTypes.TimestampType); + int ordinal = ordinal(param); + Long value = args.isNullAt(ordinal) ? defaultValue : (Long) args.getLong(ordinal); + if (value != null) { + value = DateTimeUtil.microsToMillis(value); + } + return value; + } + + public long asLong(ProcedureParameter param) { + Long value = asLong(param, null); + Preconditions.checkArgument(value != null, "Parameter '%s' is not set", param.name()); + return value; + } + + public Long asLong(ProcedureParameter param, Long defaultValue) { + validateParamType(param, DataTypes.LongType); + int ordinal = ordinal(param); + return args.isNullAt(ordinal) ? defaultValue : (Long) args.getLong(ordinal); + } + + public long[] asLongArray(ProcedureParameter param, Long[] defaultValue) { + validateParamType(param, DataTypes.createArrayType(DataTypes.LongType)); + Long[] source = + array(param, (array, ordinal) -> array.getLong(ordinal), Long.class, defaultValue); + + if (source == null) { + return null; + } + + long[] result = new long[source.length]; + for (int i = 0; i < source.length; i++) { + result[i] = source[i]; + } + return result; + } + + public String asString(ProcedureParameter param) { + String value = asString(param, null); + Preconditions.checkArgument(value != null, "Parameter '%s' is not set", param.name()); + return value; + } + + public String asString(ProcedureParameter param, String defaultValue) { + validateParamType(param, DataTypes.StringType); + int ordinal = ordinal(param); + return args.isNullAt(ordinal) ? defaultValue : args.getString(ordinal); + } + + public String[] asStringArray(ProcedureParameter param) { + String[] value = asStringArray(param, null); + Preconditions.checkArgument(value != null, "Parameter '%s' is not set", param.name()); + return value; + } + + public String[] asStringArray(ProcedureParameter param, String[] defaultValue) { + validateParamType(param, STRING_ARRAY); + return array( + param, + (array, ordinal) -> array.getUTF8String(ordinal).toString(), + String.class, + defaultValue); + } + + @SuppressWarnings("unchecked") + private T[] array( + ProcedureParameter param, + BiFunction convertElement, + Class elementClass, + T[] defaultValue) { + + int ordinal = ordinal(param); + + if (args.isNullAt(ordinal)) { + return defaultValue; + } + + ArrayData arrayData = args.getArray(ordinal); + + T[] convertedArray = (T[]) Array.newInstance(elementClass, arrayData.numElements()); + + for (int index = 0; index < arrayData.numElements(); index++) { + convertedArray[index] = convertElement.apply(arrayData, index); + } + + return convertedArray; + } + + public Map asStringMap( + ProcedureParameter param, Map defaultValue) { + validateParamType(param, STRING_MAP); + return map( + param, + (keys, ordinal) -> keys.getUTF8String(ordinal).toString(), + (values, ordinal) -> values.getUTF8String(ordinal).toString(), + defaultValue); + } + + private Map map( + ProcedureParameter param, + BiFunction convertKey, + BiFunction convertValue, + Map defaultValue) { + + int ordinal = ordinal(param); + + if (args.isNullAt(ordinal)) { + return defaultValue; + } + + MapData mapData = args.getMap(ordinal); + + Map convertedMap = Maps.newHashMap(); + + for (int index = 0; index < mapData.numElements(); index++) { + K convertedKey = convertKey.apply(mapData.keyArray(), index); + V convertedValue = convertValue.apply(mapData.valueArray(), index); + convertedMap.put(convertedKey, convertedValue); + } + + return convertedMap; + } + + public Identifier ident(ProcedureParameter param) { + CatalogAndIdentifier catalogAndIdent = catalogAndIdent(param, catalog); + + Preconditions.checkArgument( + catalogAndIdent.catalog().equals(catalog), + "Cannot run procedure in catalog '%s': '%s' is a table in catalog '%s'", + catalog.name(), + catalogAndIdent.identifier(), + catalogAndIdent.catalog().name()); + + return catalogAndIdent.identifier(); + } + + public Identifier ident(ProcedureParameter param, CatalogPlugin defaultCatalog) { + CatalogAndIdentifier catalogAndIdent = catalogAndIdent(param, defaultCatalog); + return catalogAndIdent.identifier(); + } + + private CatalogAndIdentifier catalogAndIdent( + ProcedureParameter param, CatalogPlugin defaultCatalog) { + + String identAsString = asString(param); + + Preconditions.checkArgument( + StringUtils.isNotBlank(identAsString), + "Cannot handle an empty identifier for parameter '%s'", + param.name()); + + String desc = String.format("identifier for parameter '%s'", param.name()); + return Spark3Util.catalogAndIdentifier(desc, spark, identAsString, defaultCatalog); + } + + private int ordinal(ProcedureParameter param) { + return paramOrdinals.get(param.name()); + } + + private Map computeParamOrdinals(ProcedureParameter[] params) { + Map ordinals = Maps.newHashMap(); + + for (int index = 0; index < params.length; index++) { + String paramName = params[index].name(); + + Preconditions.checkArgument( + !ordinals.containsKey(paramName), + "Detected multiple parameters named as '%s'", + paramName); + + ordinals.put(paramName, index); + } + + return ordinals; + } + + private void validateParamType(ProcedureParameter param, DataType expectedDataType) { + Preconditions.checkArgument( + expectedDataType.sameType(param.dataType()), + "Parameter '%s' must be of type %s", + param.name(), + expectedDataType.catalogString()); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/PublishChangesProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/PublishChangesProcedure.java new file mode 100644 index 000000000000..874888204334 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/PublishChangesProcedure.java @@ -0,0 +1,129 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import java.util.Optional; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.iceberg.util.WapUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +/** + * A procedure that applies changes in a snapshot created within a Write-Audit-Publish workflow with + * a wap_id and creates a new snapshot which will be set as the current snapshot in a table. + * + *

    Note: this procedure invalidates all cached Spark plans that reference the affected + * table. + * + * @see org.apache.iceberg.ManageSnapshots#cherrypick(long) + */ +class PublishChangesProcedure extends BaseProcedure { + + static final String NAME = "publish_changes"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter WAP_ID_PARAM = + requiredInParameter("wap_id", DataTypes.StringType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, WAP_ID_PARAM}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("source_snapshot_id", DataTypes.LongType, false, Metadata.empty()), + new StructField("current_snapshot_id", DataTypes.LongType, false, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new Builder() { + @Override + protected PublishChangesProcedure doBuild() { + return new PublishChangesProcedure(tableCatalog()); + } + }; + } + + private PublishChangesProcedure(TableCatalog catalog) { + super(catalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + + Identifier tableIdent = input.ident(TABLE_PARAM); + String wapId = input.asString(WAP_ID_PARAM); + + return modifyIcebergTable( + tableIdent, + table -> { + Optional wapSnapshot = + Optional.ofNullable( + Iterables.find( + table.snapshots(), + snapshot -> wapId.equals(WapUtil.stagedWapId(snapshot)), + null)); + if (!wapSnapshot.isPresent()) { + throw new ValidationException("Cannot apply unknown WAP ID '%s'", wapId); + } + + long wapSnapshotId = wapSnapshot.get().snapshotId(); + table.manageSnapshots().cherrypick(wapSnapshotId).commit(); + + Snapshot currentSnapshot = table.currentSnapshot(); + + InternalRow outputRow = newInternalRow(wapSnapshotId, currentSnapshot.snapshotId()); + return asScanIterator(OUTPUT_TYPE, outputRow); + }); + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "ApplyWapChangesProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RegisterTableProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RegisterTableProcedure.java new file mode 100644 index 000000000000..9ba577ad7e24 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RegisterTableProcedure.java @@ -0,0 +1,126 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.iceberg.spark.source.HasIcebergCatalog; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +class RegisterTableProcedure extends BaseProcedure { + + static final String NAME = "register_table"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter METADATA_FILE_PARAM = + requiredInParameter("metadata_file", DataTypes.StringType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, METADATA_FILE_PARAM}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("current_snapshot_id", DataTypes.LongType, true, Metadata.empty()), + new StructField("total_records_count", DataTypes.LongType, true, Metadata.empty()), + new StructField("total_data_files_count", DataTypes.LongType, true, Metadata.empty()) + }); + + private RegisterTableProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + protected RegisterTableProcedure doBuild() { + return new RegisterTableProcedure(tableCatalog()); + } + }; + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + TableIdentifier tableName = + Spark3Util.identifierToTableIdentifier( + toIdentifier(input.asString(TABLE_PARAM), TABLE_PARAM.name())); + String metadataFile = input.asString(METADATA_FILE_PARAM); + Preconditions.checkArgument( + tableCatalog() instanceof HasIcebergCatalog, + "Cannot use Register Table in a non-Iceberg catalog"); + Preconditions.checkArgument( + metadataFile != null && !metadataFile.isEmpty(), + "Cannot handle an empty argument metadata_file"); + + Catalog icebergCatalog = ((HasIcebergCatalog) tableCatalog()).icebergCatalog(); + Table table = icebergCatalog.registerTable(tableName, metadataFile); + Long currentSnapshotId = null; + Long totalDataFiles = null; + Long totalRecords = null; + + Snapshot currentSnapshot = table.currentSnapshot(); + if (currentSnapshot != null) { + currentSnapshotId = currentSnapshot.snapshotId(); + totalDataFiles = + Long.parseLong(currentSnapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP)); + totalRecords = + Long.parseLong(currentSnapshot.summary().get(SnapshotSummary.TOTAL_RECORDS_PROP)); + } + + return asScanIterator( + OUTPUT_TYPE, newInternalRow(currentSnapshotId, totalRecords, totalDataFiles)); + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "RegisterTableProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java new file mode 100644 index 000000000000..89e218c10360 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.procedures; + +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.actions.DeleteOrphanFiles.PrefixMismatchMode; +import org.apache.iceberg.io.SupportsBulkOperations; +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.Iterables; +import org.apache.iceberg.spark.actions.DeleteOrphanFilesSparkAction; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A procedure that removes orphan files in a table. + * + * @see SparkActions#deleteOrphanFiles(Table) + */ +public class RemoveOrphanFilesProcedure extends BaseProcedure { + + static final String NAME = "remove_orphan_files"; + + private static final Logger LOG = LoggerFactory.getLogger(RemoveOrphanFilesProcedure.class); + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter OLDER_THAN_PARAM = + optionalInParameter("older_than", DataTypes.TimestampType); + private static final ProcedureParameter LOCATION_PARAM = + optionalInParameter("location", DataTypes.StringType); + private static final ProcedureParameter DRY_RUN_PARAM = + optionalInParameter("dry_run", DataTypes.BooleanType); + private static final ProcedureParameter MAX_CONCURRENT_DELETES_PARAM = + optionalInParameter("max_concurrent_deletes", DataTypes.IntegerType); + private static final ProcedureParameter FILE_LIST_VIEW_PARAM = + optionalInParameter("file_list_view", DataTypes.StringType); + private static final ProcedureParameter EQUAL_SCHEMES_PARAM = + optionalInParameter("equal_schemes", STRING_MAP); + private static final ProcedureParameter EQUAL_AUTHORITIES_PARAM = + optionalInParameter("equal_authorities", STRING_MAP); + private static final ProcedureParameter PREFIX_MISMATCH_MODE_PARAM = + optionalInParameter("prefix_mismatch_mode", DataTypes.StringType); + // List files with prefix operations. Default is false. + private static final ProcedureParameter PREFIX_LISTING_PARAM = + optionalInParameter("prefix_listing", DataTypes.BooleanType); + // Stream results to avoid loading all orphan files in driver memory. Default is false. + private static final ProcedureParameter STREAM_RESULTS_PARAM = + optionalInParameter("stream_results", DataTypes.BooleanType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + TABLE_PARAM, + OLDER_THAN_PARAM, + LOCATION_PARAM, + DRY_RUN_PARAM, + MAX_CONCURRENT_DELETES_PARAM, + FILE_LIST_VIEW_PARAM, + EQUAL_SCHEMES_PARAM, + EQUAL_AUTHORITIES_PARAM, + PREFIX_MISMATCH_MODE_PARAM, + PREFIX_LISTING_PARAM, + STREAM_RESULTS_PARAM + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("orphan_file_location", DataTypes.StringType, false, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + protected RemoveOrphanFilesProcedure doBuild() { + return new RemoveOrphanFilesProcedure(tableCatalog()); + } + }; + } + + private RemoveOrphanFilesProcedure(TableCatalog catalog) { + super(catalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + Identifier tableIdent = input.ident(TABLE_PARAM); + Long olderThanMillis = input.asTimestampMillis(OLDER_THAN_PARAM, null); + String location = input.asString(LOCATION_PARAM, null); + boolean dryRun = input.asBoolean(DRY_RUN_PARAM, false); + Integer maxConcurrentDeletes = input.asInt(MAX_CONCURRENT_DELETES_PARAM, null); + String fileListView = input.asString(FILE_LIST_VIEW_PARAM, null); + + Preconditions.checkArgument( + maxConcurrentDeletes == null || maxConcurrentDeletes > 0, + "max_concurrent_deletes should have value > 0, value: %s", + maxConcurrentDeletes); + + Map equalSchemes = input.asStringMap(EQUAL_SCHEMES_PARAM, ImmutableMap.of()); + Map equalAuthorities = + input.asStringMap(EQUAL_AUTHORITIES_PARAM, ImmutableMap.of()); + + PrefixMismatchMode prefixMismatchMode = asPrefixMismatchMode(input, PREFIX_MISMATCH_MODE_PARAM); + + boolean prefixListing = input.asBoolean(PREFIX_LISTING_PARAM, false); + boolean streamResults = input.asBoolean(STREAM_RESULTS_PARAM, false); + + return withIcebergTable( + tableIdent, + table -> { + DeleteOrphanFilesSparkAction action = actions().deleteOrphanFiles(table); + + if (olderThanMillis != null) { + boolean isTesting = Boolean.parseBoolean(spark().conf().get("spark.testing", "false")); + if (!isTesting) { + validateInterval(olderThanMillis); + } + action.olderThan(olderThanMillis); + } + + if (location != null) { + action.location(location); + } + + if (dryRun) { + action.deleteWith(file -> {}); + } + + if (maxConcurrentDeletes != null) { + if (table.io() instanceof SupportsBulkOperations) { + LOG.warn( + "max_concurrent_deletes only works with FileIOs that do not support bulk deletes. This " + + "table is currently using {} which supports bulk deletes so the parameter will be ignored. " + + "See that IO's documentation to learn how to adjust parallelism for that particular " + + "IO's bulk delete.", + table.io().getClass().getName()); + } else { + + action.executeDeleteWith(executorService(maxConcurrentDeletes, "remove-orphans")); + } + } + + if (fileListView != null) { + action.compareToFileList(spark().table(fileListView)); + } + + action.equalSchemes(equalSchemes); + action.equalAuthorities(equalAuthorities); + + if (prefixMismatchMode != null) { + action.prefixMismatchMode(prefixMismatchMode); + } + + action.usePrefixListing(prefixListing); + + if (streamResults) { + action.option("stream-results", "true"); + } + + DeleteOrphanFiles.Result result = action.execute(); + + return asScanIterator(OUTPUT_TYPE, toOutputRows(result)); + }); + } + + private InternalRow[] toOutputRows(DeleteOrphanFiles.Result result) { + Iterable orphanFileLocations = result.orphanFileLocations(); + + int orphanFileLocationsCount = Iterables.size(orphanFileLocations); + InternalRow[] rows = new InternalRow[orphanFileLocationsCount]; + + int index = 0; + for (String fileLocation : orphanFileLocations) { + rows[index] = newInternalRow(UTF8String.fromString(fileLocation)); + index++; + } + + return rows; + } + + private void validateInterval(long olderThanMillis) { + long intervalMillis = System.currentTimeMillis() - olderThanMillis; + if (intervalMillis < TimeUnit.DAYS.toMillis(1)) { + throw new IllegalArgumentException( + "Cannot remove orphan files with an interval less than 24 hours. Executing this " + + "procedure with a short interval may corrupt the table if other operations are happening " + + "at the same time. If you are absolutely confident that no concurrent operations will be " + + "affected by removing orphan files with such a short interval, you can use the Action API " + + "to remove orphan files with an arbitrary interval."); + } + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "RemoveOrphanFilesProcedure"; + } + + private PrefixMismatchMode asPrefixMismatchMode(ProcedureInput input, ProcedureParameter param) { + String modeAsString = input.asString(param, null); + return (modeAsString == null) ? null : PrefixMismatchMode.fromString(modeAsString); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteDataFilesProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteDataFilesProcedure.java new file mode 100644 index 000000000000..3e4f9ea58723 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteDataFilesProcedure.java @@ -0,0 +1,229 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewriteDataFiles; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.NamedReference; +import org.apache.iceberg.expressions.Zorder; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.ExtendedParser; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +/** + * A procedure that rewrites datafiles in a table. + * + * @see org.apache.iceberg.spark.actions.SparkActions#rewriteDataFiles(Table) + */ +class RewriteDataFilesProcedure extends BaseProcedure { + + static final String NAME = "rewrite_data_files"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter STRATEGY_PARAM = + optionalInParameter("strategy", DataTypes.StringType); + private static final ProcedureParameter SORT_ORDER_PARAM = + optionalInParameter("sort_order", DataTypes.StringType); + private static final ProcedureParameter OPTIONS_PARAM = + optionalInParameter("options", STRING_MAP); + private static final ProcedureParameter WHERE_PARAM = + optionalInParameter("where", DataTypes.StringType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + TABLE_PARAM, STRATEGY_PARAM, SORT_ORDER_PARAM, OPTIONS_PARAM, WHERE_PARAM + }; + + // counts are not nullable since the action result is never null + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField( + "rewritten_data_files_count", DataTypes.IntegerType, false, Metadata.empty()), + new StructField( + "added_data_files_count", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("rewritten_bytes_count", DataTypes.LongType, false, Metadata.empty()), + new StructField( + "failed_data_files_count", DataTypes.IntegerType, false, Metadata.empty()), + new StructField( + "removed_delete_files_count", DataTypes.IntegerType, false, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new Builder() { + @Override + protected RewriteDataFilesProcedure doBuild() { + return new RewriteDataFilesProcedure(tableCatalog()); + } + }; + } + + private RewriteDataFilesProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + Identifier tableIdent = input.ident(TABLE_PARAM); + String strategy = input.asString(STRATEGY_PARAM, null); + String sortOrderString = input.asString(SORT_ORDER_PARAM, null); + Map options = input.asStringMap(OPTIONS_PARAM, ImmutableMap.of()); + String where = input.asString(WHERE_PARAM, null); + + return modifyIcebergTable( + tableIdent, + table -> { + RewriteDataFiles action = actions().rewriteDataFiles(table).options(options); + + if (strategy != null || sortOrderString != null) { + action = checkAndApplyStrategy(action, strategy, sortOrderString, table.schema()); + } + + action = checkAndApplyFilter(action, where, tableIdent); + + RewriteDataFiles.Result result = action.execute(); + + return asScanIterator(OUTPUT_TYPE, toOutputRows(result)); + }); + } + + private RewriteDataFiles checkAndApplyFilter( + RewriteDataFiles action, String where, Identifier ident) { + if (where != null) { + Expression expression = filterExpression(ident, where); + return action.filter(expression); + } + return action; + } + + private RewriteDataFiles checkAndApplyStrategy( + RewriteDataFiles action, String strategy, String sortOrderString, Schema schema) { + List zOrderTerms = Lists.newArrayList(); + List sortOrderFields = Lists.newArrayList(); + if (sortOrderString != null) { + ExtendedParser.parseSortOrder(spark(), sortOrderString) + .forEach( + field -> { + if (field.term() instanceof Zorder) { + zOrderTerms.add((Zorder) field.term()); + } else { + sortOrderFields.add(field); + } + }); + + if (!zOrderTerms.isEmpty() && !sortOrderFields.isEmpty()) { + // TODO: we need to allow this in future when SparkAction has handling for this. + throw new IllegalArgumentException( + "Cannot mix identity sort columns and a Zorder sort expression: " + sortOrderString); + } + } + + // caller of this function ensures that between strategy and sortOrder, at least one of them is + // not null. + if (strategy == null || strategy.equalsIgnoreCase("sort")) { + if (!zOrderTerms.isEmpty()) { + String[] columnNames = + zOrderTerms.stream() + .flatMap(zOrder -> zOrder.refs().stream().map(NamedReference::name)) + .toArray(String[]::new); + return action.zOrder(columnNames); + } else if (!sortOrderFields.isEmpty()) { + return action.sort(buildSortOrder(sortOrderFields, schema)); + } else { + return action.sort(); + } + } + if (strategy.equalsIgnoreCase("binpack")) { + RewriteDataFiles rewriteDataFiles = action.binPack(); + if (sortOrderString != null) { + // calling below method to throw the error as user has set both binpack strategy and sort + // order + return rewriteDataFiles.sort(buildSortOrder(sortOrderFields, schema)); + } + return rewriteDataFiles; + } else { + throw new IllegalArgumentException( + "unsupported strategy: " + strategy + ". Only binpack or sort is supported"); + } + } + + private SortOrder buildSortOrder( + List rawOrderFields, Schema schema) { + SortOrder.Builder builder = SortOrder.builderFor(schema); + rawOrderFields.forEach( + rawField -> builder.sortBy(rawField.term(), rawField.direction(), rawField.nullOrder())); + return builder.build(); + } + + private InternalRow[] toOutputRows(RewriteDataFiles.Result result) { + int rewrittenDataFilesCount = result.rewrittenDataFilesCount(); + long rewrittenBytesCount = result.rewrittenBytesCount(); + int addedDataFilesCount = result.addedDataFilesCount(); + int failedDataFilesCount = result.failedDataFilesCount(); + int removedDeleteFilesCount = result.removedDeleteFilesCount(); + + InternalRow row = + newInternalRow( + rewrittenDataFilesCount, + addedDataFilesCount, + rewrittenBytesCount, + failedDataFilesCount, + removedDeleteFilesCount); + return new InternalRow[] {row}; + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "RewriteDataFilesProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java new file mode 100644 index 000000000000..d4bd0764ee7e --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteManifestsProcedure.java @@ -0,0 +1,135 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewriteManifests; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.spark.actions.RewriteManifestsSparkAction; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +/** + * A procedure that rewrites manifests in a table. + * + *

    Note: this procedure invalidates all cached Spark plans that reference the affected + * table. + * + * @see SparkActions#rewriteManifests(Table) () + */ +class RewriteManifestsProcedure extends BaseProcedure { + + static final String NAME = "rewrite_manifests"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter USE_CACHING_PARAM = + optionalInParameter("use_caching", DataTypes.BooleanType); + private static final ProcedureParameter SPEC_ID_PARAM = + optionalInParameter("spec_id", DataTypes.IntegerType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, USE_CACHING_PARAM, SPEC_ID_PARAM}; + + // counts are not nullable since the action result is never null + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField( + "rewritten_manifests_count", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("added_manifests_count", DataTypes.IntegerType, false, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + protected RewriteManifestsProcedure doBuild() { + return new RewriteManifestsProcedure(tableCatalog()); + } + }; + } + + private RewriteManifestsProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + Identifier tableIdent = input.ident(TABLE_PARAM); + Boolean useCaching = input.asBoolean(USE_CACHING_PARAM, null); + Integer specId = input.asInt(SPEC_ID_PARAM, null); + + return modifyIcebergTable( + tableIdent, + table -> { + RewriteManifestsSparkAction action = actions().rewriteManifests(table); + + if (useCaching != null) { + action.option(RewriteManifestsSparkAction.USE_CACHING, useCaching.toString()); + } + + if (specId != null) { + action.specId(specId); + } + + RewriteManifests.Result result = action.execute(); + + return asScanIterator(OUTPUT_TYPE, toOutputRows(result)); + }); + } + + private InternalRow[] toOutputRows(RewriteManifests.Result result) { + int rewrittenManifestsCount = Iterables.size(result.rewrittenManifests()); + int addedManifestsCount = Iterables.size(result.addedManifests()); + InternalRow row = newInternalRow(rewrittenManifestsCount, addedManifestsCount); + return new InternalRow[] {row}; + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "RewriteManifestsProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewritePositionDeleteFilesProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewritePositionDeleteFilesProcedure.java new file mode 100644 index 000000000000..ae130cfd5208 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewritePositionDeleteFilesProcedure.java @@ -0,0 +1,131 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import java.util.Map; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewritePositionDeleteFiles; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +/** + * A procedure that rewrites position delete files in a table. + * + * @see org.apache.iceberg.spark.actions.SparkActions#rewritePositionDeletes(Table) + */ +public class RewritePositionDeleteFilesProcedure extends BaseProcedure { + + static final String NAME = "rewrite_position_delete_files"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter OPTIONS_PARAM = + optionalInParameter("options", STRING_MAP); + private static final ProcedureParameter WHERE_PARAM = + optionalInParameter("where", DataTypes.StringType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, OPTIONS_PARAM, WHERE_PARAM}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField( + "rewritten_delete_files_count", DataTypes.IntegerType, false, Metadata.empty()), + new StructField( + "added_delete_files_count", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("rewritten_bytes_count", DataTypes.LongType, false, Metadata.empty()), + new StructField("added_bytes_count", DataTypes.LongType, false, Metadata.empty()) + }); + + public static SparkProcedures.ProcedureBuilder builder() { + return new Builder() { + @Override + protected RewritePositionDeleteFilesProcedure doBuild() { + return new RewritePositionDeleteFilesProcedure(tableCatalog()); + } + }; + } + + private RewritePositionDeleteFilesProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + Identifier tableIdent = input.ident(TABLE_PARAM); + Map options = input.asStringMap(OPTIONS_PARAM, ImmutableMap.of()); + String where = input.asString(WHERE_PARAM, null); + + return modifyIcebergTable( + tableIdent, + table -> { + RewritePositionDeleteFiles action = + actions().rewritePositionDeletes(table).options(options); + + if (where != null) { + Expression whereExpression = filterExpression(tableIdent, where); + action = action.filter(whereExpression); + } + + RewritePositionDeleteFiles.Result result = action.execute(); + return asScanIterator(OUTPUT_TYPE, toOutputRow(result)); + }); + } + + private InternalRow toOutputRow(RewritePositionDeleteFiles.Result result) { + return newInternalRow( + result.rewrittenDeleteFilesCount(), + result.addedDeleteFilesCount(), + result.rewrittenBytesCount(), + result.addedBytesCount()); + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "RewritePositionDeleteFilesProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteTablePathProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteTablePathProcedure.java new file mode 100644 index 000000000000..f40ead013b64 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RewriteTablePathProcedure.java @@ -0,0 +1,157 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import org.apache.iceberg.actions.RewriteTablePath; +import org.apache.iceberg.spark.actions.RewriteTablePathSparkAction; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +public class RewriteTablePathProcedure extends BaseProcedure { + + static final String NAME = "rewrite_table_path"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter SOURCE_PREFIX_PARAM = + requiredInParameter("source_prefix", DataTypes.StringType); + private static final ProcedureParameter TARGET_PREFIX_PARAM = + requiredInParameter("target_prefix", DataTypes.StringType); + private static final ProcedureParameter START_VERSION_PARAM = + optionalInParameter("start_version", DataTypes.StringType); + private static final ProcedureParameter END_VERSION_PARM = + optionalInParameter("end_version", DataTypes.StringType); + private static final ProcedureParameter STAGING_LOCATION_PARAM = + optionalInParameter("staging_location", DataTypes.StringType); + private static final ProcedureParameter CREATE_FILE_LIST_PARAM = + optionalInParameter("create_file_list", DataTypes.BooleanType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + TABLE_PARAM, + SOURCE_PREFIX_PARAM, + TARGET_PREFIX_PARAM, + START_VERSION_PARAM, + END_VERSION_PARM, + STAGING_LOCATION_PARAM, + CREATE_FILE_LIST_PARAM + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("latest_version", DataTypes.StringType, true, Metadata.empty()), + new StructField("file_list_location", DataTypes.StringType, true, Metadata.empty()), + new StructField( + "rewritten_manifest_file_paths_count", + DataTypes.IntegerType, + true, + Metadata.empty()), + new StructField( + "rewritten_delete_file_paths_count", DataTypes.IntegerType, true, Metadata.empty()) + }); + + public static SparkProcedures.ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + protected RewriteTablePathProcedure doBuild() { + return new RewriteTablePathProcedure(tableCatalog()); + } + }; + } + + private RewriteTablePathProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + Identifier tableIdent = input.ident(TABLE_PARAM); + String sourcePrefix = input.asString(SOURCE_PREFIX_PARAM); + String targetPrefix = input.asString(TARGET_PREFIX_PARAM); + String startVersion = input.asString(START_VERSION_PARAM, null); + String endVersion = input.asString(END_VERSION_PARM, null); + String stagingLocation = input.asString(STAGING_LOCATION_PARAM, null); + boolean createFileList = input.asBoolean(CREATE_FILE_LIST_PARAM, true); + + return withIcebergTable( + tableIdent, + table -> { + RewriteTablePathSparkAction action = SparkActions.get().rewriteTablePath(table); + + if (startVersion != null) { + action.startVersion(startVersion); + } + if (endVersion != null) { + action.endVersion(endVersion); + } + if (stagingLocation != null) { + action.stagingLocation(stagingLocation); + } + + action.createFileList(createFileList); + + return asScanIterator( + OUTPUT_TYPE, + toOutputRows(action.rewriteLocationPrefix(sourcePrefix, targetPrefix).execute())); + }); + } + + private InternalRow[] toOutputRows(RewriteTablePath.Result result) { + return new InternalRow[] { + newInternalRow( + UTF8String.fromString(result.latestVersion()), + UTF8String.fromString(result.fileListLocation()), + result.rewrittenManifestFilePathsCount(), + result.rewrittenDeleteFilePathsCount()) + }; + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "RewriteTablePathProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToSnapshotProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToSnapshotProcedure.java new file mode 100644 index 000000000000..98e1e2b870f7 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToSnapshotProcedure.java @@ -0,0 +1,112 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +/** + * A procedure that rollbacks a table to a specific snapshot id. + * + *

    Note: this procedure invalidates all cached Spark plans that reference the affected + * table. + * + * @see org.apache.iceberg.ManageSnapshots#rollbackTo(long) + */ +class RollbackToSnapshotProcedure extends BaseProcedure { + + static final String NAME = "rollback_to_snapshot"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter SNAPSHOT_ID_PARAM = + requiredInParameter("snapshot_id", DataTypes.LongType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, SNAPSHOT_ID_PARAM}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("previous_snapshot_id", DataTypes.LongType, false, Metadata.empty()), + new StructField("current_snapshot_id", DataTypes.LongType, false, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + public RollbackToSnapshotProcedure doBuild() { + return new RollbackToSnapshotProcedure(tableCatalog()); + } + }; + } + + private RollbackToSnapshotProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + Identifier tableIdent = input.ident(TABLE_PARAM); + long snapshotId = input.asLong(SNAPSHOT_ID_PARAM); + + return modifyIcebergTable( + tableIdent, + table -> { + Snapshot previousSnapshot = table.currentSnapshot(); + + table.manageSnapshots().rollbackTo(snapshotId).commit(); + + InternalRow outputRow = newInternalRow(previousSnapshot.snapshotId(), snapshotId); + return asScanIterator(OUTPUT_TYPE, outputRow); + }); + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "RollbackToSnapshotProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToTimestampProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToTimestampProcedure.java new file mode 100644 index 000000000000..fd6791df28de --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/RollbackToTimestampProcedure.java @@ -0,0 +1,116 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +/** + * A procedure that rollbacks a table to a given point in time. + * + *

    Note: this procedure invalidates all cached Spark plans that reference the affected + * table. + * + * @see org.apache.iceberg.ManageSnapshots#rollbackToTime(long) + */ +class RollbackToTimestampProcedure extends BaseProcedure { + + static final String NAME = "rollback_to_timestamp"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter TIMESTAMP_PARAM = + requiredInParameter("timestamp", DataTypes.TimestampType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, TIMESTAMP_PARAM}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("previous_snapshot_id", DataTypes.LongType, false, Metadata.empty()), + new StructField("current_snapshot_id", DataTypes.LongType, false, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + protected RollbackToTimestampProcedure doBuild() { + return new RollbackToTimestampProcedure(tableCatalog()); + } + }; + } + + private RollbackToTimestampProcedure(TableCatalog catalog) { + super(catalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + Identifier tableIdent = input.ident(TABLE_PARAM); + // timestamps in Spark have microsecond precision so this conversion is lossy + long timestampMillis = input.asTimestampMillis(TIMESTAMP_PARAM); + + return modifyIcebergTable( + tableIdent, + table -> { + Snapshot previousSnapshot = table.currentSnapshot(); + + table.manageSnapshots().rollbackToTime(timestampMillis).commit(); + + Snapshot currentSnapshot = table.currentSnapshot(); + + InternalRow outputRow = + newInternalRow(previousSnapshot.snapshotId(), currentSnapshot.snapshotId()); + return asScanIterator(OUTPUT_TYPE, outputRow); + }); + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "RollbackToTimestampProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SetCurrentSnapshotProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SetCurrentSnapshotProcedure.java new file mode 100644 index 000000000000..18e3646cf529 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SetCurrentSnapshotProcedure.java @@ -0,0 +1,130 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +/** + * A procedure that sets the current snapshot in a table. + * + *

    Note: this procedure invalidates all cached Spark plans that reference the affected + * table. + * + * @see org.apache.iceberg.ManageSnapshots#setCurrentSnapshot(long) + */ +class SetCurrentSnapshotProcedure extends BaseProcedure { + + static final String NAME = "set_current_snapshot"; + + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter SNAPSHOT_ID_PARAM = + optionalInParameter("snapshot_id", DataTypes.LongType); + private static final ProcedureParameter REF_PARAM = + optionalInParameter("ref", DataTypes.StringType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] {TABLE_PARAM, SNAPSHOT_ID_PARAM, REF_PARAM}; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("previous_snapshot_id", DataTypes.LongType, true, Metadata.empty()), + new StructField("current_snapshot_id", DataTypes.LongType, false, Metadata.empty()) + }); + + public static ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + protected SetCurrentSnapshotProcedure doBuild() { + return new SetCurrentSnapshotProcedure(tableCatalog()); + } + }; + } + + private SetCurrentSnapshotProcedure(TableCatalog catalog) { + super(catalog); + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + Identifier tableIdent = input.ident(TABLE_PARAM); + Long snapshotId = input.asLong(SNAPSHOT_ID_PARAM, null); + String ref = input.asString(REF_PARAM, null); + Preconditions.checkArgument( + (snapshotId != null && ref == null) || (snapshotId == null && ref != null), + "Either snapshot_id or ref must be provided, not both"); + + return modifyIcebergTable( + tableIdent, + table -> { + Snapshot previousSnapshot = table.currentSnapshot(); + Long previousSnapshotId = previousSnapshot != null ? previousSnapshot.snapshotId() : null; + + long targetSnapshotId = snapshotId != null ? snapshotId : toSnapshotId(table, ref); + table.manageSnapshots().setCurrentSnapshot(targetSnapshotId).commit(); + + InternalRow outputRow = newInternalRow(previousSnapshotId, targetSnapshotId); + return asScanIterator(OUTPUT_TYPE, outputRow); + }); + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "SetCurrentSnapshotProcedure"; + } + + private long toSnapshotId(Table table, String refName) { + SnapshotRef ref = table.refs().get(refName); + ValidationException.check(ref != null, "Cannot find matching snapshot ID for ref %s", refName); + return ref.snapshotId(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java new file mode 100644 index 000000000000..1498e3b56864 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java @@ -0,0 +1,132 @@ +/* + * 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.procedures; + +import java.util.Iterator; +import java.util.Map; +import org.apache.iceberg.actions.SnapshotTable; +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.SparkTableUtil; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.BoundProcedure; +import org.apache.spark.sql.connector.catalog.procedures.ProcedureParameter; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +class SnapshotTableProcedure extends BaseProcedure { + + static final String NAME = "snapshot"; + + private static final ProcedureParameter SOURCE_TABLE_PARAM = + requiredInParameter("source_table", DataTypes.StringType); + private static final ProcedureParameter TABLE_PARAM = + requiredInParameter("table", DataTypes.StringType); + private static final ProcedureParameter LOCATION_PARAM = + optionalInParameter("location", DataTypes.StringType); + private static final ProcedureParameter PROPERTIES_PARAM = + optionalInParameter("properties", STRING_MAP); + private static final ProcedureParameter PARALLELISM_PARAM = + optionalInParameter("parallelism", DataTypes.IntegerType); + + private static final ProcedureParameter[] PARAMETERS = + new ProcedureParameter[] { + SOURCE_TABLE_PARAM, TABLE_PARAM, LOCATION_PARAM, PROPERTIES_PARAM, PARALLELISM_PARAM + }; + + private static final StructType OUTPUT_TYPE = + new StructType( + new StructField[] { + new StructField("imported_files_count", DataTypes.LongType, false, Metadata.empty()) + }); + + private SnapshotTableProcedure(TableCatalog tableCatalog) { + super(tableCatalog); + } + + public static SparkProcedures.ProcedureBuilder builder() { + return new BaseProcedure.Builder() { + @Override + protected SnapshotTableProcedure doBuild() { + return new SnapshotTableProcedure(tableCatalog()); + } + }; + } + + @Override + public BoundProcedure bind(StructType inputType) { + return this; + } + + @Override + public ProcedureParameter[] parameters() { + return PARAMETERS; + } + + @Override + public Iterator call(InternalRow args) { + ProcedureInput input = new ProcedureInput(spark(), tableCatalog(), PARAMETERS, args); + + String source = input.asString(SOURCE_TABLE_PARAM, null); + Preconditions.checkArgument( + source != null && !source.isEmpty(), + "Cannot handle an empty identifier for argument source_table"); + + String dest = input.asString(TABLE_PARAM, null); + Preconditions.checkArgument( + dest != null && !dest.isEmpty(), "Cannot handle an empty identifier for argument table"); + + String snapshotLocation = input.asString(LOCATION_PARAM, null); + + Map properties = input.asStringMap(PROPERTIES_PARAM, ImmutableMap.of()); + + Preconditions.checkArgument( + !source.equals(dest), + "Cannot create a snapshot with the same name as the source of the snapshot."); + SnapshotTable action = SparkActions.get().snapshotTable(source).as(dest); + + if (snapshotLocation != null) { + action.tableLocation(snapshotLocation); + } + + if (input.isProvided(PARALLELISM_PARAM)) { + int parallelism = input.asInt(PARALLELISM_PARAM); + Preconditions.checkArgument(parallelism > 0, "Parallelism should be larger than 0"); + action = action.executeWith(SparkTableUtil.migrationService(parallelism)); + } + + SnapshotTable.Result result = action.tableProperties(properties).execute(); + return asScanIterator(OUTPUT_TYPE, newInternalRow(result.importedDataFilesCount())); + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "SnapshotTableProcedure"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java new file mode 100644 index 000000000000..6b42a04421dc --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java @@ -0,0 +1,71 @@ +/* + * 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.procedures; + +import java.util.Locale; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.procedures.UnboundProcedure; + +public class SparkProcedures { + + private static final Map> BUILDERS = initProcedureBuilders(); + + private SparkProcedures() {} + + public static ProcedureBuilder newBuilder(String name) { + // procedure resolution is case insensitive to match the existing Spark behavior for functions + Supplier builderSupplier = BUILDERS.get(name.toLowerCase(Locale.ROOT)); + return builderSupplier != null ? builderSupplier.get() : null; + } + + private static Map> initProcedureBuilders() { + ImmutableMap.Builder> mapBuilder = ImmutableMap.builder(); + mapBuilder.put(RollbackToSnapshotProcedure.NAME, RollbackToSnapshotProcedure::builder); + mapBuilder.put(RollbackToTimestampProcedure.NAME, RollbackToTimestampProcedure::builder); + mapBuilder.put(SetCurrentSnapshotProcedure.NAME, SetCurrentSnapshotProcedure::builder); + mapBuilder.put(CherrypickSnapshotProcedure.NAME, CherrypickSnapshotProcedure::builder); + mapBuilder.put(RewriteDataFilesProcedure.NAME, RewriteDataFilesProcedure::builder); + mapBuilder.put(RewriteManifestsProcedure.NAME, RewriteManifestsProcedure::builder); + mapBuilder.put(RemoveOrphanFilesProcedure.NAME, RemoveOrphanFilesProcedure::builder); + mapBuilder.put(ExpireSnapshotsProcedure.NAME, ExpireSnapshotsProcedure::builder); + mapBuilder.put(MigrateTableProcedure.NAME, MigrateTableProcedure::builder); + mapBuilder.put(SnapshotTableProcedure.NAME, SnapshotTableProcedure::builder); + mapBuilder.put(AddFilesProcedure.NAME, AddFilesProcedure::builder); + mapBuilder.put(AncestorsOfProcedure.NAME, AncestorsOfProcedure::builder); + mapBuilder.put(RegisterTableProcedure.NAME, RegisterTableProcedure::builder); + mapBuilder.put(PublishChangesProcedure.NAME, PublishChangesProcedure::builder); + mapBuilder.put(CreateChangelogViewProcedure.NAME, CreateChangelogViewProcedure::builder); + mapBuilder.put( + RewritePositionDeleteFilesProcedure.NAME, RewritePositionDeleteFilesProcedure::builder); + mapBuilder.put(FastForwardBranchProcedure.NAME, FastForwardBranchProcedure::builder); + mapBuilder.put(ComputeTableStatsProcedure.NAME, ComputeTableStatsProcedure::builder); + mapBuilder.put(ComputePartitionStatsProcedure.NAME, ComputePartitionStatsProcedure::builder); + mapBuilder.put(RewriteTablePathProcedure.NAME, RewriteTablePathProcedure::builder); + return mapBuilder.build(); + } + + public interface ProcedureBuilder { + ProcedureBuilder withTableCatalog(TableCatalog tableCatalog); + + UnboundProcedure build(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java new file mode 100644 index 000000000000..ff30f29aeae6 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -0,0 +1,221 @@ +/* + * 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.Map; +import java.util.Set; +import javax.annotation.Nonnull; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.OrcBatchReadConf; +import org.apache.iceberg.spark.ParquetBatchReadConf; +import org.apache.iceberg.spark.ParquetReaderType; +import org.apache.iceberg.spark.data.vectorized.ColumnVectorWithFilter; +import org.apache.iceberg.spark.data.vectorized.ColumnarBatchUtil; +import org.apache.iceberg.spark.data.vectorized.UpdatableDeletedColumnVector; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +abstract class BaseBatchReader extends BaseReader { + private final ParquetBatchReadConf parquetConf; + private final OrcBatchReadConf orcConf; + + BaseBatchReader( + Table table, + ScanTaskGroup taskGroup, + Schema tableSchema, + Schema expectedSchema, + boolean caseSensitive, + ParquetBatchReadConf parquetConf, + OrcBatchReadConf orcConf, + boolean cacheDeleteFilesOnExecutors) { + super( + table, taskGroup, tableSchema, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors); + this.parquetConf = parquetConf; + this.orcConf = orcConf; + } + + protected CloseableIterable newBatchIterable( + InputFile inputFile, + FileFormat format, + long start, + long length, + Expression residual, + Map idToConstant, + @Nonnull SparkDeleteFilter deleteFilter) { + CloseableIterable iterable; + switch (format) { + case PARQUET: + iterable = + newParquetIterable( + inputFile, start, length, residual, idToConstant, deleteFilter.requiredSchema()); + break; + case ORC: + iterable = newOrcIterable(inputFile, start, length, residual, idToConstant); + break; + default: + throw new UnsupportedOperationException( + "Format: " + format + " not supported for batched reads"); + } + + return CloseableIterable.transform(iterable, new BatchDeleteFilter(deleteFilter)::filterBatch); + } + + private CloseableIterable newParquetIterable( + InputFile inputFile, + long start, + long length, + Expression residual, + Map idToConstant, + Schema requiredSchema) { + return Parquet.read(inputFile) + .project(requiredSchema) + .split(start, length) + .createBatchedReaderFunc( + fileSchema -> { + if (parquetConf.readerType() == ParquetReaderType.COMET) { + return VectorizedSparkParquetReaders.buildCometReader( + requiredSchema, fileSchema, idToConstant); + } else { + return VectorizedSparkParquetReaders.buildReader( + requiredSchema, fileSchema, idToConstant); + } + }) + .recordsPerBatch(parquetConf.batchSize()) + .filter(residual) + .caseSensitive(caseSensitive()) + // Spark eagerly consumes the batches. So the underlying memory allocated could be reused + // without worrying about subsequent reads clobbering over each other. This improves + // read performance as every batch read doesn't have to pay the cost of allocating memory. + .reuseContainers() + .withNameMapping(nameMapping()) + .build(); + } + + private CloseableIterable newOrcIterable( + InputFile inputFile, + long start, + long length, + Expression residual, + Map idToConstant) { + Set constantFieldIds = idToConstant.keySet(); + Set metadataFieldIds = MetadataColumns.metadataFieldIds(); + Sets.SetView constantAndMetadataFieldIds = + Sets.union(constantFieldIds, metadataFieldIds); + Schema schemaWithoutConstantAndMetadataFields = + TypeUtil.selectNot(expectedSchema(), constantAndMetadataFieldIds); + + return ORC.read(inputFile) + .project(schemaWithoutConstantAndMetadataFields) + .split(start, length) + .createBatchedReaderFunc( + fileSchema -> + VectorizedSparkOrcReaders.buildReader(expectedSchema(), fileSchema, idToConstant)) + .recordsPerBatch(orcConf.batchSize()) + .filter(residual) + .caseSensitive(caseSensitive()) + .withNameMapping(nameMapping()) + .build(); + } + + @VisibleForTesting + static class BatchDeleteFilter { + private final DeleteFilter deletes; + private boolean hasIsDeletedColumn; + private int rowPositionColumnIndex = -1; + + BatchDeleteFilter(DeleteFilter deletes) { + this.deletes = deletes; + + Schema schema = deletes.requiredSchema(); + for (int i = 0; i < schema.columns().size(); i++) { + if (schema.columns().get(i).fieldId() == MetadataColumns.ROW_POSITION.fieldId()) { + this.rowPositionColumnIndex = i; + } else if (schema.columns().get(i).fieldId() == MetadataColumns.IS_DELETED.fieldId()) { + this.hasIsDeletedColumn = true; + } + } + } + + ColumnarBatch filterBatch(ColumnarBatch batch) { + if (!needDeletes()) { + return batch; + } + + ColumnVector[] vectors = new ColumnVector[batch.numCols()]; + for (int i = 0; i < batch.numCols(); i++) { + vectors[i] = batch.column(i); + } + + int numLiveRows = batch.numRows(); + long rowStartPosInBatch = + rowPositionColumnIndex == -1 ? -1 : vectors[rowPositionColumnIndex].getLong(0); + + if (hasIsDeletedColumn) { + boolean[] isDeleted = + ColumnarBatchUtil.buildIsDeleted(vectors, deletes, rowStartPosInBatch, numLiveRows); + for (ColumnVector vector : vectors) { + if (vector instanceof UpdatableDeletedColumnVector) { + ((UpdatableDeletedColumnVector) vector).setValue(isDeleted); + } + } + } else { + Pair pair = + ColumnarBatchUtil.buildRowIdMapping(vectors, deletes, rowStartPosInBatch, numLiveRows); + if (pair != null) { + int[] rowIdMapping = pair.first(); + numLiveRows = pair.second(); + for (int i = 0; i < vectors.length; i++) { + vectors[i] = new ColumnVectorWithFilter(vectors[i], rowIdMapping); + } + } + } + + if (deletes != null && deletes.hasEqDeletes()) { + vectors = ColumnarBatchUtil.removeExtraColumns(deletes, vectors); + } + + ColumnarBatch output = new ColumnarBatch(vectors); + output.setNumRows(numLiveRows); + return output; + } + + private boolean needDeletes() { + return hasIsDeletedColumn + || (deletes != null && (deletes.hasEqDeletes() || deletes.hasPosDeletes())); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java new file mode 100644 index 000000000000..bf1622617185 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseReader.java @@ -0,0 +1,256 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.BaseDeleteLoader; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.data.DeleteLoader; +import org.apache.iceberg.deletes.DeleteCounter; +import org.apache.iceberg.encryption.EncryptingFileIO; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.spark.SparkExecutorCache; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkUtil; +import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.util.PartitionUtil; +import org.apache.spark.rdd.InputFileBlockHolder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Base class of Spark readers. + * + * @param is the Java class returned by this reader whose objects contain one or more rows. + */ +abstract class BaseReader implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger(BaseReader.class); + + private final Table table; + private final Schema tableSchema; + private final Schema expectedSchema; + private final boolean caseSensitive; + private final NameMapping nameMapping; + private final ScanTaskGroup taskGroup; + private final Iterator tasks; + private final DeleteCounter counter; + private final boolean cacheDeleteFilesOnExecutors; + + private Map lazyInputFiles; + private CloseableIterator currentIterator; + private T current = null; + private TaskT currentTask = null; + + BaseReader( + Table table, + ScanTaskGroup taskGroup, + Schema tableSchema, + Schema expectedSchema, + boolean caseSensitive, + boolean cacheDeleteFilesOnExecutors) { + this.table = table; + this.taskGroup = taskGroup; + this.tasks = taskGroup.tasks().iterator(); + this.currentIterator = CloseableIterator.empty(); + this.tableSchema = tableSchema; + this.expectedSchema = expectedSchema; + this.caseSensitive = caseSensitive; + String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + this.nameMapping = + nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; + this.counter = new DeleteCounter(); + this.cacheDeleteFilesOnExecutors = cacheDeleteFilesOnExecutors; + } + + protected abstract CloseableIterator open(TaskT task); + + protected abstract Stream> referencedFiles(TaskT task); + + protected Schema expectedSchema() { + return expectedSchema; + } + + protected boolean caseSensitive() { + return caseSensitive; + } + + protected boolean cacheDeleteFilesOnExecutors() { + return cacheDeleteFilesOnExecutors; + } + + protected NameMapping nameMapping() { + return nameMapping; + } + + protected Table table() { + return table; + } + + protected DeleteCounter counter() { + return counter; + } + + public boolean next() throws IOException { + try { + while (true) { + if (currentIterator.hasNext()) { + this.current = currentIterator.next(); + return true; + } else if (tasks.hasNext()) { + this.currentIterator.close(); + this.currentTask = tasks.next(); + this.currentIterator = open(currentTask); + } else { + this.currentIterator.close(); + return false; + } + } + } catch (IOException | RuntimeException e) { + if (currentTask != null && !currentTask.isDataTask()) { + String filePaths = + referencedFiles(currentTask) + .map(ContentFile::location) + .collect(Collectors.joining(", ")); + LOG.error("Error reading file(s): {}", filePaths, e); + } + throw e; + } + } + + public T get() { + return current; + } + + @Override + public void close() throws IOException { + InputFileBlockHolder.unset(); + + // close the current iterator + this.currentIterator.close(); + + // exhaust the task iterator + while (tasks.hasNext()) { + tasks.next(); + } + } + + protected InputFile getInputFile(String location) { + return inputFiles().get(location); + } + + private Map inputFiles() { + if (lazyInputFiles == null) { + this.lazyInputFiles = + EncryptingFileIO.combine(table().io(), table().encryption()) + .bulkDecrypt( + () -> taskGroup.tasks().stream().flatMap(this::referencedFiles).iterator()); + } + + return lazyInputFiles; + } + + protected Map constantsMap(ContentScanTask task, Schema readSchema) { + if (readSchema.findField(MetadataColumns.PARTITION_COLUMN_ID) != null) { + StructType partitionType = Partitioning.partitionType(table); + return PartitionUtil.constantsMap(task, partitionType, SparkUtil::internalToSpark); + } else { + return PartitionUtil.constantsMap(task, SparkUtil::internalToSpark); + } + } + + protected class SparkDeleteFilter extends DeleteFilter { + private final InternalRowWrapper asStructLike; + + SparkDeleteFilter( + String filePath, List deletes, DeleteCounter counter, boolean needRowPosCol) { + super(filePath, deletes, tableSchema, expectedSchema, counter, needRowPosCol); + this.asStructLike = + new InternalRowWrapper( + SparkSchemaUtil.convert(requiredSchema()), requiredSchema().asStruct()); + } + + @Override + protected StructLike asStructLike(InternalRow row) { + return asStructLike.wrap(row); + } + + @Override + protected InputFile getInputFile(String location) { + return BaseReader.this.getInputFile(location); + } + + @Override + protected void markRowDeleted(InternalRow row) { + if (!row.getBoolean(columnIsDeletedPosition())) { + row.setBoolean(columnIsDeletedPosition(), true); + counter().increment(); + } + } + + @Override + protected DeleteLoader newDeleteLoader() { + if (cacheDeleteFilesOnExecutors) { + return new CachingDeleteLoader(this::loadInputFile); + } + return new BaseDeleteLoader(this::loadInputFile); + } + + private class CachingDeleteLoader extends BaseDeleteLoader { + private final SparkExecutorCache cache; + + CachingDeleteLoader(Function loadInputFile) { + super(loadInputFile); + this.cache = SparkExecutorCache.getOrCreate(); + } + + @Override + protected boolean canCache(long size) { + return cache != null && size < cache.maxEntrySize(); + } + + @Override + protected V getOrLoad(String key, Supplier valueSupplier, long valueSize) { + return cache.getOrLoad(table().name(), key, valueSupplier, valueSize); + } + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java new file mode 100644 index 000000000000..c12931e786b1 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseRowReader.java @@ -0,0 +1,127 @@ +/* + * 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.Map; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.data.SparkOrcReader; +import org.apache.iceberg.spark.data.SparkParquetReaders; +import org.apache.iceberg.spark.data.SparkPlannedAvroReader; +import org.apache.iceberg.types.TypeUtil; +import org.apache.spark.sql.catalyst.InternalRow; + +abstract class BaseRowReader extends BaseReader { + BaseRowReader( + Table table, + ScanTaskGroup taskGroup, + Schema tableSchema, + Schema expectedSchema, + boolean caseSensitive, + boolean cacheDeleteFilesOnExecutors) { + super( + table, taskGroup, tableSchema, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors); + } + + protected CloseableIterable newIterable( + InputFile file, + FileFormat format, + long start, + long length, + Expression residual, + Schema projection, + Map idToConstant) { + switch (format) { + case PARQUET: + return newParquetIterable(file, start, length, residual, projection, idToConstant); + + case AVRO: + return newAvroIterable(file, start, length, projection, idToConstant); + + case ORC: + return newOrcIterable(file, start, length, residual, projection, idToConstant); + + default: + throw new UnsupportedOperationException("Cannot read unknown format: " + format); + } + } + + private CloseableIterable newAvroIterable( + InputFile file, long start, long length, Schema projection, Map idToConstant) { + return Avro.read(file) + .reuseContainers() + .project(projection) + .split(start, length) + .createResolvingReader(schema -> SparkPlannedAvroReader.create(schema, idToConstant)) + .withNameMapping(nameMapping()) + .build(); + } + + private CloseableIterable newParquetIterable( + InputFile file, + long start, + long length, + Expression residual, + Schema readSchema, + Map idToConstant) { + return Parquet.read(file) + .reuseContainers() + .split(start, length) + .project(readSchema) + .createReaderFunc( + fileSchema -> SparkParquetReaders.buildReader(readSchema, fileSchema, idToConstant)) + .filter(residual) + .caseSensitive(caseSensitive()) + .withNameMapping(nameMapping()) + .build(); + } + + private CloseableIterable newOrcIterable( + InputFile file, + long start, + long length, + Expression residual, + Schema readSchema, + Map idToConstant) { + Schema readSchemaWithoutConstantAndMetadataFields = + TypeUtil.selectNot( + readSchema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds())); + + return ORC.read(file) + .project(readSchemaWithoutConstantAndMetadataFields) + .split(start, length) + .createReaderFunc( + readOrcSchema -> new SparkOrcReader(readSchema, readOrcSchema, idToConstant)) + .filter(residual) + .caseSensitive(caseSensitive()) + .withNameMapping(nameMapping()) + .build(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java new file mode 100644 index 000000000000..9ec0f885775f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -0,0 +1,126 @@ +/* + * 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.Map; +import java.util.stream.Stream; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.OrcBatchReadConf; +import org.apache.iceberg.spark.ParquetBatchReadConf; +import org.apache.iceberg.spark.source.metrics.TaskNumDeletes; +import org.apache.iceberg.spark.source.metrics.TaskNumSplits; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.rdd.InputFileBlockHolder; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class BatchDataReader extends BaseBatchReader + implements PartitionReader { + + private static final Logger LOG = LoggerFactory.getLogger(BatchDataReader.class); + + private final long numSplits; + + BatchDataReader( + SparkInputPartition partition, + ParquetBatchReadConf parquetBatchReadConf, + OrcBatchReadConf orcBatchReadConf) { + this( + partition.table(), + partition.taskGroup(), + SnapshotUtil.schemaFor(partition.table(), partition.branch()), + partition.expectedSchema(), + partition.isCaseSensitive(), + parquetBatchReadConf, + orcBatchReadConf, + partition.cacheDeleteFilesOnExecutors()); + } + + BatchDataReader( + Table table, + ScanTaskGroup taskGroup, + Schema tableSchema, + Schema expectedSchema, + boolean caseSensitive, + ParquetBatchReadConf parquetConf, + OrcBatchReadConf orcConf, + boolean cacheDeleteFilesOnExecutors) { + super( + table, + taskGroup, + tableSchema, + expectedSchema, + caseSensitive, + parquetConf, + orcConf, + cacheDeleteFilesOnExecutors); + + numSplits = taskGroup.tasks().size(); + LOG.debug("Reading {} file split(s) for table {}", numSplits, table.name()); + } + + @Override + public CustomTaskMetric[] currentMetricsValues() { + return new CustomTaskMetric[] { + new TaskNumSplits(numSplits), new TaskNumDeletes(counter().get()) + }; + } + + @Override + protected Stream> referencedFiles(FileScanTask task) { + return Stream.concat(Stream.of(task.file()), task.deletes().stream()); + } + + @Override + protected CloseableIterator open(FileScanTask task) { + String filePath = task.file().location(); + LOG.debug("Opening data file {}", filePath); + + // update the current file for Spark's filename() function + InputFileBlockHolder.set(filePath, task.start(), task.length()); + + InputFile inputFile = getInputFile(filePath); + Preconditions.checkNotNull(inputFile, "Could not find InputFile associated with FileScanTask"); + + SparkDeleteFilter deleteFilter = + new SparkDeleteFilter(filePath, task.deletes(), counter(), true); + + Map idToConstant = constantsMap(task, deleteFilter.requiredSchema()); + + return newBatchIterable( + inputFile, + task.file().format(), + task.start(), + task.length(), + task.residual(), + idToConstant, + deleteFilter) + .iterator(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java new file mode 100644 index 000000000000..b8fa129f6a44 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/ChangelogRowReader.java @@ -0,0 +1,176 @@ +/* + * 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.stream.Stream; +import org.apache.iceberg.AddedRowsScanTask; +import org.apache.iceberg.ChangelogScanTask; +import org.apache.iceberg.ChangelogUtil; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DeletedDataFileScanTask; +import org.apache.iceberg.DeletedRowsScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.rdd.InputFileBlockHolder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.expressions.JoinedRow; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.unsafe.types.UTF8String; + +class ChangelogRowReader extends BaseRowReader + implements PartitionReader { + + ChangelogRowReader(SparkInputPartition partition) { + this( + partition.table(), + partition.taskGroup(), + SnapshotUtil.schemaFor(partition.table(), partition.branch()), + partition.expectedSchema(), + partition.isCaseSensitive(), + partition.cacheDeleteFilesOnExecutors()); + } + + ChangelogRowReader( + Table table, + ScanTaskGroup taskGroup, + Schema tableSchema, + Schema expectedSchema, + boolean caseSensitive, + boolean cacheDeleteFilesOnExecutors) { + super( + table, + taskGroup, + tableSchema, + ChangelogUtil.dropChangelogMetadata(expectedSchema), + caseSensitive, + cacheDeleteFilesOnExecutors); + } + + @Override + protected CloseableIterator open(ChangelogScanTask task) { + JoinedRow cdcRow = new JoinedRow(); + + cdcRow.withRight(changelogMetadata(task)); + + CloseableIterable rows = openChangelogScanTask(task); + CloseableIterable cdcRows = CloseableIterable.transform(rows, cdcRow::withLeft); + + return cdcRows.iterator(); + } + + private static InternalRow changelogMetadata(ChangelogScanTask task) { + InternalRow metadataRow = new GenericInternalRow(3); + + metadataRow.update(0, UTF8String.fromString(task.operation().name())); + metadataRow.update(1, task.changeOrdinal()); + metadataRow.update(2, task.commitSnapshotId()); + + return metadataRow; + } + + private CloseableIterable openChangelogScanTask(ChangelogScanTask task) { + if (task instanceof AddedRowsScanTask) { + return openAddedRowsScanTask((AddedRowsScanTask) task); + + } else if (task instanceof DeletedRowsScanTask) { + throw new UnsupportedOperationException("Deleted rows scan task is not supported yet"); + + } else if (task instanceof DeletedDataFileScanTask) { + return openDeletedDataFileScanTask((DeletedDataFileScanTask) task); + + } else { + throw new IllegalArgumentException( + "Unsupported changelog scan task type: " + task.getClass().getName()); + } + } + + CloseableIterable openAddedRowsScanTask(AddedRowsScanTask task) { + String filePath = task.file().location(); + SparkDeleteFilter deletes = new SparkDeleteFilter(filePath, task.deletes(), counter(), true); + return deletes.filter(rows(task, deletes.requiredSchema())); + } + + private CloseableIterable openDeletedDataFileScanTask(DeletedDataFileScanTask task) { + String filePath = task.file().location(); + SparkDeleteFilter deletes = + new SparkDeleteFilter(filePath, task.existingDeletes(), counter(), true); + return deletes.filter(rows(task, deletes.requiredSchema())); + } + + private CloseableIterable rows(ContentScanTask task, Schema readSchema) { + Map idToConstant = constantsMap(task, readSchema); + + String filePath = task.file().location(); + + // update the current file for Spark's filename() function + InputFileBlockHolder.set(filePath, task.start(), task.length()); + + InputFile location = getInputFile(filePath); + Preconditions.checkNotNull(location, "Could not find InputFile"); + return newIterable( + location, + task.file().format(), + task.start(), + task.length(), + task.residual(), + readSchema, + idToConstant); + } + + @Override + protected Stream> referencedFiles(ChangelogScanTask task) { + if (task instanceof AddedRowsScanTask) { + return addedRowsScanTaskFiles((AddedRowsScanTask) task); + + } else if (task instanceof DeletedRowsScanTask) { + throw new UnsupportedOperationException("Deleted rows scan task is not supported yet"); + + } else if (task instanceof DeletedDataFileScanTask) { + return deletedDataFileScanTaskFiles((DeletedDataFileScanTask) task); + + } else { + throw new IllegalArgumentException( + "Unsupported changelog scan task type: " + task.getClass().getName()); + } + } + + private static Stream> deletedDataFileScanTaskFiles(DeletedDataFileScanTask task) { + DataFile file = task.file(); + List existingDeletes = task.existingDeletes(); + return Stream.concat(Stream.of(file), existingDeletes.stream()); + } + + private static Stream> addedRowsScanTaskFiles(AddedRowsScanTask task) { + DataFile file = task.file(); + List deletes = task.deletes(); + return Stream.concat(Stream.of(file), deletes.stream()); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/DVIterator.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/DVIterator.java new file mode 100644 index 000000000000..2942413a7543 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/DVIterator.java @@ -0,0 +1,109 @@ +/* + * 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.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.BaseDeleteLoader; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ScanTaskUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.unsafe.types.UTF8String; + +class DVIterator implements CloseableIterator { + private final DeleteFile deleteFile; + private final Schema projection; + private final Map idToConstant; + private final Iterator positions; + private Integer deletedPositionIndex; + private GenericInternalRow row; + + DVIterator( + InputFile inputFile, DeleteFile deleteFile, Schema projection, Map idToConstant) { + this.deleteFile = deleteFile; + this.projection = projection; + this.idToConstant = idToConstant; + List pos = Lists.newArrayList(); + new BaseDeleteLoader(ignored -> inputFile) + .loadPositionDeletes(ImmutableList.of(deleteFile), deleteFile.referencedDataFile()) + .forEach(pos::add); + this.positions = pos.iterator(); + } + + @Override + public boolean hasNext() { + return positions.hasNext(); + } + + @Override + public InternalRow next() { + long position = positions.next(); + + if (null == row) { + List rowValues = Lists.newArrayList(); + for (Types.NestedField column : projection.columns()) { + int fieldId = column.fieldId(); + if (fieldId == MetadataColumns.DELETE_FILE_PATH.fieldId()) { + rowValues.add(UTF8String.fromString(deleteFile.referencedDataFile())); + } else if (fieldId == MetadataColumns.DELETE_FILE_POS.fieldId()) { + rowValues.add(position); + // remember the index where the deleted position needs to be set + deletedPositionIndex = rowValues.size() - 1; + } else if (fieldId == MetadataColumns.PARTITION_COLUMN_ID) { + rowValues.add(idToConstant.get(MetadataColumns.PARTITION_COLUMN_ID)); + } else if (fieldId == MetadataColumns.SPEC_ID_COLUMN_ID) { + rowValues.add(idToConstant.get(MetadataColumns.SPEC_ID_COLUMN_ID)); + } else if (fieldId == MetadataColumns.FILE_PATH_COLUMN_ID) { + rowValues.add(idToConstant.get(MetadataColumns.FILE_PATH_COLUMN_ID)); + } else if (fieldId == MetadataColumns.CONTENT_OFFSET_COLUMN_ID) { + rowValues.add(deleteFile.contentOffset()); + } else if (fieldId == MetadataColumns.CONTENT_SIZE_IN_BYTES_COLUMN_ID) { + rowValues.add(ScanTaskUtil.contentSizeInBytes(deleteFile)); + } else if (fieldId == MetadataColumns.DELETE_FILE_ROW_FIELD_ID) { + // DVs don't track the row that was deleted + rowValues.add(null); + } + } + + this.row = new GenericInternalRow(rowValues.toArray()); + } else if (null != deletedPositionIndex) { + // only update the deleted position if necessary, everything else stays the same + row.update(deletedPositionIndex, position); + } + + return row; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Remove is not supported"); + } + + @Override + public void close() {} +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java new file mode 100644 index 000000000000..e1292647b797 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/EqualityDeleteRowReader.java @@ -0,0 +1,57 @@ +/* + * 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.Map; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.spark.rdd.InputFileBlockHolder; +import org.apache.spark.sql.catalyst.InternalRow; + +public class EqualityDeleteRowReader extends RowDataReader { + public EqualityDeleteRowReader( + CombinedScanTask task, + Table table, + Schema tableSchema, + Schema expectedSchema, + boolean caseSensitive, + boolean cacheDeleteFilesOnExecutors) { + super(table, task, tableSchema, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors); + } + + @Override + protected CloseableIterator open(FileScanTask task) { + SparkDeleteFilter matches = + new SparkDeleteFilter(task.file().location(), task.deletes(), counter(), true); + + // schema or rows returned by readers + Schema requiredSchema = matches.requiredSchema(); + Map idToConstant = constantsMap(task, expectedSchema()); + DataFile file = task.file(); + + // update the current file for Spark's filename() function + InputFileBlockHolder.set(file.location(), task.start(), task.length()); + + return matches.findEqualityDeleteRows(open(task, requiredSchema, idToConstant)).iterator(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/ExtractRowLineage.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/ExtractRowLineage.java new file mode 100644 index 000000000000..7a337cf22bee --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/ExtractRowLineage.java @@ -0,0 +1,92 @@ +/* + * 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.function.Function; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.ProjectingInternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.types.LongType$; +import org.apache.spark.sql.types.StructType; +import scala.collection.JavaConverters; + +class ExtractRowLineage implements Function { + private static final StructType ROW_LINEAGE_SCHEMA = + new StructType() + .add(MetadataColumns.ROW_ID.name(), LongType$.MODULE$, true) + .add(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name(), LongType$.MODULE$, true); + private static final InternalRow EMPTY_LINEAGE_ROW = new GenericInternalRow(2); + + private final boolean rowLineageRequired; + + private ProjectingInternalRow cachedRowLineageProjection; + + ExtractRowLineage(Schema writeSchema) { + Preconditions.checkArgument(writeSchema != null, "Write schema cannot be null"); + this.rowLineageRequired = writeSchema.findField(MetadataColumns.ROW_ID.name()) != null; + } + + @Override + public InternalRow apply(InternalRow meta) { + // If row lineage is not required on write return a null row + if (!rowLineageRequired) { + return null; + } + + // If metadata row is null but the write schema requires lineage, return an empty lineage row + if (meta == null) { + return EMPTY_LINEAGE_ROW; + } + + ProjectingInternalRow metaProj = (ProjectingInternalRow) meta; + // Use cached ordinals if they exist + if (cachedRowLineageProjection == null) { + this.cachedRowLineageProjection = rowLineageProjection(metaProj); + } + + cachedRowLineageProjection.project(metaProj); + return cachedRowLineageProjection; + } + + private ProjectingInternalRow rowLineageProjection(ProjectingInternalRow metadataRow) { + Integer rowIdOrdinal = null; + Integer lastUpdatedOrdinal = null; + for (int i = 0; i < metadataRow.numFields(); i++) { + String fieldName = metadataRow.schema().fields()[i].name(); + if (fieldName.equals(MetadataColumns.ROW_ID.name())) { + rowIdOrdinal = i; + } else if (fieldName.equals(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name())) { + lastUpdatedOrdinal = i; + } + } + + Preconditions.checkArgument(rowIdOrdinal != null, "Expected to find row ID in metadata row"); + Preconditions.checkArgument( + lastUpdatedOrdinal != null, + "Expected to find last updated sequence number in metadata row"); + List rowLineageProjectionOrdinals = ImmutableList.of(rowIdOrdinal, lastUpdatedOrdinal); + return new ProjectingInternalRow( + ROW_LINEAGE_SCHEMA, JavaConverters.asScala(rowLineageProjectionOrdinals).toIndexedSeq()); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/HasIcebergCatalog.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/HasIcebergCatalog.java new file mode 100644 index 000000000000..37e0c4dfcdb6 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/HasIcebergCatalog.java @@ -0,0 +1,30 @@ +/* + * 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.catalog.Catalog; +import org.apache.spark.sql.connector.catalog.TableCatalog; + +public interface HasIcebergCatalog extends TableCatalog { + + /** + * Returns the underlying {@link org.apache.iceberg.catalog.Catalog} backing this Spark Catalog + */ + Catalog icebergCatalog(); +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java new file mode 100644 index 000000000000..45d842f1c36a --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java @@ -0,0 +1,265 @@ +/* + * 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.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.SparkCachedTableCatalog; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.spark.SparkTableCache; +import org.apache.iceberg.spark.SparkWriteOptions; +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; +import org.apache.spark.sql.connector.catalog.CatalogPlugin; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.SessionConfigSupport; +import org.apache.spark.sql.connector.catalog.SupportsCatalogOptions; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.sources.DataSourceRegister; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * The IcebergSource loads/writes tables with format "iceberg". It can load paths and tables. + * + *

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

    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. + */ +public class IcebergSource + implements DataSourceRegister, SupportsCatalogOptions, SessionConfigSupport { + private static final String DEFAULT_CATALOG_NAME = "default_iceberg"; + private static final String DEFAULT_CACHE_CATALOG_NAME = "default_cache_iceberg"; + private static final String DEFAULT_CATALOG = "spark.sql.catalog." + DEFAULT_CATALOG_NAME; + private static final String DEFAULT_CACHE_CATALOG = + "spark.sql.catalog." + DEFAULT_CACHE_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 REWRITE_SELECTOR = "rewrite"; + private static final String[] EMPTY_NAMESPACE = new String[0]; + + private static final SparkTableCache TABLE_CACHE = SparkTableCache.get(); + + @Override + public String shortName() { + return "iceberg"; + } + + @Override + public String keyPrefix() { + return shortName(); + } + + @Override + public StructType inferSchema(CaseInsensitiveStringMap options) { + return null; + } + + @Override + public Transform[] inferPartitioning(CaseInsensitiveStringMap options) { + return getTable(null, null, options).partitioning(); + } + + @Override + public boolean supportsExternalMetadata() { + return true; + } + + @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(); + + try { + if (catalog instanceof TableCatalog) { + return ((TableCatalog) catalog).loadTable(ident); + } + } catch (NoSuchTableException e) { + // throwing an iceberg NoSuchTableException because the Spark one is typed and cant be thrown + // from this interface + throw new org.apache.iceberg.exceptions.NoSuchTableException( + e, "Cannot find table for %s.", ident); + } + + // throwing an iceberg NoSuchTableException because the Spark one is typed and cant be thrown + // from this interface + throw new org.apache.iceberg.exceptions.NoSuchTableException( + "Cannot find table for %s.", ident); + } + + private Spark3Util.CatalogAndIdentifier catalogAndIdentifier(CaseInsensitiveStringMap options) { + Preconditions.checkArgument( + options.containsKey(SparkReadOptions.PATH), "Cannot open table: path is not set"); + SparkSession spark = SparkSession.active(); + setupDefaultSparkCatalogs(spark); + String path = options.get(SparkReadOptions.PATH); + + Long snapshotId = propertyAsLong(options, SparkReadOptions.SNAPSHOT_ID); + Long asOfTimestamp = propertyAsLong(options, SparkReadOptions.AS_OF_TIMESTAMP); + 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; + } + + String groupId = + options.getOrDefault( + SparkReadOptions.SCAN_TASK_SET_ID, + options.get(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID)); + if (groupId != null) { + selector = REWRITE_SELECTOR; + } + + CatalogManager catalogManager = spark.sessionState().catalogManager(); + + if (TABLE_CACHE.contains(path)) { + return new Spark3Util.CatalogAndIdentifier( + catalogManager.catalog(DEFAULT_CACHE_CATALOG_NAME), + Identifier.of(EMPTY_NAMESPACE, pathWithSelector(path, selector))); + } else if (path.contains("/")) { + // contains a path. Return iceberg default catalog and a PathIdentifier + return new Spark3Util.CatalogAndIdentifier( + catalogManager.catalog(DEFAULT_CATALOG_NAME), + new PathIdentifier(pathWithSelector(path, selector))); + } + + final Spark3Util.CatalogAndIdentifier catalogAndIdentifier = + Spark3Util.catalogAndIdentifier("path or identifier", spark, path); + + 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 String pathWithSelector(String path, String selector) { + return (selector == null) ? path : path + "#" + selector; + } + + private Identifier identifierWithSelector(Identifier ident, String selector) { + if (selector == null) { + return ident; + } else { + String[] namespace = ident.namespace(); + String[] ns = Arrays.copyOf(namespace, namespace.length + 1); + ns[namespace.length] = ident.name(); + return Identifier.of(ns, selector); + } + } + + @Override + public Identifier extractIdentifier(CaseInsensitiveStringMap options) { + return catalogAndIdentifier(options).identifier(); + } + + @Override + public String extractCatalog(CaseInsensitiveStringMap options) { + return catalogAndIdentifier(options).catalog().name(); + } + + @Override + public Optional extractTimeTravelVersion(CaseInsensitiveStringMap options) { + return Optional.ofNullable( + PropertyUtil.propertyAsString(options, SparkReadOptions.VERSION_AS_OF, null)); + } + + @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; + } + + private static void setupDefaultSparkCatalogs(SparkSession spark) { + if (spark.conf().getOption(DEFAULT_CATALOG).isEmpty()) { + ImmutableMap config = + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "cache-enabled", "false" // the source should not use a cache + ); + spark.conf().set(DEFAULT_CATALOG, SparkCatalog.class.getName()); + config.forEach((key, value) -> spark.conf().set(DEFAULT_CATALOG + "." + key, value)); + } + + if (spark.conf().getOption(DEFAULT_CACHE_CATALOG).isEmpty()) { + spark.conf().set(DEFAULT_CACHE_CATALOG, SparkCachedTableCatalog.class.getName()); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java new file mode 100644 index 000000000000..d1682b8c85c1 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java @@ -0,0 +1,109 @@ +/* + * 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.nio.ByteBuffer; +import java.util.UUID; +import java.util.function.BiFunction; +import java.util.stream.Stream; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.BinaryType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +/** + * Class to adapt a Spark {@code InternalRow} to Iceberg {@link StructLike} for uses like {@link + * org.apache.iceberg.PartitionKey#partition(StructLike)} + */ +class InternalRowWrapper implements StructLike { + private final DataType[] types; + private final BiFunction[] getters; + private InternalRow row = null; + + @SuppressWarnings("unchecked") + InternalRowWrapper(StructType rowType, Types.StructType icebergSchema) { + this.types = Stream.of(rowType.fields()).map(StructField::dataType).toArray(DataType[]::new); + Preconditions.checkArgument( + types.length == icebergSchema.fields().size(), + "Invalid length: Spark struct type (%s) != Iceberg struct type (%s)", + types.length, + icebergSchema.fields().size()); + this.getters = new BiFunction[types.length]; + for (int i = 0; i < types.length; i++) { + getters[i] = getter(icebergSchema.fields().get(i).type(), types[i]); + } + } + + InternalRowWrapper wrap(InternalRow internalRow) { + this.row = internalRow; + return this; + } + + @Override + public int size() { + return types.length; + } + + @Override + public T get(int pos, Class javaClass) { + if (row.isNullAt(pos)) { + return null; + } else if (getters[pos] != null) { + return javaClass.cast(getters[pos].apply(row, pos)); + } + + return javaClass.cast(row.get(pos, types[pos])); + } + + @Override + public void set(int pos, T value) { + row.update(pos, value); + } + + private static BiFunction getter(Type icebergType, DataType type) { + if (type instanceof StringType) { + // Spark represents UUIDs as strings + if (Type.TypeID.UUID == icebergType.typeId()) { + return (row, pos) -> UUID.fromString(row.getUTF8String(pos).toString()); + } + + return (row, pos) -> row.getUTF8String(pos).toString(); + } else if (type instanceof DecimalType) { + DecimalType decimal = (DecimalType) type; + return (row, pos) -> + row.getDecimal(pos, decimal.precision(), decimal.scale()).toJavaBigDecimal(); + } else if (type instanceof BinaryType) { + return (row, pos) -> ByteBuffer.wrap(row.getBinary(pos)); + } else if (type instanceof StructType) { + StructType structType = (StructType) type; + InternalRowWrapper nestedWrapper = + new InternalRowWrapper(structType, icebergType.asStructType()); + return (row, pos) -> nestedWrapper.wrap(row.getStruct(pos, structType.size())); + } + + return null; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java new file mode 100644 index 000000000000..8ad1f3ad397b --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/PositionDeletesRowReader.java @@ -0,0 +1,120 @@ +/* + * 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.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.PositionDeletesScanTask; +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.expressions.ExpressionUtil; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.primitives.Ints; +import org.apache.iceberg.util.ContentFileUtil; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.rdd.InputFileBlockHolder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class PositionDeletesRowReader extends BaseRowReader + implements PartitionReader { + + private static final Logger LOG = LoggerFactory.getLogger(PositionDeletesRowReader.class); + + PositionDeletesRowReader(SparkInputPartition partition) { + this( + partition.table(), + partition.taskGroup(), + SnapshotUtil.schemaFor(partition.table(), partition.branch()), + partition.expectedSchema(), + partition.isCaseSensitive(), + partition.cacheDeleteFilesOnExecutors()); + } + + PositionDeletesRowReader( + Table table, + ScanTaskGroup taskGroup, + Schema tableSchema, + Schema expectedSchema, + boolean caseSensitive, + boolean cacheDeleteFilesOnExecutors) { + + super( + table, taskGroup, tableSchema, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors); + + int numSplits = taskGroup.tasks().size(); + LOG.debug("Reading {} position delete file split(s) for table {}", numSplits, table.name()); + } + + @Override + protected Stream> referencedFiles(PositionDeletesScanTask task) { + return Stream.of(task.file()); + } + + @SuppressWarnings("resource") // handled by BaseReader + @Override + protected CloseableIterator open(PositionDeletesScanTask task) { + String filePath = task.file().location(); + LOG.debug("Opening position delete file {}", filePath); + + // update the current file for Spark's filename() function + InputFileBlockHolder.set(filePath, task.start(), task.length()); + + InputFile inputFile = getInputFile(task.file().location()); + Preconditions.checkNotNull(inputFile, "Could not find InputFile associated with %s", task); + + // select out constant fields when pushing down filter to row reader + Map idToConstant = constantsMap(task, expectedSchema()); + Set nonConstantFieldIds = nonConstantFieldIds(idToConstant); + Expression residualWithoutConstants = + ExpressionUtil.extractByIdInclusive( + task.residual(), expectedSchema(), caseSensitive(), Ints.toArray(nonConstantFieldIds)); + + if (ContentFileUtil.isDV(task.file())) { + return new DVIterator(inputFile, task.file(), expectedSchema(), idToConstant); + } + + return newIterable( + inputFile, + task.file().format(), + task.start(), + task.length(), + residualWithoutConstants, + expectedSchema(), + idToConstant) + .iterator(); + } + + private Set nonConstantFieldIds(Map idToConstant) { + Set fields = expectedSchema().idToName().keySet(); + return fields.stream() + .filter(id -> expectedSchema().findField(id).type().isPrimitiveType()) + .filter(id -> !idToConstant.containsKey(id)) + .collect(Collectors.toSet()); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java new file mode 100644 index 000000000000..b2b3c7856389 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -0,0 +1,125 @@ +/* + * 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.Map; +import java.util.stream.Stream; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.source.metrics.TaskNumDeletes; +import org.apache.iceberg.spark.source.metrics.TaskNumSplits; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.rdd.InputFileBlockHolder; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class RowDataReader extends BaseRowReader implements PartitionReader { + private static final Logger LOG = LoggerFactory.getLogger(RowDataReader.class); + + private final long numSplits; + + RowDataReader(SparkInputPartition partition) { + this( + partition.table(), + partition.taskGroup(), + SnapshotUtil.schemaFor(partition.table(), partition.branch()), + partition.expectedSchema(), + partition.isCaseSensitive(), + partition.cacheDeleteFilesOnExecutors()); + } + + RowDataReader( + Table table, + ScanTaskGroup taskGroup, + Schema tableSchema, + Schema expectedSchema, + boolean caseSensitive, + boolean cacheDeleteFilesOnExecutors) { + + super( + table, taskGroup, tableSchema, expectedSchema, caseSensitive, cacheDeleteFilesOnExecutors); + + numSplits = taskGroup.tasks().size(); + LOG.debug("Reading {} file split(s) for table {}", numSplits, table.name()); + } + + @Override + public CustomTaskMetric[] currentMetricsValues() { + return new CustomTaskMetric[] { + new TaskNumSplits(numSplits), new TaskNumDeletes(counter().get()) + }; + } + + @Override + protected Stream> referencedFiles(FileScanTask task) { + return Stream.concat(Stream.of(task.file()), task.deletes().stream()); + } + + @Override + protected CloseableIterator open(FileScanTask task) { + String filePath = task.file().location(); + LOG.debug("Opening data file {}", filePath); + SparkDeleteFilter deleteFilter = + new SparkDeleteFilter(filePath, task.deletes(), counter(), true); + + // schema or rows returned by readers + Schema requiredSchema = deleteFilter.requiredSchema(); + Map idToConstant = constantsMap(task, requiredSchema); + + // update the current file for Spark's filename() function + InputFileBlockHolder.set(filePath, task.start(), task.length()); + + return deleteFilter.filter(open(task, requiredSchema, idToConstant)).iterator(); + } + + protected CloseableIterable open( + FileScanTask task, Schema readSchema, Map idToConstant) { + if (task.isDataTask()) { + return newDataIterable(task.asDataTask(), readSchema); + } else { + InputFile inputFile = getInputFile(task.file().location()); + Preconditions.checkNotNull( + inputFile, "Could not find InputFile associated with FileScanTask"); + return newIterable( + inputFile, + task.file().format(), + task.start(), + task.length(), + task.residual(), + readSchema, + idToConstant); + } + } + + private CloseableIterable newDataIterable(DataTask task, Schema readSchema) { + StructInternalRow row = new StructInternalRow(readSchema.asStruct()); + return CloseableIterable.transform(task.asDataTask().rows(), row::setStruct); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SerializableTableWithSize.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SerializableTableWithSize.java new file mode 100644 index 000000000000..f6913fb9d00d --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SerializableTableWithSize.java @@ -0,0 +1,108 @@ +/* + * 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.BaseMetadataTable; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.spark.SparkExecutorCache; +import org.apache.spark.util.KnownSizeEstimation; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class provides a serializable table with a known size estimate. Spark calls its + * SizeEstimator class when broadcasting variables and this can be an expensive operation, so + * providing a known size estimate allows that operation to be skipped. + * + *

    This class also implements AutoCloseable to avoid leaking resources upon broadcasting. + * Broadcast variables are destroyed and cleaned up on the driver and executors once they are + * garbage collected on the driver. The implementation ensures only resources used by copies of the + * main table are released. + */ +public class SerializableTableWithSize extends SerializableTable + implements KnownSizeEstimation, AutoCloseable { + + private static final Logger LOG = LoggerFactory.getLogger(SerializableTableWithSize.class); + private static final long SIZE_ESTIMATE = 32_768L; + + private final transient Object serializationMarker; + + protected SerializableTableWithSize(Table table) { + super(table); + this.serializationMarker = new Object(); + } + + @Override + public long estimatedSize() { + return SIZE_ESTIMATE; + } + + public static Table copyOf(Table table) { + if (table instanceof BaseMetadataTable) { + return new SerializableMetadataTableWithSize((BaseMetadataTable) table); + } else { + return new SerializableTableWithSize(table); + } + } + + @Override + public void close() throws Exception { + if (serializationMarker == null) { + LOG.info("Releasing resources"); + io().close(); + } + invalidateCache(name()); + } + + public static class SerializableMetadataTableWithSize extends SerializableMetadataTable + implements KnownSizeEstimation, AutoCloseable { + + private static final Logger LOG = + LoggerFactory.getLogger(SerializableMetadataTableWithSize.class); + + private final transient Object serializationMarker; + + protected SerializableMetadataTableWithSize(BaseMetadataTable metadataTable) { + super(metadataTable); + this.serializationMarker = new Object(); + } + + @Override + public long estimatedSize() { + return SIZE_ESTIMATE; + } + + @Override + public void close() throws Exception { + if (serializationMarker == null) { + LOG.info("Releasing resources"); + io().close(); + } + invalidateCache(name()); + } + } + + private static void invalidateCache(String name) { + SparkExecutorCache cache = SparkExecutorCache.get(); + if (cache != null) { + cache.invalidate(name); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java new file mode 100644 index 000000000000..0626d0b43985 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java @@ -0,0 +1,232 @@ +/* + * 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 org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.Table; +import org.apache.iceberg.spark.ImmutableOrcBatchReadConf; +import org.apache.iceberg.spark.ImmutableParquetBatchReadConf; +import org.apache.iceberg.spark.OrcBatchReadConf; +import org.apache.iceberg.spark.ParquetBatchReadConf; +import org.apache.iceberg.spark.ParquetReaderType; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.SparkUtil; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.connector.read.Batch; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; + +class SparkBatch implements Batch { + + private final JavaSparkContext sparkContext; + private final Table table; + private final String branch; + private final SparkReadConf readConf; + private final Types.StructType groupingKeyType; + private final List> taskGroups; + private final Schema expectedSchema; + private final boolean caseSensitive; + private final boolean localityEnabled; + private final boolean executorCacheLocalityEnabled; + private final int scanHashCode; + private final boolean cacheDeleteFilesOnExecutors; + + SparkBatch( + JavaSparkContext sparkContext, + Table table, + SparkReadConf readConf, + Types.StructType groupingKeyType, + List> taskGroups, + Schema expectedSchema, + int scanHashCode) { + this.sparkContext = sparkContext; + this.table = table; + this.branch = readConf.branch(); + this.readConf = readConf; + this.groupingKeyType = groupingKeyType; + this.taskGroups = taskGroups; + this.expectedSchema = expectedSchema; + this.caseSensitive = readConf.caseSensitive(); + this.localityEnabled = readConf.localityEnabled(); + this.executorCacheLocalityEnabled = readConf.executorCacheLocalityEnabled(); + this.scanHashCode = scanHashCode; + this.cacheDeleteFilesOnExecutors = readConf.cacheDeleteFilesOnExecutors(); + } + + @Override + 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[][] locations = computePreferredLocations(); + + InputPartition[] partitions = new InputPartition[taskGroups.size()]; + + for (int index = 0; index < taskGroups.size(); index++) { + partitions[index] = + new SparkInputPartition( + groupingKeyType, + taskGroups.get(index), + tableBroadcast, + branch, + expectedSchemaString, + caseSensitive, + locations != null ? locations[index] : SparkPlanningUtil.NO_LOCATION_PREFERENCE, + cacheDeleteFilesOnExecutors); + } + + return partitions; + } + + private String[][] computePreferredLocations() { + if (localityEnabled) { + return SparkPlanningUtil.fetchBlockLocations(table.io(), taskGroups); + + } else if (executorCacheLocalityEnabled) { + List executorLocations = SparkUtil.executorLocations(); + if (!executorLocations.isEmpty()) { + return SparkPlanningUtil.assignExecutors(taskGroups, executorLocations); + } + } + + return null; + } + + @Override + public PartitionReaderFactory createReaderFactory() { + if (useCometBatchReads()) { + return new SparkColumnarReaderFactory(parquetBatchReadConf(ParquetReaderType.COMET)); + + } else if (useParquetBatchReads()) { + return new SparkColumnarReaderFactory(parquetBatchReadConf(ParquetReaderType.ICEBERG)); + + } else if (useOrcBatchReads()) { + return new SparkColumnarReaderFactory(orcBatchReadConf()); + + } else { + return new SparkRowReaderFactory(); + } + } + + private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) { + return ImmutableParquetBatchReadConf.builder() + .batchSize(readConf.parquetBatchSize()) + .readerType(readerType) + .build(); + } + + private OrcBatchReadConf orcBatchReadConf() { + return ImmutableOrcBatchReadConf.builder().batchSize(readConf.orcBatchSize()).build(); + } + + // conditions for using Parquet batch reads: + // - Parquet vectorization is enabled + // - only primitives or metadata columns are projected + // - all tasks are of FileScanTask type and read only Parquet files + private boolean useParquetBatchReads() { + return readConf.parquetVectorizationEnabled() + && expectedSchema.columns().stream().allMatch(this::supportsParquetBatchReads) + && taskGroups.stream().allMatch(this::supportsParquetBatchReads); + } + + private boolean supportsParquetBatchReads(ScanTask task) { + if (task instanceof ScanTaskGroup) { + ScanTaskGroup taskGroup = (ScanTaskGroup) task; + return taskGroup.tasks().stream().allMatch(this::supportsParquetBatchReads); + + } else if (task.isFileScanTask() && !task.isDataTask()) { + FileScanTask fileScanTask = task.asFileScanTask(); + return fileScanTask.file().format() == FileFormat.PARQUET; + + } else { + return false; + } + } + + private boolean supportsParquetBatchReads(Types.NestedField field) { + return field.type().isPrimitiveType() || MetadataColumns.isMetadataColumn(field.fieldId()); + } + + private boolean useCometBatchReads() { + return readConf.parquetVectorizationEnabled() + && readConf.parquetReaderType() == ParquetReaderType.COMET + && expectedSchema.columns().stream().allMatch(this::supportsCometBatchReads) + && taskGroups.stream().allMatch(this::supportsParquetBatchReads); + } + + private boolean supportsCometBatchReads(Types.NestedField field) { + return field.type().isPrimitiveType() + && !field.type().typeId().equals(Type.TypeID.UUID) + && field.fieldId() != MetadataColumns.ROW_ID.fieldId() + && field.fieldId() != MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId(); + } + + // conditions for using ORC batch reads: + // - ORC vectorization is enabled + // - all tasks are of type FileScanTask and read only ORC files with no delete files + private boolean useOrcBatchReads() { + return readConf.orcVectorizationEnabled() + && taskGroups.stream().allMatch(this::supportsOrcBatchReads); + } + + private boolean supportsOrcBatchReads(ScanTask task) { + if (task instanceof ScanTaskGroup) { + ScanTaskGroup taskGroup = (ScanTaskGroup) task; + return taskGroup.tasks().stream().allMatch(this::supportsOrcBatchReads); + + } else if (task.isFileScanTask() && !task.isDataTask()) { + FileScanTask fileScanTask = task.asFileScanTask(); + return fileScanTask.file().format() == FileFormat.ORC && fileScanTask.deletes().isEmpty(); + + } else { + return false; + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + SparkBatch that = (SparkBatch) o; + return table.name().equals(that.table.name()) && scanHashCode == that.scanHashCode; + } + + @Override + public int hashCode() { + return Objects.hash(table.name(), scanHashCode); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java new file mode 100644 index 000000000000..a361a7f1bae8 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatchQueryScan.java @@ -0,0 +1,293 @@ +/* + * 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.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 { + + 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; + + SparkBatchQueryScan( + SparkSession spark, + Table table, + Scan> scan, + SparkReadConf readConf, + Schema expectedSchema, + 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(); + } + + 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; + } + + @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); + } + } + + @Override + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + SparkBatchQueryScan that = (SparkBatchQueryScan) o; + return table().name().equals(that.table().name()) + && Objects.equals(branch(), that.branch()) + && readSchema().equals(that.readSchema()) // compare Spark schemas to ignore field ids + && filterExpressions().toString().equals(that.filterExpressions().toString()) + && runtimeFilterExpressions.toString().equals(that.runtimeFilterExpressions.toString()) + && 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); + } + + @Override + public int hashCode() { + return Objects.hash( + table().name(), + branch(), + readSchema(), + filterExpressions().toString(), + runtimeFilterExpressions.toString(), + snapshotId, + startSnapshotId, + endSnapshotId, + asOfTimestamp, + tag); + } + + @Override + public String toString() { + return String.format( + "IcebergScan(table=%s, branch=%s, type=%s, filters=%s, runtimeFilters=%s, caseSensitive=%s)", + table(), + branch(), + expectedSchema().asStruct(), + filterExpressions(), + runtimeFilterExpressions, + caseSensitive()); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java new file mode 100644 index 000000000000..55ea137ca1b0 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.Objects; +import org.apache.iceberg.ChangelogScanTask; +import org.apache.iceberg.IncrementalChangelogScan; +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.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.read.Batch; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.connector.read.Statistics; +import org.apache.spark.sql.connector.read.SupportsReportStatistics; +import org.apache.spark.sql.types.StructType; + +class SparkChangelogScan implements Scan, SupportsReportStatistics { + + private static final Types.StructType EMPTY_GROUPING_KEY_TYPE = Types.StructType.of(); + + private final JavaSparkContext sparkContext; + private final Table table; + private final IncrementalChangelogScan scan; + private final SparkReadConf readConf; + private final Schema expectedSchema; + private final List filters; + private final Long startSnapshotId; + private final Long endSnapshotId; + + // lazy variables + private List> taskGroups = null; + private StructType expectedSparkType = null; + + SparkChangelogScan( + SparkSession spark, + Table table, + IncrementalChangelogScan scan, + SparkReadConf readConf, + Schema expectedSchema, + List filters, + boolean emptyScan) { + + SparkSchemaUtil.validateMetadataColumnReferences(table.schema(), expectedSchema); + + this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + this.table = table; + this.scan = scan; + this.readConf = readConf; + this.expectedSchema = expectedSchema; + this.filters = filters != null ? filters : Collections.emptyList(); + this.startSnapshotId = readConf.startSnapshotId(); + this.endSnapshotId = readConf.endSnapshotId(); + if (emptyScan) { + this.taskGroups = Collections.emptyList(); + } + } + + @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 + public StructType readSchema() { + if (expectedSparkType == null) { + this.expectedSparkType = SparkSchemaUtil.convert(expectedSchema); + } + + return expectedSparkType; + } + + @Override + public Batch toBatch() { + return new SparkBatch( + sparkContext, + table, + readConf, + EMPTY_GROUPING_KEY_TYPE, + taskGroups(), + expectedSchema, + hashCode()); + } + + private List> taskGroups() { + if (taskGroups == null) { + try (CloseableIterable> groups = scan.planTasks()) { + this.taskGroups = Lists.newArrayList(groups); + } catch (IOException e) { + throw new UncheckedIOException("Failed to close changelog scan: " + scan, e); + } + } + + return taskGroups; + } + + @Override + public String description() { + return String.format( + Locale.ROOT, + "%s [fromSnapshotId=%d, toSnapshotId=%d, filters=%s]", + table, + startSnapshotId, + endSnapshotId, + Spark3Util.describe(filters)); + } + + @Override + public String toString() { + return String.format( + Locale.ROOT, + "IcebergChangelogScan(table=%s, type=%s, fromSnapshotId=%d, toSnapshotId=%d, filters=%s)", + table, + expectedSchema.asStruct(), + startSnapshotId, + endSnapshotId, + Spark3Util.describe(filters)); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + SparkChangelogScan that = (SparkChangelogScan) o; + return table.name().equals(that.table.name()) + && readSchema().equals(that.readSchema()) // compare Spark schemas to ignore field IDs + && filters.toString().equals(that.filters.toString()) + && Objects.equals(startSnapshotId, that.startSnapshotId) + && Objects.equals(endSnapshotId, that.endSnapshotId); + } + + @Override + public int hashCode() { + return Objects.hash( + table.name(), readSchema(), filters.toString(), startSnapshotId, endSnapshotId); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogTable.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogTable.java new file mode 100644 index 000000000000..450b1aa37024 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogTable.java @@ -0,0 +1,140 @@ +/* + * 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.Set; +import org.apache.iceberg.ChangelogUtil; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.Schema; +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.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +public class SparkChangelogTable implements 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 SparkSession lazySpark = null; + private StructType lazyTableSparkType = null; + private Schema lazyChangelogSchema = null; + + public SparkChangelogTable(org.apache.iceberg.Table icebergTable, boolean refreshEagerly) { + this.icebergTable = icebergTable; + this.refreshEagerly = refreshEagerly; + } + + @Override + public String name() { + return icebergTable.name() + "." + TABLE_NAME; + } + + @Override + public StructType schema() { + if (lazyTableSparkType == null) { + this.lazyTableSparkType = SparkSchemaUtil.convert(changelogSchema()); + } + + return lazyTableSparkType; + } + + @Override + public Set capabilities() { + return 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; + } + + private SparkSession spark() { + if (lazySpark == null) { + this.lazySpark = SparkSession.active(); + } + + return lazySpark; + } + + @Override + public MetadataColumn[] metadataColumns() { + DataType sparkPartitionType = SparkSchemaUtil.convert(Partitioning.partitionType(icebergTable)); + return new MetadataColumn[] { + SparkMetadataColumn.builder() + .name(MetadataColumns.SPEC_ID.name()) + .dataType(DataTypes.IntegerType) + .withNullability(true) + .build(), + SparkMetadataColumn.builder() + .name(MetadataColumns.PARTITION_COLUMN_NAME) + .dataType(sparkPartitionType) + .withNullability(true) + .build(), + SparkMetadataColumn.builder() + .name(MetadataColumns.FILE_PATH.name()) + .dataType(DataTypes.StringType) + .withNullability(false) + .build(), + SparkMetadataColumn.builder() + .name(MetadataColumns.ROW_POSITION.name()) + .dataType(DataTypes.LongType) + .withNullability(false) + .build(), + SparkMetadataColumn.builder() + .name(MetadataColumns.IS_DELETED.name()) + .dataType(DataTypes.BooleanType) + .withNullability(false) + .build(), + }; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java new file mode 100644 index 000000000000..93b27c566c9f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCleanupUtil.java @@ -0,0 +1,141 @@ +/* + * 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.Locale; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.exceptions.NotFoundException; +import org.apache.iceberg.io.BulkDeletionFailureException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.SupportsBulkOperations; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.util.ThreadPools; +import org.apache.spark.TaskContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** A utility for cleaning up written but not committed files. */ +class SparkCleanupUtil { + + private static final Logger LOG = LoggerFactory.getLogger(SparkCleanupUtil.class); + + private static final int DELETE_NUM_RETRIES = 3; + private static final int DELETE_MIN_RETRY_WAIT_MS = 100; // 100 ms + private static final int DELETE_MAX_RETRY_WAIT_MS = 30 * 1000; // 30 seconds + private static final int DELETE_TOTAL_RETRY_TIME_MS = 2 * 60 * 1000; // 2 minutes + + private SparkCleanupUtil() {} + + /** + * Attempts to delete as many files produced by a task as possible. + * + *

    Note this method will log Spark task info and is supposed to be called only on executors. + * Use {@link #deleteFiles(String, FileIO, List)} to delete files on the driver. + * + * @param io a {@link FileIO} instance used for deleting files + * @param files a list of files to delete + */ + public static void deleteTaskFiles(FileIO io, List> files) { + deleteFiles(taskInfo(), io, files); + } + + // the format matches what Spark uses for internal logging + private static String taskInfo() { + TaskContext taskContext = TaskContext.get(); + if (taskContext == null) { + return "unknown task"; + } else { + return String.format( + Locale.ROOT, + "partition %d (task %d, attempt %d, stage %d.%d)", + taskContext.partitionId(), + taskContext.taskAttemptId(), + taskContext.attemptNumber(), + taskContext.stageId(), + taskContext.stageAttemptNumber()); + } + } + + /** + * Attempts to delete as many given files as possible. + * + * @param context a helpful description of the operation invoking this method + * @param io a {@link FileIO} instance used for deleting files + * @param files a list of files to delete + */ + public static void deleteFiles(String context, FileIO io, List> files) { + List paths = Lists.transform(files, ContentFile::location); + deletePaths(context, io, paths); + } + + private static void deletePaths(String context, FileIO io, List paths) { + if (io instanceof SupportsBulkOperations) { + SupportsBulkOperations bulkIO = (SupportsBulkOperations) io; + bulkDelete(context, bulkIO, paths); + } else { + delete(context, io, paths); + } + } + + private static void bulkDelete(String context, SupportsBulkOperations io, List paths) { + try { + io.deleteFiles(paths); + LOG.info("Deleted {} file(s) using bulk deletes ({})", paths.size(), context); + + } catch (BulkDeletionFailureException e) { + int deletedFilesCount = paths.size() - e.numberFailedObjects(); + LOG.warn( + "Deleted only {} of {} file(s) using bulk deletes ({})", + deletedFilesCount, + paths.size(), + context, + e); + } + } + + private static void delete(String context, FileIO io, List paths) { + AtomicInteger deletedFilesCount = new AtomicInteger(0); + + Tasks.foreach(paths) + .executeWith(ThreadPools.getWorkerPool()) + .stopRetryOn(NotFoundException.class) + .suppressFailureWhenFinished() + .onFailure((path, exc) -> LOG.warn("Failed to delete {} ({})", path, context, exc)) + .retry(DELETE_NUM_RETRIES) + .exponentialBackoff( + DELETE_MIN_RETRY_WAIT_MS, + DELETE_MAX_RETRY_WAIT_MS, + DELETE_TOTAL_RETRY_TIME_MS, + 2 /* exponential */) + .run( + path -> { + io.deleteFile(path); + deletedFilesCount.incrementAndGet(); + }); + + if (deletedFilesCount.get() < paths.size()) { + LOG.warn("Deleted only {} of {} file(s) ({})", deletedFilesCount, paths.size(), context); + } else { + LOG.info("Deleted {} file(s) ({})", paths.size(), context); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java new file mode 100644 index 000000000000..faaff3631d7c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java @@ -0,0 +1,88 @@ +/* + * 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.Optional; +import java.util.OptionalLong; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; +import org.apache.spark.sql.connector.read.colstats.Histogram; + +class SparkColumnStatistics implements ColumnStatistics { + + private final OptionalLong distinctCount; + private final Optional min; + private final Optional max; + private final OptionalLong nullCount; + private final OptionalLong avgLen; + private final OptionalLong maxLen; + private final Optional histogram; + + SparkColumnStatistics( + Long distinctCount, + Object min, + Object max, + Long nullCount, + Long avgLen, + Long maxLen, + Histogram histogram) { + this.distinctCount = + (distinctCount == null) ? OptionalLong.empty() : OptionalLong.of(distinctCount); + this.min = Optional.ofNullable(min); + this.max = Optional.ofNullable(max); + this.nullCount = (nullCount == null) ? OptionalLong.empty() : OptionalLong.of(nullCount); + this.avgLen = (avgLen == null) ? OptionalLong.empty() : OptionalLong.of(avgLen); + this.maxLen = (maxLen == null) ? OptionalLong.empty() : OptionalLong.of(maxLen); + this.histogram = Optional.ofNullable(histogram); + } + + @Override + public OptionalLong distinctCount() { + return distinctCount; + } + + @Override + public Optional min() { + return min; + } + + @Override + public Optional max() { + return max; + } + + @Override + public OptionalLong nullCount() { + return nullCount; + } + + @Override + public OptionalLong avgLen() { + return avgLen; + } + + @Override + public OptionalLong maxLen() { + return maxLen; + } + + @Override + public Optional histogram() { + return histogram; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnarReaderFactory.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnarReaderFactory.java new file mode 100644 index 000000000000..887b84fb617a --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnarReaderFactory.java @@ -0,0 +1,71 @@ +/* + * 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.FileScanTask; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.OrcBatchReadConf; +import org.apache.iceberg.spark.ParquetBatchReadConf; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +class SparkColumnarReaderFactory implements PartitionReaderFactory { + private final ParquetBatchReadConf parquetConf; + private final OrcBatchReadConf orcConf; + + SparkColumnarReaderFactory(ParquetBatchReadConf conf) { + this.parquetConf = conf; + this.orcConf = null; + } + + SparkColumnarReaderFactory(OrcBatchReadConf conf) { + this.orcConf = conf; + this.parquetConf = null; + } + + @Override + public PartitionReader createReader(InputPartition inputPartition) { + throw new UnsupportedOperationException("Row-based reads are not supported"); + } + + @Override + public PartitionReader createColumnarReader(InputPartition inputPartition) { + Preconditions.checkArgument( + inputPartition instanceof SparkInputPartition, + "Unknown input partition type: %s", + inputPartition.getClass().getName()); + + SparkInputPartition partition = (SparkInputPartition) inputPartition; + + if (partition.allTasksOfType(FileScanTask.class)) { + return new BatchDataReader(partition, parquetConf, orcConf); + } else { + throw new UnsupportedOperationException( + "Unsupported task group for columnar reads: " + partition.taskGroup()); + } + } + + @Override + public boolean supportColumnarReads(InputPartition inputPartition) { + return true; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java new file mode 100644 index 000000000000..dd3d785b3b5f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteOperation.java @@ -0,0 +1,115 @@ +/* + * 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.spark.sql.connector.write.RowLevelOperation.Command.DELETE; +import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.UPDATE; + +import java.util.List; +import org.apache.iceberg.IsolationLevel; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.expressions.Expressions; +import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.RowLevelOperation; +import org.apache.spark.sql.connector.write.RowLevelOperationInfo; +import org.apache.spark.sql.connector.write.WriteBuilder; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +class SparkCopyOnWriteOperation implements RowLevelOperation { + + private final SparkSession spark; + private final Table table; + private final String branch; + private final Command command; + private final IsolationLevel isolationLevel; + + // lazy vars + private ScanBuilder lazyScanBuilder; + private Scan configuredScan; + private WriteBuilder lazyWriteBuilder; + + SparkCopyOnWriteOperation( + SparkSession spark, + Table table, + String branch, + RowLevelOperationInfo info, + IsolationLevel isolationLevel) { + this.spark = spark; + this.table = table; + this.branch = branch; + this.command = info.command(); + this.isolationLevel = isolationLevel; + } + + @Override + public Command command() { + return command; + } + + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + if (lazyScanBuilder == null) { + lazyScanBuilder = + new SparkScanBuilder(spark, table, branch, options) { + @Override + public Scan build() { + Scan scan = super.buildCopyOnWriteScan(); + SparkCopyOnWriteOperation.this.configuredScan = scan; + return scan; + } + }; + } + + return lazyScanBuilder; + } + + @Override + public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { + if (lazyWriteBuilder == null) { + SparkWriteBuilder writeBuilder = new SparkWriteBuilder(spark, table, branch, info); + lazyWriteBuilder = writeBuilder.overwriteFiles(configuredScan, command, isolationLevel); + } + + return lazyWriteBuilder; + } + + @Override + public NamedReference[] requiredMetadataAttributes() { + List metadataAttributes = Lists.newArrayList(); + metadataAttributes.add(Expressions.column(MetadataColumns.FILE_PATH.name())); + if (command == DELETE || command == UPDATE) { + metadataAttributes.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())); + } + + return metadataAttributes.toArray(NamedReference[]::new); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java new file mode 100644 index 000000000000..ee4be2461894 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkCopyOnWriteScan.java @@ -0,0 +1,191 @@ +/* + * 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.Collections; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import org.apache.iceberg.BatchScan; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +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; +import org.apache.spark.sql.connector.read.Statistics; +import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering; +import org.apache.spark.sql.sources.Filter; +import org.apache.spark.sql.sources.In; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class SparkCopyOnWriteScan extends SparkPartitioningAwareScan + implements SupportsRuntimeFiltering { + + private static final Logger LOG = LoggerFactory.getLogger(SparkCopyOnWriteScan.class); + + private final Snapshot snapshot; + private Set filteredLocations = null; + + SparkCopyOnWriteScan( + SparkSession spark, + Table table, + SparkReadConf readConf, + Schema expectedSchema, + List filters, + Supplier scanReportSupplier) { + this(spark, table, null, null, readConf, expectedSchema, filters, scanReportSupplier); + } + + SparkCopyOnWriteScan( + SparkSession spark, + Table table, + BatchScan scan, + Snapshot snapshot, + SparkReadConf readConf, + Schema expectedSchema, + List filters, + Supplier scanReportSupplier) { + super(spark, table, scan, readConf, expectedSchema, filters, scanReportSupplier); + + this.snapshot = snapshot; + + if (scan == null) { + this.filteredLocations = Collections.emptySet(); + } + } + + Long snapshotId() { + return snapshot != null ? snapshot.snapshotId() : null; + } + + @Override + protected Class taskJavaClass() { + return FileScanTask.class; + } + + @Override + public Statistics estimateStatistics() { + return estimateStatistics(snapshot); + } + + @Override + public NamedReference[] filterAttributes() { + NamedReference file = Expressions.column(MetadataColumns.FILE_PATH.name()); + return new NamedReference[] {file}; + } + + @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 + && ((In) filter).attribute().equalsIgnoreCase(MetadataColumns.FILE_PATH.name())) { + In in = (In) filter; + + Set fileLocations = Sets.newHashSet(); + for (Object value : in.values()) { + fileLocations.add((String) value); + } + + // Spark may call this multiple times for UPDATEs with subqueries + // as such cases are rewritten using UNION and the same scan on both sides + // so filter files only if it is beneficial + if (filteredLocations == null || fileLocations.size() < filteredLocations.size()) { + this.filteredLocations = fileLocations; + List filteredTasks = + tasks().stream() + .filter(file -> fileLocations.contains(file.file().location())) + .collect(Collectors.toList()); + + LOG.info( + "{} of {} task(s) for table {} matched runtime file filter with {} location(s)", + filteredTasks.size(), + tasks().size(), + table().name(), + fileLocations.size()); + + resetTasks(filteredTasks); + } + } else { + LOG.warn("Unsupported runtime filter {}", filter); + } + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + SparkCopyOnWriteScan that = (SparkCopyOnWriteScan) o; + return table().name().equals(that.table().name()) + && readSchema().equals(that.readSchema()) // compare Spark schemas to ignore field ids + && filterExpressions().toString().equals(that.filterExpressions().toString()) + && Objects.equals(snapshotId(), that.snapshotId()) + && Objects.equals(filteredLocations, that.filteredLocations); + } + + @Override + public int hashCode() { + return Objects.hash( + table().name(), + readSchema(), + filterExpressions().toString(), + snapshotId(), + filteredLocations); + } + + @Override + public String toString() { + return String.format( + "IcebergCopyOnWriteScan(table=%s, type=%s, filters=%s, caseSensitive=%s)", + table(), expectedSchema().asStruct(), filterExpressions(), caseSensitive()); + } + + private Long currentSnapshotId() { + Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table(), branch()); + return currentSnapshot != null ? currentSnapshot.snapshotId() : null; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java new file mode 100644 index 000000000000..a93db17e4a0f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java @@ -0,0 +1,343 @@ +/* + * 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.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; +import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT; + +import java.util.Map; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.BaseFileWriterFactory; +import org.apache.iceberg.io.DeleteSchemaUtil; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +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.SparkSchemaUtil; +import org.apache.iceberg.spark.data.SparkAvroWriter; +import org.apache.iceberg.spark.data.SparkOrcWriter; +import org.apache.iceberg.spark.data.SparkParquetWriters; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +class SparkFileWriterFactory extends BaseFileWriterFactory { + private StructType dataSparkType; + private StructType equalityDeleteSparkType; + private StructType positionDeleteSparkType; + private final Map writeProperties; + + /** + * @deprecated This constructor is deprecated as of version 1.11.0 and will be removed in 1.12.0. + * Position deletes that include row data are no longer supported. Use {@link + * #SparkFileWriterFactory(Table, FileFormat, Schema, StructType, SortOrder, FileFormat, + * int[], Schema, StructType, SortOrder, Map)} instead. + */ + @Deprecated + SparkFileWriterFactory( + Table table, + FileFormat dataFileFormat, + Schema dataSchema, + StructType dataSparkType, + SortOrder dataSortOrder, + FileFormat deleteFileFormat, + int[] equalityFieldIds, + Schema equalityDeleteRowSchema, + StructType equalityDeleteSparkType, + SortOrder equalityDeleteSortOrder, + Schema positionDeleteRowSchema, + StructType positionDeleteSparkType, + Map writeProperties) { + + super( + table, + dataFileFormat, + dataSchema, + dataSortOrder, + deleteFileFormat, + equalityFieldIds, + equalityDeleteRowSchema, + equalityDeleteSortOrder, + positionDeleteRowSchema); + + this.dataSparkType = dataSparkType; + this.equalityDeleteSparkType = equalityDeleteSparkType; + this.positionDeleteSparkType = positionDeleteSparkType; + this.writeProperties = writeProperties != null ? writeProperties : ImmutableMap.of(); + } + + SparkFileWriterFactory( + Table table, + FileFormat dataFileFormat, + Schema dataSchema, + StructType dataSparkType, + SortOrder dataSortOrder, + FileFormat deleteFileFormat, + int[] equalityFieldIds, + Schema equalityDeleteRowSchema, + StructType equalityDeleteSparkType, + SortOrder equalityDeleteSortOrder, + Map writeProperties) { + + super( + table, + dataFileFormat, + dataSchema, + dataSortOrder, + deleteFileFormat, + equalityFieldIds, + equalityDeleteRowSchema, + equalityDeleteSortOrder, + ImmutableMap.of()); + + this.dataSparkType = dataSparkType; + this.equalityDeleteSparkType = equalityDeleteSparkType; + this.positionDeleteSparkType = null; + this.writeProperties = writeProperties != null ? writeProperties : ImmutableMap.of(); + } + + static Builder builderFor(Table table) { + return new Builder(table); + } + + @Override + protected void configureDataWrite(Avro.DataWriteBuilder builder) { + builder.createWriterFunc(ignored -> new SparkAvroWriter(dataSparkType())); + builder.setAll(writeProperties); + } + + @Override + protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) { + builder.createWriterFunc(ignored -> new SparkAvroWriter(equalityDeleteSparkType())); + builder.setAll(writeProperties); + } + + @Override + protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) { + boolean withRow = + positionDeleteSparkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME).isDefined(); + if (withRow) { + // SparkAvroWriter accepts just the Spark type of the row ignoring the path and pos + StructField rowField = positionDeleteSparkType().apply(DELETE_FILE_ROW_FIELD_NAME); + StructType positionDeleteRowSparkType = (StructType) rowField.dataType(); + builder.createWriterFunc(ignored -> new SparkAvroWriter(positionDeleteRowSparkType)); + } + + builder.setAll(writeProperties); + } + + @Override + protected void configureDataWrite(Parquet.DataWriteBuilder builder) { + builder.createWriterFunc(msgType -> SparkParquetWriters.buildWriter(dataSparkType(), msgType)); + builder.setAll(writeProperties); + } + + @Override + protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) { + builder.createWriterFunc( + msgType -> SparkParquetWriters.buildWriter(equalityDeleteSparkType(), msgType)); + builder.setAll(writeProperties); + } + + @Override + protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) { + builder.createWriterFunc( + msgType -> SparkParquetWriters.buildWriter(positionDeleteSparkType(), msgType)); + builder.transformPaths(path -> UTF8String.fromString(path.toString())); + builder.setAll(writeProperties); + } + + @Override + protected void configureDataWrite(ORC.DataWriteBuilder builder) { + builder.createWriterFunc(SparkOrcWriter::new); + builder.setAll(writeProperties); + } + + @Override + protected void configureEqualityDelete(ORC.DeleteWriteBuilder builder) { + builder.createWriterFunc(SparkOrcWriter::new); + builder.setAll(writeProperties); + } + + @Override + protected void configurePositionDelete(ORC.DeleteWriteBuilder builder) { + builder.createWriterFunc(SparkOrcWriter::new); + builder.transformPaths(path -> UTF8String.fromString(path.toString())); + builder.setAll(writeProperties); + } + + private StructType dataSparkType() { + if (dataSparkType == null) { + Preconditions.checkNotNull(dataSchema(), "Data schema must not be null"); + this.dataSparkType = SparkSchemaUtil.convert(dataSchema()); + } + + return dataSparkType; + } + + private StructType equalityDeleteSparkType() { + if (equalityDeleteSparkType == null) { + Preconditions.checkNotNull( + equalityDeleteRowSchema(), "Equality delete schema must not be null"); + this.equalityDeleteSparkType = SparkSchemaUtil.convert(equalityDeleteRowSchema()); + } + + return equalityDeleteSparkType; + } + + private StructType positionDeleteSparkType() { + if (positionDeleteSparkType == null) { + // wrap the optional row schema into the position delete schema containing path and position + Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema()); + this.positionDeleteSparkType = SparkSchemaUtil.convert(positionDeleteSchema); + } + + return positionDeleteSparkType; + } + + static class Builder { + private final Table table; + private FileFormat dataFileFormat; + private Schema dataSchema; + private StructType dataSparkType; + private SortOrder dataSortOrder; + private FileFormat deleteFileFormat; + private int[] equalityFieldIds; + private Schema equalityDeleteRowSchema; + private StructType equalityDeleteSparkType; + private SortOrder equalityDeleteSortOrder; + private Schema positionDeleteRowSchema; + private StructType positionDeleteSparkType; + private Map writeProperties; + + Builder(Table table) { + this.table = table; + + Map properties = table.properties(); + + String dataFileFormatName = + properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); + this.dataFileFormat = FileFormat.fromString(dataFileFormatName); + + String deleteFileFormatName = + properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName); + this.deleteFileFormat = FileFormat.fromString(deleteFileFormatName); + } + + Builder dataFileFormat(FileFormat newDataFileFormat) { + this.dataFileFormat = newDataFileFormat; + return this; + } + + Builder dataSchema(Schema newDataSchema) { + this.dataSchema = newDataSchema; + return this; + } + + Builder dataSparkType(StructType newDataSparkType) { + this.dataSparkType = newDataSparkType; + return this; + } + + Builder dataSortOrder(SortOrder newDataSortOrder) { + this.dataSortOrder = newDataSortOrder; + return this; + } + + Builder deleteFileFormat(FileFormat newDeleteFileFormat) { + this.deleteFileFormat = newDeleteFileFormat; + return this; + } + + Builder equalityFieldIds(int[] newEqualityFieldIds) { + this.equalityFieldIds = newEqualityFieldIds; + return this; + } + + Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) { + this.equalityDeleteRowSchema = newEqualityDeleteRowSchema; + return this; + } + + Builder equalityDeleteSparkType(StructType newEqualityDeleteSparkType) { + this.equalityDeleteSparkType = newEqualityDeleteSparkType; + return this; + } + + Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) { + this.equalityDeleteSortOrder = newEqualityDeleteSortOrder; + return this; + } + + /** + * @deprecated This method is deprecated as of version 1.11.0 and will be removed in 1.12.0. + * Position deletes that include row data are no longer supported. + */ + @Deprecated + Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) { + this.positionDeleteRowSchema = newPositionDeleteRowSchema; + return this; + } + + /** + * @deprecated This method is deprecated as of version 1.11.0 and will be removed in 1.12.0. + * Position deletes that include row data are no longer supported. + */ + @Deprecated + Builder positionDeleteSparkType(StructType newPositionDeleteSparkType) { + this.positionDeleteSparkType = newPositionDeleteSparkType; + return this; + } + + Builder writeProperties(Map properties) { + this.writeProperties = properties; + return this; + } + + SparkFileWriterFactory build() { + boolean noEqualityDeleteConf = equalityFieldIds == null && equalityDeleteRowSchema == null; + boolean fullEqualityDeleteConf = equalityFieldIds != null && equalityDeleteRowSchema != null; + Preconditions.checkArgument( + noEqualityDeleteConf || fullEqualityDeleteConf, + "Equality field IDs and equality delete row schema must be set together"); + + return new SparkFileWriterFactory( + table, + dataFileFormat, + dataSchema, + dataSparkType, + dataSortOrder, + deleteFileFormat, + equalityFieldIds, + equalityDeleteRowSchema, + equalityDeleteSparkType, + equalityDeleteSortOrder, + positionDeleteRowSchema, + positionDeleteSparkType, + writeProperties); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkInputPartition.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkInputPartition.java new file mode 100644 index 000000000000..99b1d78a86b0 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkInputPartition.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.io.Serializable; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.Table; +import org.apache.iceberg.types.Types; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.read.HasPartitionKey; +import org.apache.spark.sql.connector.read.InputPartition; + +class SparkInputPartition implements InputPartition, HasPartitionKey, Serializable { + private final Types.StructType groupingKeyType; + private final ScanTaskGroup taskGroup; + private final Broadcast

    tableBroadcast; + private final String branch; + private final String expectedSchemaString; + private final boolean caseSensitive; + private final transient String[] preferredLocations; + private final boolean cacheDeleteFilesOnExecutors; + + private transient Schema expectedSchema = null; + + SparkInputPartition( + Types.StructType groupingKeyType, + ScanTaskGroup taskGroup, + Broadcast
    tableBroadcast, + String branch, + String expectedSchemaString, + boolean caseSensitive, + String[] preferredLocations, + boolean cacheDeleteFilesOnExecutors) { + this.groupingKeyType = groupingKeyType; + this.taskGroup = taskGroup; + this.tableBroadcast = tableBroadcast; + this.branch = branch; + this.expectedSchemaString = expectedSchemaString; + this.caseSensitive = caseSensitive; + this.preferredLocations = preferredLocations; + this.cacheDeleteFilesOnExecutors = cacheDeleteFilesOnExecutors; + } + + @Override + public String[] preferredLocations() { + return preferredLocations; + } + + @Override + public InternalRow partitionKey() { + return new StructInternalRow(groupingKeyType).setStruct(taskGroup.groupingKey()); + } + + @SuppressWarnings("unchecked") + public ScanTaskGroup taskGroup() { + return (ScanTaskGroup) taskGroup; + } + + public boolean allTasksOfType(Class javaClass) { + return taskGroup.tasks().stream().allMatch(javaClass::isInstance); + } + + public Table table() { + return tableBroadcast.value(); + } + + public String branch() { + return branch; + } + + public boolean isCaseSensitive() { + return caseSensitive; + } + + public boolean cacheDeleteFilesOnExecutors() { + return cacheDeleteFilesOnExecutors; + } + + public Schema expectedSchema() { + if (expectedSchema == null) { + this.expectedSchema = SchemaParser.fromJson(expectedSchemaString); + } + + return expectedSchema; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkLocalScan.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkLocalScan.java new file mode 100644 index 000000000000..c2f9707775dd --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkLocalScan.java @@ -0,0 +1,66 @@ +/* + * 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 org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.read.LocalScan; +import org.apache.spark.sql.types.StructType; + +class SparkLocalScan implements LocalScan { + + private final Table table; + private final StructType readSchema; + private final InternalRow[] rows; + private final List filterExpressions; + + SparkLocalScan( + Table table, StructType readSchema, InternalRow[] rows, List filterExpressions) { + this.table = table; + this.readSchema = readSchema; + this.rows = rows; + this.filterExpressions = filterExpressions; + } + + @Override + public InternalRow[] rows() { + return rows; + } + + @Override + public StructType readSchema() { + return readSchema; + } + + @Override + public String description() { + return String.format("%s [filters=%s]", table, Spark3Util.describe(filterExpressions)); + } + + @Override + public String toString() { + return String.format( + "IcebergLocalScan(table=%s, type=%s, filters=%s)", + table, SparkSchemaUtil.convert(readSchema).asStruct(), filterExpressions); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkMetadataColumn.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkMetadataColumn.java new file mode 100644 index 000000000000..54687a8f044e --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkMetadataColumn.java @@ -0,0 +1,130 @@ +/* + * 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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; +import org.apache.spark.sql.connector.catalog.MetadataColumn; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.MetadataBuilder; + +public class SparkMetadataColumn implements MetadataColumn { + + private final String name; + private final DataType dataType; + private final boolean isNullable; + private final boolean preserveOnReinsert; + private final boolean preserveOnUpdate; + private final boolean preserveOnDelete; + + public static class Builder { + private String name; + private DataType dataType; + private boolean isNullable; + private boolean preserveOnReinsert = MetadataColumn.PRESERVE_ON_REINSERT_DEFAULT; + private boolean preserveOnUpdate = MetadataColumn.PRESERVE_ON_UPDATE_DEFAULT; + private boolean preserveOnDelete = MetadataColumn.PRESERVE_ON_DELETE_DEFAULT; + + public Builder name(String fieldName) { + Preconditions.checkArgument( + !Strings.isNullOrEmpty(fieldName), "Cannot have a null or empty name"); + this.name = fieldName; + return this; + } + + public Builder dataType(DataType type) { + Preconditions.checkArgument(type != null, "Cannot have a null datatype"); + this.dataType = type; + return this; + } + + public Builder withNullability(boolean nullable) { + this.isNullable = nullable; + return this; + } + + public Builder preserveOnReinsert(boolean shouldPreserveOnReinsert) { + this.preserveOnReinsert = shouldPreserveOnReinsert; + return this; + } + + public Builder preserveOnUpdate(boolean shouldPreserveOnUpdate) { + this.preserveOnUpdate = shouldPreserveOnUpdate; + return this; + } + + public Builder preserveOnDelete(boolean shouldPreserveOnDelete) { + this.preserveOnDelete = shouldPreserveOnDelete; + return this; + } + + public SparkMetadataColumn build() { + Preconditions.checkArgument( + name != null, "Cannot build a SparkMetadataColumn with a null name"); + Preconditions.checkArgument( + dataType != null, "Cannot build a SparkMetadataColumn with a null data type"); + return new SparkMetadataColumn( + name, dataType, isNullable, preserveOnReinsert, preserveOnUpdate, preserveOnDelete); + } + } + + public static Builder builder() { + return new Builder(); + } + + private SparkMetadataColumn( + String name, + DataType dataType, + boolean isNullable, + boolean preserveOnReinsert, + boolean preserveOnUpdate, + boolean preserveOnDelete) { + this.name = name; + this.dataType = dataType; + this.isNullable = isNullable; + this.preserveOnReinsert = preserveOnReinsert; + this.preserveOnUpdate = preserveOnUpdate; + this.preserveOnDelete = preserveOnDelete; + } + + @Override + public String name() { + return name; + } + + @Override + public DataType dataType() { + return dataType; + } + + @Override + public boolean isNullable() { + return isNullable; + } + + @Override + public String metadataInJSON() { + return new MetadataBuilder() + .putBoolean(MetadataColumn.PRESERVE_ON_REINSERT, preserveOnReinsert) + .putBoolean(MetadataColumn.PRESERVE_ON_UPDATE, preserveOnUpdate) + .putBoolean(MetadataColumn.PRESERVE_ON_DELETE, preserveOnDelete) + .build() + .json(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java new file mode 100644 index 000000000000..60dd1f318ca5 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java @@ -0,0 +1,593 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.io.BufferedWriter; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.io.UncheckedIOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Locale; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.MicroBatches; +import org.apache.iceberg.MicroBatches.MicroBatch; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +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.broadcast.Broadcast; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import org.apache.spark.sql.connector.read.streaming.CompositeReadLimit; +import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; +import org.apache.spark.sql.connector.read.streaming.Offset; +import org.apache.spark.sql.connector.read.streaming.ReadLimit; +import org.apache.spark.sql.connector.read.streaming.ReadMaxFiles; +import org.apache.spark.sql.connector.read.streaming.ReadMaxRows; +import org.apache.spark.sql.connector.read.streaming.SupportsTriggerAvailableNow; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SparkMicroBatchStream implements MicroBatchStream, SupportsTriggerAvailableNow { + private static final Joiner SLASH = Joiner.on("/"); + private static final Logger LOG = LoggerFactory.getLogger(SparkMicroBatchStream.class); + private static final Types.StructType EMPTY_GROUPING_KEY_TYPE = Types.StructType.of(); + + private final Table table; + private final String branch; + private final boolean caseSensitive; + private final String expectedSchema; + private final Broadcast
    tableBroadcast; + private final long splitSize; + private final int splitLookback; + private final long splitOpenFileCost; + private final boolean localityPreferred; + private final StreamingOffset initialOffset; + private final boolean skipDelete; + private final boolean skipOverwrite; + private final long fromTimestamp; + private final int maxFilesPerMicroBatch; + private final int maxRecordsPerMicroBatch; + private final boolean cacheDeleteFilesOnExecutors; + private StreamingOffset lastOffsetForTriggerAvailableNow; + + SparkMicroBatchStream( + JavaSparkContext sparkContext, + Table table, + SparkReadConf readConf, + Schema expectedSchema, + String checkpointLocation) { + this.table = table; + this.branch = readConf.branch(); + this.caseSensitive = readConf.caseSensitive(); + this.expectedSchema = SchemaParser.toJson(expectedSchema); + this.localityPreferred = readConf.localityEnabled(); + this.tableBroadcast = sparkContext.broadcast(SerializableTableWithSize.copyOf(table)); + this.splitSize = readConf.splitSize(); + this.splitLookback = readConf.splitLookback(); + this.splitOpenFileCost = readConf.splitOpenFileCost(); + this.fromTimestamp = readConf.streamFromTimestamp(); + this.maxFilesPerMicroBatch = readConf.maxFilesPerMicroBatch(); + this.maxRecordsPerMicroBatch = readConf.maxRecordsPerMicroBatch(); + this.cacheDeleteFilesOnExecutors = readConf.cacheDeleteFilesOnExecutors(); + + InitialOffsetStore initialOffsetStore = + new InitialOffsetStore(table, checkpointLocation, fromTimestamp); + this.initialOffset = initialOffsetStore.initialOffset(); + + this.skipDelete = readConf.streamingSkipDeleteSnapshots(); + this.skipOverwrite = readConf.streamingSkipOverwriteSnapshots(); + } + + @Override + public Offset latestOffset() { + table.refresh(); + if (table.currentSnapshot() == null) { + return StreamingOffset.START_OFFSET; + } + + if (table.currentSnapshot().timestampMillis() < fromTimestamp) { + return StreamingOffset.START_OFFSET; + } + + Snapshot latestSnapshot = table.currentSnapshot(); + + return new StreamingOffset(latestSnapshot.snapshotId(), addedFilesCount(latestSnapshot), false); + } + + @Override + public InputPartition[] planInputPartitions(Offset start, Offset end) { + Preconditions.checkArgument( + end instanceof StreamingOffset, "Invalid end offset: %s is not a StreamingOffset", end); + Preconditions.checkArgument( + start instanceof StreamingOffset, + "Invalid start offset: %s is not a StreamingOffset", + start); + + if (end.equals(StreamingOffset.START_OFFSET)) { + return new InputPartition[0]; + } + + StreamingOffset endOffset = (StreamingOffset) end; + StreamingOffset startOffset = (StreamingOffset) start; + + List fileScanTasks = planFiles(startOffset, endOffset); + + CloseableIterable splitTasks = + TableScanUtil.splitFiles(CloseableIterable.withNoopClose(fileScanTasks), splitSize); + List combinedScanTasks = + Lists.newArrayList( + TableScanUtil.planTasks(splitTasks, splitSize, splitLookback, splitOpenFileCost)); + String[][] locations = computePreferredLocations(combinedScanTasks); + + InputPartition[] partitions = new InputPartition[combinedScanTasks.size()]; + + for (int index = 0; index < combinedScanTasks.size(); index++) { + partitions[index] = + new SparkInputPartition( + EMPTY_GROUPING_KEY_TYPE, + combinedScanTasks.get(index), + tableBroadcast, + branch, + expectedSchema, + caseSensitive, + locations != null ? locations[index] : SparkPlanningUtil.NO_LOCATION_PREFERENCE, + cacheDeleteFilesOnExecutors); + } + + return partitions; + } + + private String[][] computePreferredLocations(List taskGroups) { + return localityPreferred ? SparkPlanningUtil.fetchBlockLocations(table.io(), taskGroups) : null; + } + + @Override + public PartitionReaderFactory createReaderFactory() { + return new SparkRowReaderFactory(); + } + + @Override + public Offset initialOffset() { + return initialOffset; + } + + @Override + public Offset deserializeOffset(String json) { + return StreamingOffset.fromJson(json); + } + + @Override + public void commit(Offset end) {} + + @Override + public void stop() {} + + private List planFiles(StreamingOffset startOffset, StreamingOffset endOffset) { + List fileScanTasks = Lists.newArrayList(); + StreamingOffset batchStartOffset = + StreamingOffset.START_OFFSET.equals(startOffset) + ? determineStartingOffset(table, fromTimestamp) + : startOffset; + + StreamingOffset currentOffset = null; + + // [(startOffset : startFileIndex), (endOffset : endFileIndex) ) + do { + long endFileIndex; + if (currentOffset == null) { + currentOffset = batchStartOffset; + } else { + Snapshot snapshotAfter = SnapshotUtil.snapshotAfter(table, currentOffset.snapshotId()); + // it may happen that we need to read this snapshot partially in case it's equal to + // endOffset. + if (currentOffset.snapshotId() != endOffset.snapshotId()) { + currentOffset = new StreamingOffset(snapshotAfter.snapshotId(), 0L, false); + } else { + currentOffset = endOffset; + } + } + + Snapshot snapshot = table.snapshot(currentOffset.snapshotId()); + + validateCurrentSnapshotExists(snapshot, currentOffset); + + if (!shouldProcess(snapshot)) { + LOG.debug("Skipping snapshot: {} of table {}", currentOffset.snapshotId(), table.name()); + continue; + } + + Snapshot currentSnapshot = table.snapshot(currentOffset.snapshotId()); + if (currentOffset.snapshotId() == endOffset.snapshotId()) { + endFileIndex = endOffset.position(); + } else { + endFileIndex = addedFilesCount(currentSnapshot); + } + + MicroBatch latestMicroBatch = + MicroBatches.from(currentSnapshot, table.io()) + .caseSensitive(caseSensitive) + .specsById(table.specs()) + .generate( + currentOffset.position(), + endFileIndex, + Long.MAX_VALUE, + currentOffset.shouldScanAllFiles()); + + fileScanTasks.addAll(latestMicroBatch.tasks()); + } while (currentOffset.snapshotId() != endOffset.snapshotId()); + + return fileScanTasks; + } + + private boolean shouldProcess(Snapshot snapshot) { + String op = snapshot.operation(); + switch (op) { + case DataOperations.APPEND: + return true; + case DataOperations.REPLACE: + return false; + case DataOperations.DELETE: + Preconditions.checkState( + skipDelete, + "Cannot process delete snapshot: %s, to ignore deletes, set %s=true", + snapshot.snapshotId(), + SparkReadOptions.STREAMING_SKIP_DELETE_SNAPSHOTS); + return false; + case DataOperations.OVERWRITE: + Preconditions.checkState( + skipOverwrite, + "Cannot process overwrite snapshot: %s, to ignore overwrites, set %s=true", + snapshot.snapshotId(), + SparkReadOptions.STREAMING_SKIP_OVERWRITE_SNAPSHOTS); + return false; + default: + throw new IllegalStateException( + String.format( + "Cannot process unknown snapshot operation: %s (snapshot id %s)", + op.toLowerCase(Locale.ROOT), snapshot.snapshotId())); + } + } + + private static StreamingOffset determineStartingOffset(Table table, Long fromTimestamp) { + if (table.currentSnapshot() == null) { + return StreamingOffset.START_OFFSET; + } + + if (fromTimestamp == null) { + // match existing behavior and start from the oldest snapshot + return new StreamingOffset(SnapshotUtil.oldestAncestor(table).snapshotId(), 0, false); + } + + if (table.currentSnapshot().timestampMillis() < fromTimestamp) { + return StreamingOffset.START_OFFSET; + } + + try { + Snapshot snapshot = SnapshotUtil.oldestAncestorAfter(table, fromTimestamp); + if (snapshot != null) { + return new StreamingOffset(snapshot.snapshotId(), 0, false); + } else { + return StreamingOffset.START_OFFSET; + } + } catch (IllegalStateException e) { + // could not determine the first snapshot after the timestamp. use the oldest ancestor instead + return new StreamingOffset(SnapshotUtil.oldestAncestor(table).snapshotId(), 0, false); + } + } + + private static int getMaxFiles(ReadLimit readLimit) { + if (readLimit instanceof ReadMaxFiles) { + return ((ReadMaxFiles) readLimit).maxFiles(); + } + + if (readLimit instanceof CompositeReadLimit) { + // We do not expect a CompositeReadLimit to contain a nested CompositeReadLimit. + // In fact, it should only be a composite of two or more of ReadMinRows, ReadMaxRows and + // ReadMaxFiles, with no more than one of each. + ReadLimit[] limits = ((CompositeReadLimit) readLimit).getReadLimits(); + for (ReadLimit limit : limits) { + if (limit instanceof ReadMaxFiles) { + return ((ReadMaxFiles) limit).maxFiles(); + } + } + } + + // there is no ReadMaxFiles, so return the default + return Integer.MAX_VALUE; + } + + private static int getMaxRows(ReadLimit readLimit) { + if (readLimit instanceof ReadMaxRows) { + long maxRows = ((ReadMaxRows) readLimit).maxRows(); + return Math.toIntExact(maxRows); + } + + if (readLimit instanceof CompositeReadLimit) { + ReadLimit[] limits = ((CompositeReadLimit) readLimit).getReadLimits(); + for (ReadLimit limit : limits) { + if (limit instanceof ReadMaxRows) { + long maxRows = ((ReadMaxRows) limit).maxRows(); + return Math.toIntExact(maxRows); + } + } + } + + // there is no ReadMaxRows, so return the default + return Integer.MAX_VALUE; + } + + @Override + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public Offset latestOffset(Offset startOffset, ReadLimit limit) { + // calculate end offset get snapshotId from the startOffset + Preconditions.checkArgument( + startOffset instanceof StreamingOffset, + "Invalid start offset: %s is not a StreamingOffset", + startOffset); + + table.refresh(); + if (table.currentSnapshot() == null) { + return StreamingOffset.START_OFFSET; + } + + if (table.currentSnapshot().timestampMillis() < fromTimestamp) { + return StreamingOffset.START_OFFSET; + } + + // end offset can expand to multiple snapshots + StreamingOffset startingOffset = (StreamingOffset) startOffset; + + if (startOffset.equals(StreamingOffset.START_OFFSET)) { + startingOffset = determineStartingOffset(table, fromTimestamp); + } + + Snapshot curSnapshot = table.snapshot(startingOffset.snapshotId()); + validateCurrentSnapshotExists(curSnapshot, startingOffset); + + // Use the pre-computed snapshotId when Trigger.AvailableNow is enabled. + long latestSnapshotId = + lastOffsetForTriggerAvailableNow != null + ? lastOffsetForTriggerAvailableNow.snapshotId() + : table.currentSnapshot().snapshotId(); + + int startPosOfSnapOffset = (int) startingOffset.position(); + + boolean scanAllFiles = startingOffset.shouldScanAllFiles(); + + boolean shouldContinueReading = true; + int curFilesAdded = 0; + long curRecordCount = 0; + int curPos = 0; + + // Note : we produce nextOffset with pos as non-inclusive + while (shouldContinueReading) { + // generate manifest index for the curSnapshot + List> indexedManifests = + MicroBatches.skippedManifestIndexesFromSnapshot( + table.io(), curSnapshot, startPosOfSnapOffset, scanAllFiles); + // this is under assumption we will be able to add at-least 1 file in the new offset + for (int idx = 0; idx < indexedManifests.size() && shouldContinueReading; idx++) { + // be rest assured curPos >= startFileIndex + curPos = indexedManifests.get(idx).second(); + try (CloseableIterable taskIterable = + MicroBatches.openManifestFile( + table.io(), + table.specs(), + caseSensitive, + curSnapshot, + indexedManifests.get(idx).first(), + scanAllFiles); + CloseableIterator taskIter = taskIterable.iterator()) { + while (taskIter.hasNext()) { + FileScanTask task = taskIter.next(); + if (curPos >= startPosOfSnapOffset) { + if ((curFilesAdded + 1) > getMaxFiles(limit)) { + // On including the file it might happen that we might exceed, the configured + // soft limit on the number of records, since this is a soft limit its acceptable. + shouldContinueReading = false; + break; + } + + curFilesAdded += 1; + curRecordCount += task.file().recordCount(); + + if (curRecordCount >= getMaxRows(limit)) { + // we included the file, so increment the number of files + // read in the current snapshot. + ++curPos; + shouldContinueReading = false; + break; + } + } + ++curPos; + } + } catch (IOException ioe) { + LOG.warn("Failed to close task iterable", ioe); + } + } + // if the currentSnapShot was also the latestSnapshot then break + if (curSnapshot.snapshotId() == latestSnapshotId) { + break; + } + + // if everything was OK and we consumed complete snapshot then move to next snapshot + if (shouldContinueReading) { + Snapshot nextValid = nextValidSnapshot(curSnapshot); + if (nextValid == null) { + // nextValid implies all the remaining snapshots should be skipped. + break; + } + // we found the next available snapshot, continue from there. + curSnapshot = nextValid; + startPosOfSnapOffset = -1; + // if anyhow we are moving to next snapshot we should only scan addedFiles + scanAllFiles = false; + } + } + + StreamingOffset latestStreamingOffset = + new StreamingOffset(curSnapshot.snapshotId(), curPos, scanAllFiles); + + // if no new data arrived, then return null. + return latestStreamingOffset.equals(startingOffset) ? null : latestStreamingOffset; + } + + /** + * Get the next snapshot skiping over rewrite and delete snapshots. + * + * @param curSnapshot the current snapshot + * @return the next valid snapshot (not a rewrite or delete snapshot), returns null if all + * remaining snapshots should be skipped. + */ + private Snapshot nextValidSnapshot(Snapshot curSnapshot) { + Snapshot nextSnapshot = SnapshotUtil.snapshotAfter(table, curSnapshot.snapshotId()); + // skip over rewrite and delete snapshots + while (!shouldProcess(nextSnapshot)) { + LOG.debug("Skipping snapshot: {} of table {}", nextSnapshot.snapshotId(), table.name()); + // if the currentSnapShot was also the mostRecentSnapshot then break + if (nextSnapshot.snapshotId() == table.currentSnapshot().snapshotId()) { + return null; + } + nextSnapshot = SnapshotUtil.snapshotAfter(table, nextSnapshot.snapshotId()); + } + return nextSnapshot; + } + + private long addedFilesCount(Snapshot snapshot) { + long addedFilesCount = + PropertyUtil.propertyAsLong(snapshot.summary(), SnapshotSummary.ADDED_FILES_PROP, -1); + // If snapshotSummary doesn't have SnapshotSummary.ADDED_FILES_PROP, + // iterate through addedFiles iterator to find addedFilesCount. + return addedFilesCount == -1 + ? Iterables.size(snapshot.addedDataFiles(table.io())) + : addedFilesCount; + } + + private void validateCurrentSnapshotExists(Snapshot snapshot, StreamingOffset currentOffset) { + if (snapshot == null) { + throw new IllegalStateException( + String.format( + Locale.ROOT, + "Cannot load current offset at snapshot %d, the snapshot was expired or removed", + currentOffset.snapshotId())); + } + } + + @Override + public ReadLimit getDefaultReadLimit() { + if (maxFilesPerMicroBatch != Integer.MAX_VALUE + && maxRecordsPerMicroBatch != Integer.MAX_VALUE) { + ReadLimit[] readLimits = new ReadLimit[2]; + readLimits[0] = ReadLimit.maxFiles(maxFilesPerMicroBatch); + readLimits[1] = ReadLimit.maxRows(maxRecordsPerMicroBatch); + return ReadLimit.compositeLimit(readLimits); + } else if (maxFilesPerMicroBatch != Integer.MAX_VALUE) { + return ReadLimit.maxFiles(maxFilesPerMicroBatch); + } else if (maxRecordsPerMicroBatch != Integer.MAX_VALUE) { + return ReadLimit.maxRows(maxRecordsPerMicroBatch); + } else { + return ReadLimit.allAvailable(); + } + } + + @Override + public void prepareForTriggerAvailableNow() { + LOG.info("The streaming query reports to use Trigger.AvailableNow"); + + lastOffsetForTriggerAvailableNow = + (StreamingOffset) latestOffset(initialOffset, ReadLimit.allAvailable()); + + LOG.info("lastOffset for Trigger.AvailableNow is {}", lastOffsetForTriggerAvailableNow.json()); + } + + private static class InitialOffsetStore { + private final Table table; + private final FileIO io; + private final String initialOffsetLocation; + private final Long fromTimestamp; + + InitialOffsetStore(Table table, String checkpointLocation, Long fromTimestamp) { + this.table = table; + this.io = table.io(); + this.initialOffsetLocation = SLASH.join(checkpointLocation, "offsets/0"); + this.fromTimestamp = fromTimestamp; + } + + public StreamingOffset initialOffset() { + InputFile inputFile = io.newInputFile(initialOffsetLocation); + if (inputFile.exists()) { + return readOffset(inputFile); + } + + table.refresh(); + StreamingOffset offset = determineStartingOffset(table, fromTimestamp); + + OutputFile outputFile = io.newOutputFile(initialOffsetLocation); + writeOffset(offset, outputFile); + + return offset; + } + + private void writeOffset(StreamingOffset offset, OutputFile file) { + try (OutputStream outputStream = file.create()) { + BufferedWriter writer = + new BufferedWriter(new OutputStreamWriter(outputStream, StandardCharsets.UTF_8)); + writer.write(offset.json()); + writer.flush(); + } catch (IOException ioException) { + throw new UncheckedIOException( + String.format("Failed writing offset to: %s", initialOffsetLocation), ioException); + } + } + + private StreamingOffset readOffset(InputFile file) { + try (InputStream in = file.newStream()) { + return StreamingOffset.fromJson(in); + } catch (IOException ioException) { + throw new UncheckedIOException( + String.format("Failed reading offset from: %s", initialOffsetLocation), ioException); + } + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java new file mode 100644 index 000000000000..c9726518ee4e --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java @@ -0,0 +1,253 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +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.Expression; +import org.apache.iceberg.io.CloseableIterable; +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.spark.Spark3Util; +import org.apache.iceberg.spark.SparkReadConf; +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; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.connector.read.SupportsReportPartitioning; +import org.apache.spark.sql.connector.read.partitioning.KeyGroupedPartitioning; +import org.apache.spark.sql.connector.read.partitioning.Partitioning; +import org.apache.spark.sql.connector.read.partitioning.UnknownPartitioning; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +abstract class SparkPartitioningAwareScan extends SparkScan + implements SupportsReportPartitioning { + + private static final Logger LOG = LoggerFactory.getLogger(SparkPartitioningAwareScan.class); + + private final Scan> scan; + private final boolean preserveDataGrouping; + + private Set specs = null; // lazy cache of scanned specs + private List tasks = null; // lazy cache of uncombined tasks + private List> taskGroups = null; // lazy cache of task groups + private StructType groupingKeyType = null; // lazy cache of the grouping key type + private Transform[] groupingKeyTransforms = null; // lazy cache of grouping key transforms + + SparkPartitioningAwareScan( + SparkSession spark, + Table table, + Scan> scan, + SparkReadConf readConf, + Schema expectedSchema, + List filters, + Supplier scanReportSupplier) { + super(spark, table, readConf, expectedSchema, filters, scanReportSupplier); + + this.scan = scan; + this.preserveDataGrouping = readConf.preserveDataGrouping(); + + if (scan == null) { + this.specs = Collections.emptySet(); + this.tasks = Collections.emptyList(); + this.taskGroups = Collections.emptyList(); + } + } + + protected abstract Class taskJavaClass(); + + protected Scan> scan() { + return scan; + } + + @Override + public Partitioning outputPartitioning() { + if (groupingKeyType().fields().isEmpty()) { + LOG.info( + "Reporting UnknownPartitioning with {} partition(s) for table {}", + taskGroups().size(), + table().name()); + return new UnknownPartitioning(taskGroups().size()); + } else { + LOG.info( + "Reporting KeyGroupedPartitioning by {} with {} partition(s) for table {}", + groupingKeyTransforms(), + taskGroups().size(), + table().name()); + return new KeyGroupedPartitioning(groupingKeyTransforms(), taskGroups().size()); + } + } + + @Override + protected StructType groupingKeyType() { + if (groupingKeyType == null) { + if (preserveDataGrouping) { + this.groupingKeyType = computeGroupingKeyType(); + } else { + this.groupingKeyType = StructType.of(); + } + } + + return groupingKeyType; + } + + private StructType computeGroupingKeyType() { + return org.apache.iceberg.Partitioning.groupingKeyType(expectedSchema(), specs()); + } + + private Transform[] groupingKeyTransforms() { + if (groupingKeyTransforms == null) { + Map fieldsById = indexFieldsById(specs()); + + List groupingKeyFields = + groupingKeyType().fields().stream() + .map(field -> fieldsById.get(field.fieldId())) + .collect(Collectors.toList()); + + Schema schema = SnapshotUtil.schemaFor(table(), branch()); + this.groupingKeyTransforms = Spark3Util.toTransforms(schema, groupingKeyFields); + } + + return groupingKeyTransforms; + } + + private Map indexFieldsById(Iterable specIterable) { + Map fieldsById = Maps.newHashMap(); + + for (PartitionSpec spec : specIterable) { + for (PartitionField field : spec.fields()) { + fieldsById.putIfAbsent(field.fieldId(), field); + } + } + + return fieldsById; + } + + protected Set specs() { + if (specs == null) { + // avoid calling equals/hashCode on specs as those methods are relatively expensive + IntStream specIds = tasks().stream().mapToInt(task -> task.spec().specId()).distinct(); + this.specs = specIds.mapToObj(id -> table().specs().get(id)).collect(Collectors.toSet()); + } + + return specs; + } + + protected synchronized List tasks() { + if (tasks == null) { + try (CloseableIterable taskIterable = scan.planFiles()) { + List plannedTasks = Lists.newArrayList(); + + for (ScanTask task : taskIterable) { + ValidationException.check( + taskJavaClass().isInstance(task), + "Unsupported task type, expected a subtype of %s: %s", + taskJavaClass().getName(), + task.getClass().getName()); + + plannedTasks.add(taskJavaClass().cast(task)); + } + + this.tasks = plannedTasks; + } catch (IOException e) { + throw new UncheckedIOException("Failed to close scan: " + scan, e); + } + } + + return tasks; + } + + @Override + protected synchronized List> taskGroups() { + if (taskGroups == null) { + if (groupingKeyType().fields().isEmpty()) { + CloseableIterable> plannedTaskGroups = + TableScanUtil.planTaskGroups( + CloseableIterable.withNoopClose(tasks()), + adjustSplitSize(tasks(), scan.targetSplitSize()), + scan.splitLookback(), + scan.splitOpenFileCost()); + this.taskGroups = Lists.newArrayList(plannedTaskGroups); + + LOG.debug( + "Planned {} task group(s) without data grouping for table {}", + taskGroups.size(), + table().name()); + + } else { + List> plannedTaskGroups = + TableScanUtil.planTaskGroups( + tasks(), + adjustSplitSize(tasks(), scan.targetSplitSize()), + scan.splitLookback(), + scan.splitOpenFileCost(), + groupingKeyType()); + StructLikeSet plannedGroupingKeys = collectGroupingKeys(plannedTaskGroups); + + LOG.debug( + "Planned {} task group(s) with {} grouping key type and {} unique grouping key(s) for table {}", + plannedTaskGroups.size(), + groupingKeyType(), + plannedGroupingKeys.size(), + table().name()); + + this.taskGroups = plannedTaskGroups; + } + } + + return taskGroups; + } + + // only task groups can be reset while resetting tasks + // the set of scanned specs and grouping key type must never change + protected void resetTasks(List filteredTasks) { + this.taskGroups = null; + this.tasks = filteredTasks; + } + + private StructLikeSet collectGroupingKeys(Iterable> taskGroupIterable) { + StructLikeSet keys = StructLikeSet.create(groupingKeyType()); + + for (ScanTaskGroup taskGroup : taskGroupIterable) { + keys.add(taskGroup.groupingKey()); + } + + return keys; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPlanningUtil.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPlanningUtil.java new file mode 100644 index 000000000000..9cdec2c8f463 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPlanningUtil.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.JavaHash; +import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.util.ThreadPools; + +class SparkPlanningUtil { + + public static final String[] NO_LOCATION_PREFERENCE = new String[0]; + + private SparkPlanningUtil() {} + + public static String[][] fetchBlockLocations( + FileIO io, List> taskGroups) { + String[][] locations = new String[taskGroups.size()][]; + + Tasks.range(taskGroups.size()) + .stopOnFailure() + .executeWith(ThreadPools.getWorkerPool()) + .run(index -> locations[index] = Util.blockLocations(io, taskGroups.get(index))); + + return locations; + } + + public static String[][] assignExecutors( + List> taskGroups, List executorLocations) { + Map> partitionHashes = Maps.newHashMap(); + String[][] locations = new String[taskGroups.size()][]; + + for (int index = 0; index < taskGroups.size(); index++) { + locations[index] = assign(taskGroups.get(index), executorLocations, partitionHashes); + } + + return locations; + } + + private static String[] assign( + ScanTaskGroup taskGroup, + List executorLocations, + Map> partitionHashes) { + List locations = Lists.newArrayList(); + + for (ScanTask task : taskGroup.tasks()) { + if (task.isFileScanTask()) { + FileScanTask fileTask = task.asFileScanTask(); + PartitionSpec spec = fileTask.spec(); + if (spec.isPartitioned() && !fileTask.deletes().isEmpty()) { + JavaHash partitionHash = + partitionHashes.computeIfAbsent(spec.specId(), key -> partitionHash(spec)); + int partitionHashCode = partitionHash.hash(fileTask.partition()); + int index = Math.floorMod(partitionHashCode, executorLocations.size()); + String executorLocation = executorLocations.get(index); + locations.add(executorLocation); + } + } + } + + return locations.toArray(NO_LOCATION_PREFERENCE); + } + + private static JavaHash partitionHash(PartitionSpec spec) { + return JavaHash.forType(spec.partitionType()); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java new file mode 100644 index 000000000000..0ec7084bfd1b --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java @@ -0,0 +1,522 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PositionDeletesTable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableUtil; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.io.ClusteredPositionDeleteWriter; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.PartitioningDVWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator; +import org.apache.iceberg.spark.ScanTaskSetManager; +import org.apache.iceberg.spark.SparkWriteConf; +import org.apache.iceberg.util.DeleteFileSet; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.write.BatchWrite; +import org.apache.spark.sql.connector.write.DataWriter; +import org.apache.spark.sql.connector.write.DataWriterFactory; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.PhysicalWriteInfo; +import org.apache.spark.sql.connector.write.Write; +import org.apache.spark.sql.connector.write.WriterCommitMessage; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; + +/** + * {@link Write} class for rewriting position delete files from Spark. Responsible for creating + * {@link SparkPositionDeletesRewrite.PositionDeleteBatchWrite} + * + *

    This class is meant to be used for an action to rewrite position delete files. Hence, it + * assumes all position deletes to rewrite have come from {@link ScanTaskSetManager} and that all + * have the same partition spec id and partition values. + */ +public class SparkPositionDeletesRewrite implements Write { + + private final JavaSparkContext sparkContext; + private final Table table; + private final String queryId; + private final FileFormat format; + private final long targetFileSize; + private final DeleteGranularity deleteGranularity; + private final Schema writeSchema; + private final StructType dsSchema; + private final String fileSetId; + private final int specId; + private final StructLike partition; + private final Map writeProperties; + + /** + * Constructs a {@link SparkPositionDeletesRewrite}. + * + * @param spark Spark session + * @param table instance of {@link PositionDeletesTable} + * @param writeConf Spark write config + * @param writeInfo Spark write info + * @param writeSchema Iceberg output schema + * @param dsSchema schema of original incoming position deletes dataset + * @param specId spec id of position deletes + * @param partition partition value of position deletes + */ + SparkPositionDeletesRewrite( + SparkSession spark, + Table table, + SparkWriteConf writeConf, + LogicalWriteInfo writeInfo, + Schema writeSchema, + StructType dsSchema, + int specId, + StructLike partition) { + this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + this.table = table; + this.queryId = writeInfo.queryId(); + this.format = writeConf.deleteFileFormat(); + this.targetFileSize = writeConf.targetDeleteFileSize(); + this.deleteGranularity = writeConf.deleteGranularity(); + this.writeSchema = writeSchema; + this.dsSchema = dsSchema; + this.fileSetId = writeConf.rewrittenFileSetId(); + this.specId = specId; + this.partition = partition; + this.writeProperties = writeConf.writeProperties(); + } + + @Override + public BatchWrite toBatch() { + return new PositionDeleteBatchWrite(); + } + + /** {@link BatchWrite} class for rewriting position deletes files from Spark */ + class PositionDeleteBatchWrite implements BatchWrite { + + @Override + public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { + // broadcast the table metadata as the writer factory will be sent to executors + Broadcast

    tableBroadcast = + sparkContext.broadcast(SerializableTableWithSize.copyOf(table)); + return new PositionDeletesWriterFactory( + tableBroadcast, + queryId, + format, + targetFileSize, + deleteGranularity, + writeSchema, + dsSchema, + specId, + partition, + writeProperties); + } + + @Override + public boolean useCommitCoordinator() { + return false; + } + + @Override + public void commit(WriterCommitMessage[] messages) { + PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get(); + coordinator.stageRewrite(table, fileSetId, DeleteFileSet.of(files(messages))); + } + + @Override + public void abort(WriterCommitMessage[] messages) { + SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages)); + } + + private List files(WriterCommitMessage[] messages) { + List files = Lists.newArrayList(); + + for (WriterCommitMessage message : messages) { + if (message != null) { + DeleteTaskCommit taskCommit = (DeleteTaskCommit) message; + files.addAll(Arrays.asList(taskCommit.files())); + } + } + + return files; + } + } + + /** + * Writer factory for position deletes metadata table. Responsible for creating {@link + * DeleteWriter}. + * + *

    This writer is meant to be used for an action to rewrite delete files. Hence, it makes an + * assumption that all incoming deletes belong to the same partition, and that incoming dataset is + * from {@link ScanTaskSetManager}. + */ + static class PositionDeletesWriterFactory implements DataWriterFactory { + private final Broadcast

    tableBroadcast; + private final String queryId; + private final FileFormat format; + private final Long targetFileSize; + private final DeleteGranularity deleteGranularity; + private final Schema writeSchema; + private final StructType dsSchema; + private final int specId; + private final StructLike partition; + private final Map writeProperties; + + PositionDeletesWriterFactory( + Broadcast
    tableBroadcast, + String queryId, + FileFormat format, + long targetFileSize, + DeleteGranularity deleteGranularity, + Schema writeSchema, + StructType dsSchema, + int specId, + StructLike partition, + Map writeProperties) { + this.tableBroadcast = tableBroadcast; + this.queryId = queryId; + this.format = format; + this.targetFileSize = targetFileSize; + this.deleteGranularity = deleteGranularity; + this.writeSchema = writeSchema; + this.dsSchema = dsSchema; + this.specId = specId; + this.partition = partition; + this.writeProperties = writeProperties; + } + + @Override + public DataWriter createWriter(int partitionId, long taskId) { + Table table = tableBroadcast.value(); + int formatVersion = TableUtil.formatVersion(table); + + OutputFileFactory deleteFileFactory = + OutputFileFactory.builderFor(table, partitionId, taskId) + .format(formatVersion >= 3 ? FileFormat.PUFFIN : format) + .operationId(queryId) + .suffix("deletes") + .build(); + + if (formatVersion >= 3) { + return new DVWriter(table, deleteFileFactory, dsSchema, specId, partition); + } else { + Schema positionDeleteRowSchema = positionDeleteRowSchema(); + StructType deleteSparkType = deleteSparkType(); + StructType deleteSparkTypeWithoutRow = deleteSparkTypeWithoutRow(); + + SparkFileWriterFactory writerFactoryWithRow = + SparkFileWriterFactory.builderFor(table) + .deleteFileFormat(format) + .positionDeleteRowSchema(positionDeleteRowSchema) + .positionDeleteSparkType(deleteSparkType) + .writeProperties(writeProperties) + .build(); + SparkFileWriterFactory writerFactoryWithoutRow = + SparkFileWriterFactory.builderFor(table) + .deleteFileFormat(format) + .positionDeleteSparkType(deleteSparkTypeWithoutRow) + .writeProperties(writeProperties) + .build(); + + return new DeleteWriter( + table, + writerFactoryWithRow, + writerFactoryWithoutRow, + deleteFileFactory, + targetFileSize, + deleteGranularity, + dsSchema, + specId, + partition); + } + } + + private Schema positionDeleteRowSchema() { + return new Schema( + writeSchema + .findField(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME) + .type() + .asStructType() + .fields()); + } + + private StructType deleteSparkType() { + return new StructType( + new StructField[] { + dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()), + dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()), + dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME) + }); + } + + private StructType deleteSparkTypeWithoutRow() { + return new StructType( + new StructField[] { + dsSchema.apply(MetadataColumns.DELETE_FILE_PATH.name()), + dsSchema.apply(MetadataColumns.DELETE_FILE_POS.name()), + }); + } + } + + /** + * Writer for position deletes metadata table. + * + *

    Iceberg specifies delete files schema as having either 'row' as a required field, or omits + * 'row' altogether. This is to ensure accuracy of delete file statistics on 'row' column. Hence, + * this writer, if receiving source position deletes with null and non-null rows, redirects rows + * with null 'row' to one file writer, and non-null 'row' to another file writer. + * + *

    This writer is meant to be used for an action to rewrite delete files. Hence, it makes an + * assumption that all incoming deletes belong to the same partition. + */ + private static class DeleteWriter implements DataWriter { + private final SparkFileWriterFactory writerFactoryWithRow; + private final SparkFileWriterFactory writerFactoryWithoutRow; + private final OutputFileFactory deleteFileFactory; + private final long targetFileSize; + private final DeleteGranularity deleteGranularity; + private final PositionDelete positionDelete; + private final FileIO io; + private final PartitionSpec spec; + private final int fileOrdinal; + private final int positionOrdinal; + private final int rowOrdinal; + private final int rowSize; + private final StructLike partition; + + private ClusteredPositionDeleteWriter writerWithRow; + private ClusteredPositionDeleteWriter writerWithoutRow; + private boolean closed = false; + + /** + * Constructs a {@link DeleteWriter}. + * + * @param table position deletes metadata table + * @param writerFactoryWithRow writer factory for deletes with non-null 'row' + * @param writerFactoryWithoutRow writer factory for deletes with null 'row' + * @param deleteFileFactory delete file factory + * @param targetFileSize target file size + * @param dsSchema schema of incoming dataset of position deletes + * @param specId partition spec id of incoming position deletes. All incoming partition deletes + * are required to have the same spec id. + * @param partition partition value of incoming position delete. All incoming partition deletes + * are required to have the same partition. + */ + DeleteWriter( + Table table, + SparkFileWriterFactory writerFactoryWithRow, + SparkFileWriterFactory writerFactoryWithoutRow, + OutputFileFactory deleteFileFactory, + long targetFileSize, + DeleteGranularity deleteGranularity, + StructType dsSchema, + int specId, + StructLike partition) { + this.deleteFileFactory = deleteFileFactory; + this.targetFileSize = targetFileSize; + this.deleteGranularity = deleteGranularity; + this.writerFactoryWithRow = writerFactoryWithRow; + this.writerFactoryWithoutRow = writerFactoryWithoutRow; + this.positionDelete = PositionDelete.create(); + this.io = table.io(); + this.spec = table.specs().get(specId); + this.partition = partition; + + this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name()); + this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name()); + + this.rowOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME); + DataType type = dsSchema.apply(MetadataColumns.DELETE_FILE_ROW_FIELD_NAME).dataType(); + Preconditions.checkArgument( + type instanceof StructType, "Expected row as struct type but was %s", type); + this.rowSize = ((StructType) type).size(); + } + + @Override + public void write(InternalRow record) { + String file = record.getString(fileOrdinal); + long position = record.getLong(positionOrdinal); + InternalRow row = record.getStruct(rowOrdinal, rowSize); + if (row != null) { + positionDelete.set(file, position, row); + lazyWriterWithRow().write(positionDelete, spec, partition); + } else { + positionDelete.set(file, position, null); + lazyWriterWithoutRow().write(positionDelete, spec, partition); + } + } + + @Override + public WriterCommitMessage commit() throws IOException { + close(); + return new DeleteTaskCommit(allDeleteFiles()); + } + + @Override + public void abort() throws IOException { + close(); + SparkCleanupUtil.deleteTaskFiles(io, allDeleteFiles()); + } + + @Override + public void close() throws IOException { + if (!closed) { + if (writerWithRow != null) { + writerWithRow.close(); + } + if (writerWithoutRow != null) { + writerWithoutRow.close(); + } + this.closed = true; + } + } + + private ClusteredPositionDeleteWriter lazyWriterWithRow() { + if (writerWithRow == null) { + this.writerWithRow = + new ClusteredPositionDeleteWriter<>( + writerFactoryWithRow, deleteFileFactory, io, targetFileSize, deleteGranularity); + } + return writerWithRow; + } + + private ClusteredPositionDeleteWriter lazyWriterWithoutRow() { + if (writerWithoutRow == null) { + this.writerWithoutRow = + new ClusteredPositionDeleteWriter<>( + writerFactoryWithoutRow, deleteFileFactory, io, targetFileSize, deleteGranularity); + } + return writerWithoutRow; + } + + private List allDeleteFiles() { + List allDeleteFiles = Lists.newArrayList(); + if (writerWithRow != null) { + allDeleteFiles.addAll(writerWithRow.result().deleteFiles()); + } + if (writerWithoutRow != null) { + allDeleteFiles.addAll(writerWithoutRow.result().deleteFiles()); + } + return allDeleteFiles; + } + } + + /** + * DV Writer for position deletes metadata table. + * + *

    This writer is meant to be used for an action to rewrite delete files when the table + * supports DVs. + */ + private static class DVWriter implements DataWriter { + private final PositionDelete positionDelete; + private final FileIO io; + private final PartitionSpec spec; + private final int fileOrdinal; + private final int positionOrdinal; + private final StructLike partition; + private final PartitioningDVWriter dvWriter; + private boolean closed = false; + + /** + * Constructs a {@link DeleteWriter}. + * + * @param table position deletes metadata table + * @param deleteFileFactory delete file factory + * @param dsSchema schema of incoming dataset of position deletes + * @param specId partition spec id of incoming position deletes. All incoming partition deletes + * are required to have the same spec id. + * @param partition partition value of incoming position delete. All incoming partition deletes + * are required to have the same partition. + */ + DVWriter( + Table table, + OutputFileFactory deleteFileFactory, + StructType dsSchema, + int specId, + StructLike partition) { + this.positionDelete = PositionDelete.create(); + this.io = table.io(); + this.spec = table.specs().get(specId); + this.partition = partition; + this.fileOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_PATH.name()); + this.positionOrdinal = dsSchema.fieldIndex(MetadataColumns.DELETE_FILE_POS.name()); + this.dvWriter = new PartitioningDVWriter<>(deleteFileFactory, p -> null); + } + + @Override + public void write(InternalRow record) { + String file = record.getString(fileOrdinal); + long position = record.getLong(positionOrdinal); + positionDelete.set(file, position, null); + dvWriter.write(positionDelete, spec, partition); + } + + @Override + public WriterCommitMessage commit() throws IOException { + close(); + return new DeleteTaskCommit(allDeleteFiles()); + } + + @Override + public void abort() throws IOException { + close(); + SparkCleanupUtil.deleteTaskFiles(io, allDeleteFiles()); + } + + @Override + public void close() throws IOException { + if (!closed) { + if (null != dvWriter) { + dvWriter.close(); + } + this.closed = true; + } + } + + private List allDeleteFiles() { + return dvWriter.result().deleteFiles(); + } + } + + public static class DeleteTaskCommit implements WriterCommitMessage { + private final DeleteFile[] taskFiles; + + DeleteTaskCommit(List deleteFiles) { + this.taskFiles = deleteFiles.toArray(new DeleteFile[0]); + } + + DeleteFile[] files() { + return taskFiles; + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java new file mode 100644 index 000000000000..9fccc05ea25c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewriteBuilder.java @@ -0,0 +1,108 @@ +/* + * 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.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.spark.ScanTaskSetManager; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkWriteConf; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.Write; +import org.apache.spark.sql.connector.write.WriteBuilder; +import org.apache.spark.sql.types.StructType; + +/** + * Builder class for rewrites of position delete files from Spark. Responsible for creating {@link + * SparkPositionDeletesRewrite}. + * + *

    This class is meant to be used for an action to rewrite delete files. Hence, it makes an + * assumption that all incoming deletes belong to the same partition, and that incoming dataset is + * from {@link ScanTaskSetManager}. + */ +public class SparkPositionDeletesRewriteBuilder implements WriteBuilder { + + private final SparkSession spark; + private final Table table; + private final SparkWriteConf writeConf; + private final LogicalWriteInfo writeInfo; + private final StructType dsSchema; + private final Schema writeSchema; + + SparkPositionDeletesRewriteBuilder( + 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.writeSchema = SparkSchemaUtil.convert(table.schema(), dsSchema, writeConf.caseSensitive()); + } + + @Override + public Write build() { + String fileSetId = writeConf.rewrittenFileSetId(); + + Preconditions.checkArgument( + fileSetId != null, "Can only write to %s via actions", table.name()); + + // all files of rewrite group have same partition and spec id + ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + List tasks = taskSetManager.fetchTasks(table, fileSetId); + Preconditions.checkArgument( + tasks != null && !tasks.isEmpty(), "No scan tasks found for %s", fileSetId); + + int specId = specId(fileSetId, tasks); + StructLike partition = partition(fileSetId, tasks); + + return new SparkPositionDeletesRewrite( + spark, table, writeConf, writeInfo, writeSchema, dsSchema, specId, partition); + } + + private int specId(String fileSetId, List tasks) { + Set specIds = tasks.stream().map(t -> t.spec().specId()).collect(Collectors.toSet()); + Preconditions.checkArgument( + specIds.size() == 1, + "All scan tasks of %s are expected to have same spec id, but got %s", + fileSetId, + Joiner.on(",").join(specIds)); + return tasks.get(0).spec().specId(); + } + + private StructLike partition(String fileSetId, List tasks) { + StructLikeSet partitions = StructLikeSet.create(tasks.get(0).spec().partitionType()); + tasks.stream().map(ContentScanTask::partition).forEach(partitions::add); + Preconditions.checkArgument( + partitions.size() == 1, + "All scan tasks of %s are expected to have the same partition, but got %s", + fileSetId, + Joiner.on(",").join(partitions)); + return tasks.get(0).partition(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java new file mode 100644 index 000000000000..97f9601cedd9 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaOperation.java @@ -0,0 +1,125 @@ +/* + * 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 org.apache.iceberg.IsolationLevel; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.expressions.Expressions; +import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.connector.write.DeltaWriteBuilder; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.RowLevelOperation; +import org.apache.spark.sql.connector.write.RowLevelOperationInfo; +import org.apache.spark.sql.connector.write.SupportsDelta; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +class SparkPositionDeltaOperation implements RowLevelOperation, SupportsDelta { + + private final SparkSession spark; + private final Table table; + private final String branch; + private final Command command; + private final IsolationLevel isolationLevel; + + // lazy vars + private ScanBuilder lazyScanBuilder; + private Scan configuredScan; + private DeltaWriteBuilder lazyWriteBuilder; + + SparkPositionDeltaOperation( + SparkSession spark, + Table table, + String branch, + RowLevelOperationInfo info, + IsolationLevel isolationLevel) { + this.spark = spark; + this.table = table; + this.branch = branch; + this.command = info.command(); + this.isolationLevel = isolationLevel; + } + + @Override + public Command command() { + return command; + } + + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + if (lazyScanBuilder == null) { + this.lazyScanBuilder = + new SparkScanBuilder(spark, table, branch, options) { + @Override + public Scan build() { + Scan scan = super.buildMergeOnReadScan(); + SparkPositionDeltaOperation.this.configuredScan = scan; + return scan; + } + }; + } + + return lazyScanBuilder; + } + + @Override + public DeltaWriteBuilder newWriteBuilder(LogicalWriteInfo info) { + if (lazyWriteBuilder == null) { + // don't validate the scan is not null as if the condition evaluates to false, + // the optimizer replaces the original scan relation with a local relation + lazyWriteBuilder = + new SparkPositionDeltaWriteBuilder( + spark, table, branch, command, configuredScan, isolationLevel, info); + } + + return lazyWriteBuilder; + } + + @Override + public NamedReference[] requiredMetadataAttributes() { + List metadataAttributes = Lists.newArrayList(); + metadataAttributes.add(Expressions.column(MetadataColumns.SPEC_ID.name())); + metadataAttributes.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())); + } + + return metadataAttributes.toArray(new NamedReference[0]); + } + + @Override + public NamedReference[] rowId() { + NamedReference file = Expressions.column(MetadataColumns.FILE_PATH.name()); + NamedReference pos = Expressions.column(MetadataColumns.ROW_POSITION.name()); + return new NamedReference[] {file, pos}; + } + + @Override + public boolean representUpdateAsDeleteAndInsert() { + return true; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java new file mode 100644 index 000000000000..d072397dc6a3 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -0,0 +1,937 @@ +/* + * 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.IsolationLevel.SERIALIZABLE; +import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE; +import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.MERGE; +import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.UPDATE; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.function.Function; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.IsolationLevel; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.BaseDeleteLoader; +import org.apache.iceberg.data.DeleteLoader; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteIndex; +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; +import org.apache.iceberg.io.DataWriteResult; +import org.apache.iceberg.io.DeleteWriteResult; +import org.apache.iceberg.io.FanoutDataWriter; +import org.apache.iceberg.io.FanoutPositionOnlyDeleteWriter; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.PartitioningDVWriter; +import org.apache.iceberg.io.PartitioningWriter; +import org.apache.iceberg.io.PositionDeltaWriter; +import org.apache.iceberg.io.WriteResult; +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.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkWriteConf; +import org.apache.iceberg.spark.SparkWriteRequirements; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.DeleteFileSet; +import org.apache.iceberg.util.StructProjection; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.JoinedRow; +import org.apache.spark.sql.connector.distributions.Distribution; +import org.apache.spark.sql.connector.expressions.SortOrder; +import org.apache.spark.sql.connector.write.DeltaBatchWrite; +import org.apache.spark.sql.connector.write.DeltaWrite; +import org.apache.spark.sql.connector.write.DeltaWriter; +import org.apache.spark.sql.connector.write.DeltaWriterFactory; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.PhysicalWriteInfo; +import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering; +import org.apache.spark.sql.connector.write.RowLevelOperation.Command; +import org.apache.spark.sql.connector.write.WriterCommitMessage; +import org.apache.spark.sql.types.LongType$; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrdering { + + private static final Logger LOG = LoggerFactory.getLogger(SparkPositionDeltaWrite.class); + + private final JavaSparkContext sparkContext; + private final Table table; + 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; + private final Map writeProperties; + + private boolean cleanupOnAbort = false; + + SparkPositionDeltaWrite( + SparkSession spark, + Table table, + Command command, + SparkBatchQueryScan scan, + IsolationLevel isolationLevel, + SparkWriteConf writeConf, + LogicalWriteInfo info, + Schema dataSchema) { + this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + this.table = table; + 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); + this.writeProperties = writeConf.writeProperties(); + } + + @Override + public Distribution requiredDistribution() { + Distribution distribution = writeRequirements.distribution(); + LOG.debug("Requesting {} as write distribution for table {}", distribution, table.name()); + return distribution; + } + + @Override + public boolean distributionStrictlyRequired() { + return false; + } + + @Override + public SortOrder[] requiredOrdering() { + SortOrder[] ordering = writeRequirements.ordering(); + LOG.debug("Requesting {} as write ordering for table {}", ordering, table.name()); + return ordering; + } + + @Override + public long advisoryPartitionSizeInBytes() { + long size = writeRequirements.advisoryPartitionSize(); + LOG.debug("Requesting {} bytes advisory partition size for table {}", size, table.name()); + return size; + } + + @Override + public DeltaBatchWrite toBatch() { + return new PositionDeltaBatchWrite(); + } + + private class PositionDeltaBatchWrite implements DeltaBatchWrite { + + @Override + public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { + // broadcast large objects since the writer factory will be sent to executors + return new PositionDeltaWriteFactory( + sparkContext.broadcast(SerializableTableWithSize.copyOf(table)), + broadcastRewritableDeletes(), + command, + context, + writeProperties); + } + + private Broadcast> broadcastRewritableDeletes() { + if (scan != null && shouldRewriteDeletes()) { + Map rewritableDeletes = scan.rewritableDeletes(context.useDVs()); + if (rewritableDeletes != null && !rewritableDeletes.isEmpty()) { + return sparkContext.broadcast(rewritableDeletes); + } + } + return null; + } + + private boolean shouldRewriteDeletes() { + // deletes must be rewritten when there are DVs and file-scoped deletes + return context.useDVs() || context.deleteGranularity() == DeleteGranularity.FILE; + } + + @Override + public boolean useCommitCoordinator() { + return false; + } + + @Override + public void commit(WriterCommitMessage[] messages) { + RowDelta rowDelta = table.newRowDelta(); + + CharSequenceSet referencedDataFiles = CharSequenceSet.empty(); + + int addedDataFilesCount = 0; + int addedDeleteFilesCount = 0; + int removedDeleteFilesCount = 0; + + for (WriterCommitMessage message : messages) { + DeltaTaskCommit taskCommit = (DeltaTaskCommit) message; + + for (DataFile dataFile : taskCommit.dataFiles()) { + rowDelta.addRows(dataFile); + addedDataFilesCount += 1; + } + + for (DeleteFile deleteFile : taskCommit.deleteFiles()) { + rowDelta.addDeletes(deleteFile); + addedDeleteFilesCount += 1; + } + + for (DeleteFile deleteFile : taskCommit.rewrittenDeleteFiles()) { + rowDelta.removeDeletes(deleteFile); + removedDeleteFilesCount += 1; + } + + referencedDataFiles.addAll(Arrays.asList(taskCommit.referencedDataFiles())); + } + + // 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); + rowDelta.conflictDetectionFilter(conflictDetectionFilter); + + rowDelta.validateDataFilesExist(referencedDataFiles); + + if (scan.snapshotId() != null) { + // set the read snapshot ID to check only snapshots that happened after the table was read + // otherwise, the validation will go through all snapshots present in the table + rowDelta.validateFromSnapshot(scan.snapshotId()); + } + + if (command == UPDATE || command == MERGE) { + rowDelta.validateDeletedFiles(); + rowDelta.validateNoConflictingDeleteFiles(); + } + + if (isolationLevel == SERIALIZABLE) { + rowDelta.validateNoConflictingDataFiles(); + } + + String commitMsg = + String.format( + Locale.ROOT, + "position delta with %d data files, %d delete files and %d rewritten delete files" + + "(scanSnapshotId: %d, conflictDetectionFilter: %s, isolationLevel: %s)", + addedDataFilesCount, + addedDeleteFilesCount, + removedDeleteFilesCount, + scan.snapshotId(), + conflictDetectionFilter, + isolationLevel); + commitOperation(rowDelta, commitMsg); + + } else { + String commitMsg = + String.format( + Locale.ROOT, + "position delta with %d data files and %d delete files (no validation required)", + addedDataFilesCount, + addedDeleteFilesCount); + commitOperation(rowDelta, commitMsg); + } + } + + 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) { + SparkCleanupUtil.deleteFiles("job abort", table.io(), files(messages)); + } else { + LOG.warn("Skipping cleanup of written files"); + } + } + + private List> files(WriterCommitMessage[] messages) { + List> files = Lists.newArrayList(); + + for (WriterCommitMessage message : messages) { + if (message != null) { + DeltaTaskCommit taskCommit = (DeltaTaskCommit) message; + files.addAll(Arrays.asList(taskCommit.dataFiles())); + files.addAll(Arrays.asList(taskCommit.deleteFiles())); + } + } + + return files; + } + + private void commitOperation(SnapshotUpdate operation, String description) { + LOG.info("Committing {} to table {}", description, table); + if (applicationId != null) { + operation.set("spark.app.id", applicationId); + } + + extraSnapshotMetadata.forEach(operation::set); + + CommitMetadata.commitProperties().forEach(operation::set); + + if (wapEnabled && wapId != null) { + // write-audit-publish is enabled for this table and job + // stage the changes without changing the current snapshot + operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId); + operation.stageOnly(); + } + + if (branch != null) { + operation.toBranch(branch); + } + + try { + long start = System.currentTimeMillis(); + operation.commit(); // abort is automatically called if this fails + long duration = System.currentTimeMillis() - start; + LOG.info("Committed in {} ms", duration); + } catch (Exception e) { + cleanupOnAbort = e instanceof CleanableFailure; + throw e; + } + } + } + + public static class DeltaTaskCommit implements WriterCommitMessage { + private final DataFile[] dataFiles; + private final DeleteFile[] deleteFiles; + private final DeleteFile[] rewrittenDeleteFiles; + private final CharSequence[] referencedDataFiles; + + DeltaTaskCommit(WriteResult result) { + this.dataFiles = result.dataFiles(); + this.deleteFiles = result.deleteFiles(); + this.referencedDataFiles = result.referencedDataFiles(); + this.rewrittenDeleteFiles = result.rewrittenDeleteFiles(); + } + + DeltaTaskCommit(DeleteWriteResult result) { + this.dataFiles = new DataFile[0]; + this.deleteFiles = result.deleteFiles().toArray(new DeleteFile[0]); + this.referencedDataFiles = result.referencedDataFiles().toArray(new CharSequence[0]); + this.rewrittenDeleteFiles = result.rewrittenDeleteFiles().toArray(new DeleteFile[0]); + } + + DataFile[] dataFiles() { + return dataFiles; + } + + DeleteFile[] deleteFiles() { + return deleteFiles; + } + + DeleteFile[] rewrittenDeleteFiles() { + return rewrittenDeleteFiles; + } + + CharSequence[] referencedDataFiles() { + return referencedDataFiles; + } + } + + private static class PositionDeltaWriteFactory implements DeltaWriterFactory { + private final Broadcast

    tableBroadcast; + private final Broadcast> rewritableDeletesBroadcast; + private final Command command; + private final Context context; + private final Map writeProperties; + + PositionDeltaWriteFactory( + Broadcast
    tableBroadcast, + Broadcast> rewritableDeletesBroadcast, + Command command, + Context context, + Map writeProperties) { + this.tableBroadcast = tableBroadcast; + this.rewritableDeletesBroadcast = rewritableDeletesBroadcast; + this.command = command; + this.context = context; + this.writeProperties = writeProperties; + } + + @Override + public DeltaWriter createWriter(int partitionId, long taskId) { + Table table = tableBroadcast.value(); + + OutputFileFactory dataFileFactory = + OutputFileFactory.builderFor(table, partitionId, taskId) + .format(context.dataFileFormat()) + .operationId(context.queryId()) + .build(); + OutputFileFactory deleteFileFactory = + OutputFileFactory.builderFor(table, partitionId, taskId) + .format(context.deleteFileFormat()) + .operationId(context.queryId()) + .suffix("deletes") + .build(); + + SparkFileWriterFactory writerFactory = + SparkFileWriterFactory.builderFor(table) + .dataFileFormat(context.dataFileFormat()) + .dataSchema(context.dataSchema()) + .dataSparkType(context.dataSparkType()) + .deleteFileFormat(context.deleteFileFormat()) + .positionDeleteSparkType(context.deleteSparkType()) + .writeProperties(writeProperties) + .build(); + + if (command == DELETE) { + return new DeleteOnlyDeltaWriter( + table, rewritableDeletes(), writerFactory, deleteFileFactory, context); + + } else if (table.spec().isUnpartitioned()) { + return new UnpartitionedDeltaWriter( + table, + rewritableDeletes(), + writerFactory, + dataFileFactory, + deleteFileFactory, + new ExtractRowLineage(context.dataSchema()), + context); + + } else { + return new PartitionedDeltaWriter( + table, + rewritableDeletes(), + writerFactory, + dataFileFactory, + deleteFileFactory, + new ExtractRowLineage(context.dataSchema()), + context); + } + } + + private Map rewritableDeletes() { + return rewritableDeletesBroadcast != null ? rewritableDeletesBroadcast.getValue() : null; + } + } + + private abstract static class BaseDeltaWriter implements DeltaWriter { + + protected InternalRowWrapper initPartitionRowWrapper(Types.StructType partitionType) { + StructType sparkPartitionType = (StructType) SparkSchemaUtil.convert(partitionType); + return new InternalRowWrapper(sparkPartitionType, partitionType); + } + + protected Map buildPartitionProjections( + Types.StructType partitionType, Map specs) { + Map partitionProjections = Maps.newHashMap(); + + for (int specId : specs.keySet()) { + PartitionSpec spec = specs.get(specId); + StructProjection projection = StructProjection.create(partitionType, spec.partitionType()); + partitionProjections.put(specId, projection); + } + + return partitionProjections; + } + + // use a fanout writer only if enabled and the input is unordered and the table is partitioned + protected PartitioningWriter newDataWriter( + Table table, SparkFileWriterFactory writers, OutputFileFactory files, Context context) { + + FileIO io = table.io(); + boolean useFanoutWriter = context.useFanoutWriter(); + long targetFileSize = context.targetDataFileSize(); + + if (table.spec().isPartitioned() && useFanoutWriter) { + return new FanoutDataWriter<>(writers, files, io, targetFileSize); + } else { + return new ClusteredDataWriter<>(writers, files, io, targetFileSize); + } + } + + // Use the DV writer for V3+ tables + // The spec requires position deletes to be ordered by file and position for V2 tables + // use a fanout writer if the input is unordered no matter whether fanout writers are enabled + // clustered writers assume that the position deletes are already ordered by file and position + protected PartitioningWriter, DeleteWriteResult> newDeleteWriter( + Table table, + Map rewritableDeletes, + SparkFileWriterFactory writers, + OutputFileFactory files, + Context context) { + Function previousDeleteLoader = + PreviousDeleteLoader.create(table, rewritableDeletes); + FileIO io = table.io(); + boolean inputOrdered = context.inputOrdered(); + long targetFileSize = context.targetDeleteFileSize(); + DeleteGranularity deleteGranularity = context.deleteGranularity(); + + if (context.useDVs()) { + return new PartitioningDVWriter<>(files, previousDeleteLoader); + } else if (inputOrdered && rewritableDeletes == null) { + return new ClusteredPositionDeleteWriter<>( + writers, files, io, targetFileSize, deleteGranularity); + } else { + return new FanoutPositionOnlyDeleteWriter<>( + writers, files, io, targetFileSize, deleteGranularity, previousDeleteLoader); + } + } + } + + private static class PreviousDeleteLoader implements Function { + private final Map deleteFiles; + private final DeleteLoader deleteLoader; + + private PreviousDeleteLoader(Table table, Map deleteFiles) { + this.deleteFiles = deleteFiles; + this.deleteLoader = + new BaseDeleteLoader( + deleteFile -> + EncryptingFileIO.combine(table.io(), table.encryption()) + .newInputFile(deleteFile)); + } + + @Override + public PositionDeleteIndex apply(CharSequence path) { + DeleteFileSet deleteFileSet = deleteFiles.get(path.toString()); + if (deleteFileSet == null) { + return null; + } + + return deleteLoader.loadPositionDeletes(deleteFileSet, path); + } + + public static Function create( + Table table, Map deleteFiles) { + if (deleteFiles == null) { + return path -> null; + } + + return new PreviousDeleteLoader(table, deleteFiles); + } + } + + private static class DeleteOnlyDeltaWriter extends BaseDeltaWriter { + private final PartitioningWriter, DeleteWriteResult> delegate; + private final PositionDelete positionDelete; + private final FileIO io; + private final Map specs; + private final InternalRowWrapper partitionRowWrapper; + private final Map partitionProjections; + private final int specIdOrdinal; + private final int partitionOrdinal; + private final int fileOrdinal; + private final int positionOrdinal; + + private boolean closed = false; + + DeleteOnlyDeltaWriter( + Table table, + Map rewritableDeletes, + SparkFileWriterFactory writerFactory, + OutputFileFactory deleteFileFactory, + Context context) { + + this.delegate = + newDeleteWriter(table, rewritableDeletes, writerFactory, deleteFileFactory, context); + this.positionDelete = PositionDelete.create(); + this.io = table.io(); + this.specs = table.specs(); + + Types.StructType partitionType = Partitioning.partitionType(table); + this.partitionRowWrapper = initPartitionRowWrapper(partitionType); + this.partitionProjections = buildPartitionProjections(partitionType, specs); + + this.specIdOrdinal = context.specIdOrdinal(); + this.partitionOrdinal = context.partitionOrdinal(); + this.fileOrdinal = context.fileOrdinal(); + this.positionOrdinal = context.positionOrdinal(); + } + + @Override + public void delete(InternalRow metadata, InternalRow id) throws IOException { + int specId = metadata.getInt(specIdOrdinal); + PartitionSpec spec = specs.get(specId); + + InternalRow partition = metadata.getStruct(partitionOrdinal, partitionRowWrapper.size()); + StructProjection partitionProjection = partitionProjections.get(specId); + partitionProjection.wrap(partitionRowWrapper.wrap(partition)); + + String file = id.getString(fileOrdinal); + long position = id.getLong(positionOrdinal); + positionDelete.set(file, position); + delegate.write(positionDelete, spec, partitionProjection); + } + + @Override + public void update(InternalRow metadata, InternalRow id, InternalRow row) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement update"); + } + + @Override + public void insert(InternalRow row) throws IOException { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement insert"); + } + + @Override + public WriterCommitMessage commit() throws IOException { + close(); + + DeleteWriteResult result = delegate.result(); + return new DeltaTaskCommit(result); + } + + @Override + public void abort() throws IOException { + close(); + + DeleteWriteResult result = delegate.result(); + SparkCleanupUtil.deleteTaskFiles(io, result.deleteFiles()); + } + + @Override + public void close() throws IOException { + if (!closed) { + delegate.close(); + this.closed = true; + } + } + } + + @SuppressWarnings("checkstyle:VisibilityModifier") + private abstract static class DeleteAndDataDeltaWriter extends BaseDeltaWriter { + protected final PositionDeltaWriter delegate; + protected final Function rowLineageExtractor; + + private final FileIO io; + private final Map specs; + private final InternalRowWrapper deletePartitionRowWrapper; + private final Map deletePartitionProjections; + private final int specIdOrdinal; + private final int partitionOrdinal; + private final int fileOrdinal; + private final int positionOrdinal; + private boolean closed = false; + + DeleteAndDataDeltaWriter( + Table table, + Map rewritableDeletes, + SparkFileWriterFactory writerFactory, + OutputFileFactory dataFileFactory, + OutputFileFactory deleteFileFactory, + Function rowLineageExtractor, + Context context) { + + this.delegate = + new BasePositionDeltaWriter<>( + newDataWriter(table, writerFactory, dataFileFactory, context), + newDeleteWriter(table, rewritableDeletes, writerFactory, deleteFileFactory, context)); + this.io = table.io(); + this.specs = table.specs(); + + Types.StructType partitionType = Partitioning.partitionType(table); + this.deletePartitionRowWrapper = initPartitionRowWrapper(partitionType); + this.deletePartitionProjections = buildPartitionProjections(partitionType, specs); + + this.rowLineageExtractor = rowLineageExtractor; + + this.specIdOrdinal = context.specIdOrdinal(); + this.partitionOrdinal = context.partitionOrdinal(); + this.fileOrdinal = context.fileOrdinal(); + this.positionOrdinal = context.positionOrdinal(); + } + + @Override + public void delete(InternalRow meta, InternalRow id) throws IOException { + int specId = meta.getInt(specIdOrdinal); + PartitionSpec spec = specs.get(specId); + + InternalRow partition = meta.getStruct(partitionOrdinal, deletePartitionRowWrapper.size()); + StructProjection partitionProjection = deletePartitionProjections.get(specId); + partitionProjection.wrap(deletePartitionRowWrapper.wrap(partition)); + + String file = id.getString(fileOrdinal); + long position = id.getLong(positionOrdinal); + delegate.delete(file, position, spec, partitionProjection); + } + + @Override + public WriterCommitMessage commit() throws IOException { + close(); + + WriteResult result = delegate.result(); + return new DeltaTaskCommit(result); + } + + @Override + public void abort() throws IOException { + close(); + + WriteResult result = delegate.result(); + SparkCleanupUtil.deleteTaskFiles(io, files(result)); + } + + private List> files(WriteResult result) { + List> files = Lists.newArrayList(); + files.addAll(Arrays.asList(result.dataFiles())); + files.addAll(Arrays.asList(result.deleteFiles())); + return files; + } + + @Override + public void close() throws IOException { + if (!closed) { + delegate.close(); + this.closed = true; + } + } + + protected InternalRow decorateWithRowLineage(InternalRow meta, InternalRow data) { + InternalRow rowLineage = rowLineageExtractor.apply(meta); + return rowLineage == null ? data : new JoinedRow(data, rowLineage); + } + } + + private static class UnpartitionedDeltaWriter extends DeleteAndDataDeltaWriter { + private final PartitionSpec dataSpec; + + UnpartitionedDeltaWriter( + Table table, + Map rewritableDeletes, + SparkFileWriterFactory writerFactory, + OutputFileFactory dataFileFactory, + OutputFileFactory deleteFileFactory, + Function rowLineageFromMetadata, + Context context) { + super( + table, + rewritableDeletes, + writerFactory, + dataFileFactory, + deleteFileFactory, + rowLineageFromMetadata, + context); + this.dataSpec = table.spec(); + } + + @Override + public void update(InternalRow meta, InternalRow id, InternalRow row) throws IOException { + throw new UnsupportedOperationException("Update must be represented as delete and insert"); + } + + @Override + public void insert(InternalRow row) throws IOException { + reinsert(null, row); + } + + @Override + public void reinsert(InternalRow meta, InternalRow row) throws IOException { + delegate.insert(decorateWithRowLineage(meta, row), dataSpec, null); + } + } + + private static class PartitionedDeltaWriter extends DeleteAndDataDeltaWriter { + private final PartitionSpec dataSpec; + private final PartitionKey dataPartitionKey; + private final InternalRowWrapper internalRowDataWrapper; + + PartitionedDeltaWriter( + Table table, + Map rewritableDeletes, + SparkFileWriterFactory writerFactory, + OutputFileFactory dataFileFactory, + OutputFileFactory deleteFileFactory, + Function rowLineageFromMetadata, + Context context) { + super( + table, + rewritableDeletes, + writerFactory, + dataFileFactory, + deleteFileFactory, + rowLineageFromMetadata, + context); + + this.dataSpec = table.spec(); + this.dataPartitionKey = new PartitionKey(dataSpec, context.dataSchema()); + this.internalRowDataWrapper = + new InternalRowWrapper(context.dataSparkType(), context.dataSchema().asStruct()); + } + + @Override + public void update(InternalRow meta, InternalRow id, InternalRow row) throws IOException { + throw new UnsupportedOperationException("Update must be represented as delete and insert"); + } + + @Override + public void insert(InternalRow row) throws IOException { + reinsert(null, row); + } + + @Override + public void reinsert(InternalRow meta, InternalRow row) throws IOException { + dataPartitionKey.partition(internalRowDataWrapper.wrap(row)); + delegate.insert(decorateWithRowLineage(meta, row), dataSpec, dataPartitionKey); + } + } + + // a serializable helper class for common parameters required to configure writers + private static class Context implements Serializable { + private final Schema dataSchema; + private StructType dataSparkType; + private final FileFormat dataFileFormat; + private final long targetDataFileSize; + private final StructType deleteSparkType; + private final StructType metadataSparkType; + private final FileFormat deleteFileFormat; + private final long targetDeleteFileSize; + private final DeleteGranularity deleteGranularity; + private final String queryId; + private final boolean useFanoutWriter; + private final boolean inputOrdered; + + Context( + Schema dataSchema, + SparkWriteConf writeConf, + LogicalWriteInfo info, + SparkWriteRequirements writeRequirements) { + this.dataSchema = dataSchema; + this.dataSparkType = info.schema(); + if (dataSchema != null && dataSchema.findField(MetadataColumns.ROW_ID.fieldId()) != null) { + dataSparkType = dataSparkType.add(MetadataColumns.ROW_ID.name(), LongType$.MODULE$); + dataSparkType = + dataSparkType.add( + MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name(), LongType$.MODULE$); + } + + this.dataFileFormat = writeConf.dataFileFormat(); + this.targetDataFileSize = writeConf.targetDataFileSize(); + this.deleteSparkType = info.rowIdSchema().get(); + this.deleteFileFormat = writeConf.deleteFileFormat(); + this.targetDeleteFileSize = writeConf.targetDeleteFileSize(); + this.deleteGranularity = writeConf.deleteGranularity(); + this.metadataSparkType = info.metadataSchema().get(); + this.queryId = info.queryId(); + this.useFanoutWriter = writeConf.useFanoutWriter(writeRequirements); + this.inputOrdered = writeRequirements.hasOrdering(); + } + + Schema dataSchema() { + return dataSchema; + } + + StructType dataSparkType() { + return dataSparkType; + } + + FileFormat dataFileFormat() { + return dataFileFormat; + } + + long targetDataFileSize() { + return targetDataFileSize; + } + + /* @deprecated This method is deprecated as of version 1.11.0 and will be removed in 1.12.0. + * Position deletes that include row data are no longer supported. + */ + StructType deleteSparkType() { + return deleteSparkType; + } + + FileFormat deleteFileFormat() { + return deleteFileFormat; + } + + long targetDeleteFileSize() { + return targetDeleteFileSize; + } + + DeleteGranularity deleteGranularity() { + return deleteGranularity; + } + + String queryId() { + return queryId; + } + + boolean useFanoutWriter() { + return useFanoutWriter; + } + + boolean inputOrdered() { + return inputOrdered; + } + + boolean useDVs() { + return deleteFileFormat == FileFormat.PUFFIN; + } + + int specIdOrdinal() { + return metadataSparkType.fieldIndex(MetadataColumns.SPEC_ID.name()); + } + + int partitionOrdinal() { + return metadataSparkType.fieldIndex(MetadataColumns.PARTITION_COLUMN_NAME); + } + + int fileOrdinal() { + return deleteSparkType.fieldIndex(MetadataColumns.FILE_PATH.name()); + } + + int positionOrdinal() { + return deleteSparkType.fieldIndex(MetadataColumns.ROW_POSITION.name()); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java new file mode 100644 index 000000000000..9151e41c0cf9 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWriteBuilder.java @@ -0,0 +1,135 @@ +/* + * 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.MetadataColumns.SPEC_ID_COLUMN_DOC; +import static org.apache.iceberg.MetadataColumns.SPEC_ID_COLUMN_ID; +import static org.apache.iceberg.MetadataColumns.schemaWithRowLineage; + +import org.apache.iceberg.IsolationLevel; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +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.SparkUtil; +import org.apache.iceberg.spark.SparkWriteConf; +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.write.DeltaWrite; +import org.apache.spark.sql.connector.write.DeltaWriteBuilder; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.RowLevelOperation.Command; +import org.apache.spark.sql.types.StructType; + +class SparkPositionDeltaWriteBuilder implements DeltaWriteBuilder { + + private static final Schema EXPECTED_ROW_ID_SCHEMA = + new Schema(MetadataColumns.FILE_PATH, MetadataColumns.ROW_POSITION); + + private final SparkSession spark; + private final Table table; + private final Command command; + private final SparkBatchQueryScan scan; + private final IsolationLevel isolationLevel; + private final SparkWriteConf writeConf; + private final LogicalWriteInfo info; + private final boolean checkNullability; + private final boolean checkOrdering; + + SparkPositionDeltaWriteBuilder( + SparkSession spark, + Table table, + String branch, + Command command, + Scan scan, + IsolationLevel isolationLevel, + LogicalWriteInfo info) { + this.spark = spark; + this.table = table; + this.command = command; + this.scan = (SparkBatchQueryScan) scan; + this.isolationLevel = isolationLevel; + this.writeConf = new SparkWriteConf(spark, table, branch, info.options()); + this.info = info; + this.checkNullability = writeConf.checkNullability(); + this.checkOrdering = writeConf.checkOrdering(); + } + + @Override + public DeltaWrite build() { + Schema dataSchema = dataSchema(); + + validateRowIdSchema(); + validateMetadataSchema(); + if (dataSchema != null + && info.metadataSchema().isPresent() + && info.metadataSchema() + .get() + .find(f -> f.name().equals(MetadataColumns.ROW_ID.name())) + .isDefined()) { + dataSchema = MetadataColumns.schemaWithRowLineage(dataSchema); + } + + SparkUtil.validatePartitionTransforms(table.spec()); + + return new SparkPositionDeltaWrite( + spark, table, command, scan, isolationLevel, writeConf, info, dataSchema); + } + + private Schema dataSchema() { + if (info.schema() == null || info.schema().isEmpty()) { + return null; + } else { + Schema dataSchema = SparkSchemaUtil.convert(table.schema(), info.schema()); + validateSchema("data", table.schema(), dataSchema); + return dataSchema; + } + } + + private void validateRowIdSchema() { + Preconditions.checkArgument(info.rowIdSchema().isPresent(), "Row ID schema must be set"); + StructType rowIdSparkType = info.rowIdSchema().get(); + Schema rowIdSchema = SparkSchemaUtil.convert(EXPECTED_ROW_ID_SCHEMA, rowIdSparkType); + validateSchema("row ID", EXPECTED_ROW_ID_SCHEMA, rowIdSchema); + } + + private void validateMetadataSchema() { + Preconditions.checkArgument(info.metadataSchema().isPresent(), "Metadata schema must be set"); + Schema expectedMetadataSchema = + new Schema( + Types.NestedField.optional( + SPEC_ID_COLUMN_ID, "_spec_id", Types.IntegerType.get(), SPEC_ID_COLUMN_DOC), + MetadataColumns.metadataColumn(table, MetadataColumns.PARTITION_COLUMN_NAME)); + if (TableUtil.supportsRowLineage(table)) { + expectedMetadataSchema = schemaWithRowLineage(expectedMetadataSchema); + } + + StructType metadataSparkType = info.metadataSchema().get(); + Schema metadataSchema = SparkSchemaUtil.convert(expectedMetadataSchema, metadataSparkType); + validateSchema("metadata", expectedMetadataSchema, metadataSchema); + } + + private void validateSchema(String context, Schema expected, Schema actual) { + TypeUtil.validateSchema(context, expected, actual, checkNullability, checkOrdering); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java new file mode 100644 index 000000000000..b113bd9b25af --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowLevelOperationBuilder.java @@ -0,0 +1,114 @@ +/* + * 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.DELETE_ISOLATION_LEVEL; +import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL_DEFAULT; +import static org.apache.iceberg.TableProperties.DELETE_MODE; +import static org.apache.iceberg.TableProperties.DELETE_MODE_DEFAULT; +import static org.apache.iceberg.TableProperties.MERGE_ISOLATION_LEVEL; +import static org.apache.iceberg.TableProperties.MERGE_ISOLATION_LEVEL_DEFAULT; +import static org.apache.iceberg.TableProperties.MERGE_MODE; +import static org.apache.iceberg.TableProperties.MERGE_MODE_DEFAULT; +import static org.apache.iceberg.TableProperties.UPDATE_ISOLATION_LEVEL; +import static org.apache.iceberg.TableProperties.UPDATE_ISOLATION_LEVEL_DEFAULT; +import static org.apache.iceberg.TableProperties.UPDATE_MODE; +import static org.apache.iceberg.TableProperties.UPDATE_MODE_DEFAULT; + +import java.util.Map; +import org.apache.iceberg.IsolationLevel; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Table; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.write.RowLevelOperation; +import org.apache.spark.sql.connector.write.RowLevelOperation.Command; +import org.apache.spark.sql.connector.write.RowLevelOperationBuilder; +import org.apache.spark.sql.connector.write.RowLevelOperationInfo; + +class SparkRowLevelOperationBuilder implements RowLevelOperationBuilder { + + private final SparkSession spark; + private final Table table; + 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) { + this.spark = spark; + this.table = table; + this.branch = branch; + this.info = info; + this.mode = mode(table.properties(), info.command()); + this.isolationLevel = isolationLevel(table.properties(), info.command()); + } + + @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); + } + } + + private RowLevelOperationMode mode(Map properties, Command command) { + String modeName; + + switch (command) { + case DELETE: + modeName = properties.getOrDefault(DELETE_MODE, DELETE_MODE_DEFAULT); + break; + case UPDATE: + modeName = properties.getOrDefault(UPDATE_MODE, UPDATE_MODE_DEFAULT); + break; + case MERGE: + modeName = properties.getOrDefault(MERGE_MODE, MERGE_MODE_DEFAULT); + break; + default: + throw new IllegalArgumentException("Unsupported command: " + command); + } + + return RowLevelOperationMode.fromName(modeName); + } + + private IsolationLevel isolationLevel(Map properties, Command command) { + String levelName; + + switch (command) { + case DELETE: + levelName = properties.getOrDefault(DELETE_ISOLATION_LEVEL, DELETE_ISOLATION_LEVEL_DEFAULT); + break; + case UPDATE: + levelName = properties.getOrDefault(UPDATE_ISOLATION_LEVEL, UPDATE_ISOLATION_LEVEL_DEFAULT); + break; + case MERGE: + levelName = properties.getOrDefault(MERGE_ISOLATION_LEVEL, MERGE_ISOLATION_LEVEL_DEFAULT); + break; + default: + throw new IllegalArgumentException("Unsupported command: " + command); + } + + return IsolationLevel.fromName(levelName); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowReaderFactory.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowReaderFactory.java new file mode 100644 index 000000000000..23699aeb167c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkRowReaderFactory.java @@ -0,0 +1,68 @@ +/* + * 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.ChangelogScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.PartitionReader; +import org.apache.spark.sql.connector.read.PartitionReaderFactory; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +class SparkRowReaderFactory implements PartitionReaderFactory { + + SparkRowReaderFactory() {} + + @Override + public PartitionReader createReader(InputPartition inputPartition) { + Preconditions.checkArgument( + inputPartition instanceof SparkInputPartition, + "Unknown input partition type: %s", + inputPartition.getClass().getName()); + + SparkInputPartition partition = (SparkInputPartition) inputPartition; + + if (partition.allTasksOfType(FileScanTask.class)) { + return new RowDataReader(partition); + + } else if (partition.allTasksOfType(ChangelogScanTask.class)) { + return new ChangelogRowReader(partition); + + } else if (partition.allTasksOfType(PositionDeletesScanTask.class)) { + return new PositionDeletesRowReader(partition); + + } else { + throw new UnsupportedOperationException( + "Unsupported task group for row-based reads: " + partition.taskGroup()); + } + } + + @Override + public PartitionReader createColumnarReader(InputPartition inputPartition) { + throw new UnsupportedOperationException("Columnar reads are not supported"); + } + + @Override + public boolean supportColumnarReads(InputPartition inputPartition) { + return false; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java new file mode 100644 index 000000000000..106b296de098 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -0,0 +1,354 @@ +/* + * 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.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import org.apache.iceberg.BlobMetadata; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.StatisticsFile; +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.Strings; +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; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.source.metrics.EqualityDeleteFiles; +import org.apache.iceberg.spark.source.metrics.IndexedDeleteFiles; +import org.apache.iceberg.spark.source.metrics.NumDeletes; +import org.apache.iceberg.spark.source.metrics.NumSplits; +import org.apache.iceberg.spark.source.metrics.PositionalDeleteFiles; +import org.apache.iceberg.spark.source.metrics.ResultDataFiles; +import org.apache.iceberg.spark.source.metrics.ResultDeleteFiles; +import org.apache.iceberg.spark.source.metrics.ScannedDataManifests; +import org.apache.iceberg.spark.source.metrics.ScannedDeleteManifests; +import org.apache.iceberg.spark.source.metrics.SkippedDataFiles; +import org.apache.iceberg.spark.source.metrics.SkippedDataManifests; +import org.apache.iceberg.spark.source.metrics.SkippedDeleteFiles; +import org.apache.iceberg.spark.source.metrics.SkippedDeleteManifests; +import org.apache.iceberg.spark.source.metrics.TaskEqualityDeleteFiles; +import org.apache.iceberg.spark.source.metrics.TaskIndexedDeleteFiles; +import org.apache.iceberg.spark.source.metrics.TaskPositionalDeleteFiles; +import org.apache.iceberg.spark.source.metrics.TaskResultDataFiles; +import org.apache.iceberg.spark.source.metrics.TaskResultDeleteFiles; +import org.apache.iceberg.spark.source.metrics.TaskScannedDataManifests; +import org.apache.iceberg.spark.source.metrics.TaskScannedDeleteManifests; +import org.apache.iceberg.spark.source.metrics.TaskSkippedDataFiles; +import org.apache.iceberg.spark.source.metrics.TaskSkippedDataManifests; +import org.apache.iceberg.spark.source.metrics.TaskSkippedDeleteFiles; +import org.apache.iceberg.spark.source.metrics.TaskSkippedDeleteManifests; +import org.apache.iceberg.spark.source.metrics.TaskTotalDataFileSize; +import org.apache.iceberg.spark.source.metrics.TaskTotalDataManifests; +import org.apache.iceberg.spark.source.metrics.TaskTotalDeleteFileSize; +import org.apache.iceberg.spark.source.metrics.TaskTotalDeleteManifests; +import org.apache.iceberg.spark.source.metrics.TaskTotalPlanningDuration; +import org.apache.iceberg.spark.source.metrics.TotalDataFileSize; +import org.apache.iceberg.spark.source.metrics.TotalDataManifests; +import org.apache.iceberg.spark.source.metrics.TotalDeleteFileSize; +import org.apache.iceberg.spark.source.metrics.TotalDeleteManifests; +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; +import org.apache.spark.sql.connector.expressions.FieldReference; +import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.metric.CustomMetric; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; +import org.apache.spark.sql.connector.read.Batch; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.connector.read.Statistics; +import org.apache.spark.sql.connector.read.SupportsReportStatistics; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; +import org.apache.spark.sql.connector.read.streaming.MicroBatchStream; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +abstract class SparkScan implements Scan, SupportsReportStatistics { + private static final Logger LOG = LoggerFactory.getLogger(SparkScan.class); + private static final String NDV_KEY = "ndv"; + + private final JavaSparkContext sparkContext; + private final Table table; + 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 Supplier scanReportSupplier; + + // lazy variables + private StructType readSchema; + + SparkScan( + SparkSession spark, + Table table, + SparkReadConf readConf, + Schema expectedSchema, + 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.readConf = readConf; + this.caseSensitive = readConf.caseSensitive(); + this.expectedSchema = expectedSchema; + this.filterExpressions = filters != null ? filters : Collections.emptyList(); + this.branch = readConf.branch(); + this.scanReportSupplier = scanReportSupplier; + } + + protected Table table() { + return table; + } + + protected String branch() { + return branch; + } + + protected boolean caseSensitive() { + return caseSensitive; + } + + protected Schema expectedSchema() { + return expectedSchema; + } + + protected List filterExpressions() { + return filterExpressions; + } + + protected Types.StructType groupingKeyType() { + return Types.StructType.of(); + } + + protected abstract List> taskGroups(); + + @Override + public Batch toBatch() { + return new SparkBatch( + sparkContext, table, readConf, groupingKeyType(), taskGroups(), expectedSchema, hashCode()); + } + + @Override + public MicroBatchStream toMicroBatchStream(String checkpointLocation) { + return new SparkMicroBatchStream( + sparkContext, table, readConf, expectedSchema, checkpointLocation); + } + + @Override + public StructType readSchema() { + if (readSchema == null) { + this.readSchema = SparkSchemaUtil.convert(expectedSchema); + } + return readSchema; + } + + @Override + public Statistics estimateStatistics() { + return estimateStatistics(SnapshotUtil.latestSnapshot(table, branch)); + } + + protected Statistics estimateStatistics(Snapshot snapshot) { + // its a fresh table, no data + if (snapshot == null) { + return new Stats(0L, 0L, Collections.emptyMap()); + } + + boolean cboEnabled = + Boolean.parseBoolean(spark.conf().get(SQLConf.CBO_ENABLED().key(), "false")); + Map colStatsMap = Collections.emptyMap(); + if (readConf.reportColumnStats() && cboEnabled) { + colStatsMap = Maps.newHashMap(); + List files = table.statisticsFiles(); + Optional file = + files.stream().filter(f -> f.snapshotId() == snapshot.snapshotId()).findFirst(); + if (file.isPresent()) { + List metadataList = file.get().blobMetadata(); + + Map> groupedByField = + metadataList.stream() + .collect( + Collectors.groupingBy( + metadata -> metadata.fields().get(0), Collectors.toList())); + + for (Map.Entry> entry : groupedByField.entrySet()) { + String colName = table.schema().findColumnName(entry.getKey()); + NamedReference ref = FieldReference.column(colName); + Long ndv = null; + + for (BlobMetadata blobMetadata : entry.getValue()) { + if (blobMetadata + .type() + .equals(org.apache.iceberg.puffin.StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1)) { + String ndvStr = blobMetadata.properties().get(NDV_KEY); + if (!Strings.isNullOrEmpty(ndvStr)) { + ndv = Long.parseLong(ndvStr); + } else { + LOG.debug("{} is not set in BlobMetadata for column {}", NDV_KEY, colName); + } + } else { + LOG.debug("Blob type {} is not supported yet", blobMetadata.type()); + } + } + ColumnStatistics colStats = + new SparkColumnStatistics(ndv, null, null, null, null, null, null); + + colStatsMap.put(ref, colStats); + } + } + } + + // estimate stats using snapshot summary only for partitioned tables + // (metadata tables are unpartitioned) + if (!table.spec().isUnpartitioned() && filterExpressions.isEmpty()) { + LOG.debug( + "Using snapshot {} metadata to estimate statistics for table {}", + snapshot.snapshotId(), + table.name()); + long totalRecords = totalRecords(snapshot); + return new Stats( + SparkSchemaUtil.estimateSize(readSchema(), totalRecords), totalRecords, colStatsMap); + } + + long rowsCount = taskGroups().stream().mapToLong(ScanTaskGroup::estimatedRowsCount).sum(); + long sizeInBytes = SparkSchemaUtil.estimateSize(readSchema(), rowsCount); + return new Stats(sizeInBytes, rowsCount, colStatsMap); + } + + private long totalRecords(Snapshot snapshot) { + Map summary = snapshot.summary(); + return PropertyUtil.propertyAsLong(summary, SnapshotSummary.TOTAL_RECORDS_PROP, Long.MAX_VALUE); + } + + @Override + public String description() { + String groupingKeyFieldNamesAsString = + groupingKeyType().fields().stream() + .map(Types.NestedField::name) + .collect(Collectors.joining(", ")); + + return String.format( + "%s (branch=%s) [filters=%s, groupedBy=%s]", + table(), branch(), Spark3Util.describe(filterExpressions), groupingKeyFieldNamesAsString); + } + + @Override + public CustomTaskMetric[] reportDriverMetrics() { + ScanReport scanReport = scanReportSupplier != null ? scanReportSupplier.get() : null; + + if (scanReport == null) { + return new CustomTaskMetric[0]; + } + + List driverMetrics = Lists.newArrayList(); + + // common + driverMetrics.add(TaskTotalPlanningDuration.from(scanReport)); + + // data manifests + driverMetrics.add(TaskTotalDataManifests.from(scanReport)); + driverMetrics.add(TaskScannedDataManifests.from(scanReport)); + driverMetrics.add(TaskSkippedDataManifests.from(scanReport)); + + // data files + driverMetrics.add(TaskResultDataFiles.from(scanReport)); + driverMetrics.add(TaskSkippedDataFiles.from(scanReport)); + driverMetrics.add(TaskTotalDataFileSize.from(scanReport)); + + // delete manifests + driverMetrics.add(TaskTotalDeleteManifests.from(scanReport)); + driverMetrics.add(TaskScannedDeleteManifests.from(scanReport)); + driverMetrics.add(TaskSkippedDeleteManifests.from(scanReport)); + + // delete files + driverMetrics.add(TaskTotalDeleteFileSize.from(scanReport)); + driverMetrics.add(TaskResultDeleteFiles.from(scanReport)); + driverMetrics.add(TaskEqualityDeleteFiles.from(scanReport)); + driverMetrics.add(TaskIndexedDeleteFiles.from(scanReport)); + driverMetrics.add(TaskPositionalDeleteFiles.from(scanReport)); + driverMetrics.add(TaskSkippedDeleteFiles.from(scanReport)); + + return driverMetrics.toArray(new CustomTaskMetric[0]); + } + + @Override + public CustomMetric[] supportedCustomMetrics() { + return new CustomMetric[] { + // task metrics + new NumSplits(), + new NumDeletes(), + + // common + new TotalPlanningDuration(), + + // data manifests + new TotalDataManifests(), + new ScannedDataManifests(), + new SkippedDataManifests(), + + // data files + new ResultDataFiles(), + new SkippedDataFiles(), + new TotalDataFileSize(), + + // delete manifests + new TotalDeleteManifests(), + new ScannedDeleteManifests(), + new SkippedDeleteManifests(), + + // delete files + new TotalDeleteFileSize(), + new ResultDeleteFiles(), + new EqualityDeleteFiles(), + new IndexedDeleteFiles(), + new PositionalDeleteFiles(), + new SkippedDeleteFiles() + }; + } + + protected long adjustSplitSize(List tasks, long splitSize) { + if (readConf.splitSizeOption() == null && readConf.adaptiveSplitSizeEnabled()) { + long scanSize = tasks.stream().mapToLong(ScanTask::sizeBytes).sum(); + int parallelism = readConf.parallelism(); + return TableScanUtil.adjustSplitSize(scanSize, parallelism, splitSize); + } else { + return splitSize; + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java new file mode 100644 index 000000000000..dd914f1617bd --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -0,0 +1,778 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.Stream; +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.PartitionSpec; +import org.apache.iceberg.RequiresRemoteScanPlanning; +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.ImmutableSet; +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.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SnapshotUtil; +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, + SupportsPushDownRequiredColumns, + SupportsReportStatistics, + SupportsPushDownLimit { + + 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 List metaColumns = Lists.newArrayList(); + private final InMemoryMetricsReporter metricsReporter; + + private Schema schema; + private boolean caseSensitive; + private List filterExpressions = null; + private Predicate[] pushedPredicates = NO_PREDICATES; + private Integer limit = null; + + SparkScanBuilder( + SparkSession spark, + Table table, + String branch, + Schema schema, + CaseInsensitiveStringMap options) { + this.spark = spark; + this.table = table; + this.schema = 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); + } + + 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(schema.asStruct(), expr, caseSensitive); + expressions.add(expr); + pushableFilters.add(predicate); + } + + if (expr == null + || unpartitioned() + || !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); + } + } + + this.filterExpressions = expressions; + this.pushedPredicates = pushableFilters.toArray(new Predicate[0]); + + return postScanFilters.toArray(new Predicate[0]); + } + + private boolean unpartitioned() { + return table.specs().values().stream().noneMatch(PartitionSpec::isPartitioned); + } + + @Override + public Predicate[] pushedPredicates() { + return pushedPredicates; + } + + @Override + public boolean pushAggregation(Aggregation aggregation) { + if (!canPushDownAggregation(aggregation)) { + return false; + } + + AggregateEvaluator aggregateEvaluator; + List> expressions = + Lists.newArrayListWithExpectedSize(aggregation.aggregateExpressions().length); + + for (AggregateFunc aggregateFunc : aggregation.aggregateExpressions()) { + try { + Expression expr = SparkAggregates.convert(aggregateFunc); + if (expr != null) { + Expression bound = Binder.bind(schema.asStruct(), expr, caseSensitive); + expressions.add((BoundAggregate) bound); + } else { + LOG.info( + "Skipping aggregate pushdown: AggregateFunc {} can't be converted to iceberg expression", + aggregateFunc); + return false; + } + } catch (IllegalArgumentException e) { + LOG.info("Skipping aggregate pushdown: Bind failed for AggregateFunc {}", aggregateFunc, e); + return false; + } + } + + aggregateEvaluator = AggregateEvaluator.create(expressions); + + if (!metricsModeSupportsAggregatePushDown(aggregateEvaluator.aggregates())) { + return false; + } + + org.apache.iceberg.Scan scan = + buildIcebergBatchScan(true /* include Column Stats */, schemaWithMetadataColumns()); + + try (CloseableIterable fileScanTasks = scan.planFiles()) { + for (FileScanTask task : fileScanTasks) { + if (!task.deletes().isEmpty()) { + LOG.info("Skipping aggregate pushdown: detected row level deletes"); + return false; + } + + aggregateEvaluator.update(task.file()); + } + } catch (IOException e) { + LOG.info("Skipping aggregate pushdown: ", e); + return false; + } + + if (!aggregateEvaluator.allAggregatorsValid()) { + return false; + } + + StructType pushedAggregateSchema = + SparkSchemaUtil.convert(new Schema(aggregateEvaluator.resultType().fields())); + InternalRow[] pushedAggregateRows = new InternalRow[1]; + StructLike structLike = aggregateEvaluator.result(); + pushedAggregateRows[0] = + new StructInternalRow(aggregateEvaluator.resultType()).setStruct(structLike); + localScan = + new SparkLocalScan(table, pushedAggregateSchema, pushedAggregateRows, filterExpressions); + + return true; + } + + private boolean canPushDownAggregation(Aggregation aggregation) { + if (!(table instanceof BaseTable)) { + return false; + } + + if (!readConf.aggregatePushDownEnabled()) { + return false; + } + + // If group by expression is the same as the partition, the statistics information can still + // be used to calculate min/max/count, will enable aggregate push down in next phase. + // TODO: enable aggregate push down for partition col group by expression + if (aggregation.groupByExpressions().length > 0) { + LOG.info("Skipping aggregate pushdown: group by aggregation push down is not supported"); + return false; + } + + return true; + } + + private boolean metricsModeSupportsAggregatePushDown(List> aggregates) { + MetricsConfig config = MetricsConfig.forTable(table); + for (BoundAggregate aggregate : aggregates) { + String colName = aggregate.columnName(); + if (!colName.equals("*")) { + MetricsModes.MetricsMode mode = config.columnMode(colName); + if (mode instanceof MetricsModes.None) { + LOG.info("Skipping aggregate pushdown: No metrics for column {}", colName); + return false; + } else if (mode instanceof MetricsModes.Counts) { + if (aggregate.op() == Expression.Operation.MAX + || aggregate.op() == Expression.Operation.MIN) { + LOG.info( + "Skipping aggregate pushdown: Cannot produce min or max from count for column {}", + colName); + return false; + } + } else if (mode instanceof MetricsModes.Truncate) { + // lower_bounds and upper_bounds may be truncated, so disable push down + if (aggregate.type().typeId() == Type.TypeID.STRING) { + if (aggregate.op() == Expression.Operation.MAX + || aggregate.op() == Expression.Operation.MIN) { + LOG.info( + "Skipping aggregate pushdown: Cannot produce min or max from truncated values for column {}", + colName); + return false; + } + } + } + } + } + + return true; + } + + @Override + public void pruneColumns(StructType requestedSchema) { + StructType requestedProjection = + new StructType( + Stream.of(requestedSchema.fields()) + .filter(field -> MetadataColumns.nonMetadataColumn(field.name())) + .toArray(StructField[]::new)); + + // the projection should include all columns that will be returned, including those only used in + // filters + this.schema = + SparkSchemaUtil.prune(schema, requestedProjection, filterExpression(), caseSensitive); + + Stream.of(requestedSchema.fields()) + .map(StructField::name) + .filter(MetadataColumns::isMetadataColumn) + .distinct() + .forEach(metaColumns::add); + } + + private Schema schemaWithMetadataColumns() { + // metadata columns + List metadataFields = + metaColumns.stream() + .distinct() + .map(name -> MetadataColumns.metadataColumn(table, name)) + .collect(Collectors.toList()); + Schema metadataSchema = calculateMetadataSchema(metadataFields); + + // schema or rows returned by readers + return TypeUtil.join(schema, metadataSchema); + } + + private Schema calculateMetadataSchema(List metaColumnFields) { + Optional partitionField = + metaColumnFields.stream() + .filter(f -> MetadataColumns.PARTITION_COLUMN_ID == f.fieldId()) + .findFirst(); + + // only calculate potential column id collision if partition metadata column was requested + if (!partitionField.isPresent()) { + return new Schema(metaColumnFields); + } + + Set idsToReassign = + TypeUtil.indexById(partitionField.get().type().asStructType()).keySet(); + + // Calculate used ids by union metadata columns with all base table schemas + Set currentlyUsedIds = + metaColumnFields.stream().map(Types.NestedField::fieldId).collect(Collectors.toSet()); + Set allUsedIds = + table.schemas().values().stream() + .map(currSchema -> TypeUtil.indexById(currSchema.asStruct()).keySet()) + .reduce(currentlyUsedIds, Sets::union); + + // Reassign selected ids to deduplicate with used ids. + AtomicInteger nextId = new AtomicInteger(); + return new Schema( + metaColumnFields, + ImmutableSet.of(), + oldId -> { + if (!idsToReassign.contains(oldId)) { + return oldId; + } + int candidate = nextId.incrementAndGet(); + while (allUsedIds.contains(candidate)) { + candidate = nextId.incrementAndGet(); + } + return candidate; + }); + } + + @Override + public Scan build() { + if (localScan != null) { + return localScan; + } else { + return buildBatchScan(); + } + } + + private Scan buildBatchScan() { + Schema expectedSchema = schemaWithMetadataColumns(); + return new SparkBatchQueryScan( + spark, + table, + buildIcebergBatchScan(false /* not include Column Stats */, expectedSchema), + readConf, + expectedSchema, + filterExpressions, + 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); + } else { + return buildBatchScan(snapshotId, asOfTimestamp, branch, tag, withStats, expectedSchema); + } + } + + 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); + } + + return configureSplitPlanning(scan); + } + + private org.apache.iceberg.Scan buildIncrementalAppendScan( + long startSnapshotId, Long endSnapshotId, boolean withStats, Schema expectedSchema) { + IncrementalAppendScan scan = + table + .newIncrementalAppendScan() + .fromSnapshotExclusive(startSnapshotId) + .caseSensitive(caseSensitive) + .filter(filterExpression()) + .project(expectedSchema) + .metricsReporter(metricsReporter); + + if (withStats) { + scan = scan.includeColumnStats(); + } + + if (endSnapshotId != null) { + scan = scan.toSnapshot(endSnapshotId); + } + + 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 = schemaWithMetadataColumns(); + + 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) { + 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, + schemaWithMetadataColumns(), + 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 = schemaWithMetadataColumns(); + + BatchScan scan = + newBatchScan() + .useSnapshot(snapshotId) + .caseSensitive(caseSensitive) + .filter(filterExpression()) + .project(expectedSchema) + .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, + schemaWithMetadataColumns(), + filterExpressions, + metricsReporter::scanReport); + } + + Schema expectedSchema = schemaWithMetadataColumns(); + + BatchScan scan = + table + .newBatchScan() + .useSnapshot(snapshot.snapshotId()) + .ignoreResiduals() + .caseSensitive(caseSensitive) + .filter(filterExpression()) + .project(expectedSchema) + .metricsReporter(metricsReporter); + + scan = configureSplitPlanning(scan); + + return new SparkCopyOnWriteScan( + spark, + table, + scan, + snapshot, + readConf, + expectedSchema, + filterExpressions, + metricsReporter::scanReport); + } + + private > T configureSplitPlanning(T scan) { + T configuredScan = scan; + + Long splitSize = readConf.splitSizeOption(); + if (splitSize != null) { + configuredScan = configuredScan.option(TableProperties.SPLIT_SIZE, String.valueOf(splitSize)); + } + + Integer splitLookback = readConf.splitLookbackOption(); + if (splitLookback != null) { + configuredScan = + configuredScan.option(TableProperties.SPLIT_LOOKBACK, String.valueOf(splitLookback)); + } + + Long splitOpenFileCost = readConf.splitOpenFileCostOption(); + if (splitOpenFileCost != null) { + configuredScan = + configuredScan.option( + TableProperties.SPLIT_OPEN_FILE_COST, String.valueOf(splitOpenFileCost)); + } + + if (null != limit) { + configuredScan = configuredScan.minRowsRequested(limit.longValue()); + } + + return configuredScan; + } + + @Override + public Statistics estimateStatistics() { + return ((SupportsReportStatistics) build()).estimateStatistics(); + } + + @Override + public StructType readSchema() { + return build().readSchema(); + } + + private BatchScan newBatchScan() { + if (table instanceof RequiresRemoteScanPlanning) { + return table.newBatchScan(); + } else if (table instanceof BaseTable && readConf.distributedPlanningEnabled()) { + return new SparkDistributedDataScan(spark, table, readConf); + } else { + return table.newBatchScan(); + } + } + + @Override + public boolean pushLimit(int pushedLimit) { + this.limit = pushedLimit; + return true; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java new file mode 100644 index 000000000000..d2eb4e5a56e9 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java @@ -0,0 +1,98 @@ +/* + * 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 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.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.ScanTaskSetManager; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.util.TableScanUtil; +import org.apache.spark.sql.SparkSession; + +class SparkStagedScan extends SparkScan { + + private final String taskSetId; + private final long splitSize; + private final int splitLookback; + private final long openFileCost; + + private List> taskGroups = null; // lazy cache of tasks + + SparkStagedScan(SparkSession spark, Table table, Schema expectedSchema, SparkReadConf readConf) { + super(spark, table, readConf, expectedSchema, ImmutableList.of(), null); + this.taskSetId = readConf.scanTaskSetId(); + this.splitSize = readConf.splitSize(); + this.splitLookback = readConf.splitLookback(); + this.openFileCost = readConf.splitOpenFileCost(); + } + + @Override + protected List> taskGroups() { + if (taskGroups == null) { + ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + List tasks = taskSetManager.fetchTasks(table(), taskSetId); + ValidationException.check( + tasks != null, + "Task set manager has no tasks for table %s with task set ID %s", + table(), + taskSetId); + + this.taskGroups = TableScanUtil.planTaskGroups(tasks, splitSize, splitLookback, openFileCost); + } + return taskGroups; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + SparkStagedScan that = (SparkStagedScan) other; + return table().name().equals(that.table().name()) + && Objects.equals(taskSetId, that.taskSetId) + && readSchema().equals(that.readSchema()) + && splitSize == that.splitSize + && splitLookback == that.splitLookback + && openFileCost == that.openFileCost; + } + + @Override + public int hashCode() { + return Objects.hash( + table().name(), taskSetId, readSchema(), splitSize, splitSize, openFileCost); + } + + @Override + public String toString() { + return String.format( + "IcebergStagedScan(table=%s, type=%s, taskSetID=%s, caseSensitive=%s)", + table(), expectedSchema().asStruct(), taskSetId, caseSensitive()); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java new file mode 100644 index 000000000000..c5c86c3ebf28 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java @@ -0,0 +1,92 @@ +/* + * 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.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 { + + private final SparkSession spark; + private final Table table; + private final SparkReadConf readConf; + private final List metaColumns = Lists.newArrayList(); + + private Schema schema; + + SparkStagedScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { + this.spark = spark; + this.table = table; + this.readConf = new SparkReadConf(spark, table, options); + this.schema = table.schema(); + } + + @Override + public Scan build() { + return new SparkStagedScan(spark, table, schemaWithMetadataColumns(), 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); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java new file mode 100644 index 000000000000..1ee9e9b08074 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -0,0 +1,502 @@ +/* + * 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.CURRENT_SNAPSHOT_ID; +import static org.apache.iceberg.TableProperties.FORMAT_VERSION; + +import java.io.IOException; +import java.util.Map; +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.Partitioning; +import org.apache.iceberg.PositionDeletesTable; +import org.apache.iceberg.Schema; +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; +import org.apache.iceberg.expressions.ExpressionUtil; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Projections; +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.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.relocated.com.google.common.collect.Iterables; +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.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.DataType; +import org.apache.spark.sql.types.DataTypes; +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 { + + 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.BATCH_READ, + TableCapability.BATCH_WRITE, + TableCapability.MICRO_BATCH_READ, + TableCapability.STREAMING_WRITE, + TableCapability.OVERWRITE_BY_FILTER, + TableCapability.OVERWRITE_DYNAMIC); + private static final Set CAPABILITIES_WITH_ACCEPT_ANY_SCHEMA = + ImmutableSet.builder() + .addAll(CAPABILITIES) + .add(TableCapability.ACCEPT_ANY_SCHEMA) + .build(); + + private final Table icebergTable; + private final Long snapshotId; + private final boolean refreshEagerly; + 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 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 SparkSession sparkSession() { + if (lazySpark == null) { + this.lazySpark = SparkSession.active(); + } + + return lazySpark; + } + + public Table table() { + return icebergTable; + } + + @Override + public String name() { + return icebergTable.toString(); + } + + public Long snapshotId() { + return snapshotId; + } + + 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(); + } + + @Override + public Set capabilities() { + return capabilities; + } + + @Override + public MetadataColumn[] metadataColumns() { + DataType sparkPartitionType = SparkSchemaUtil.convert(Partitioning.partitionType(table())); + ImmutableList.Builder metadataColumns = ImmutableList.builder(); + metadataColumns.add( + SparkMetadataColumn.builder() + .name(MetadataColumns.SPEC_ID.name()) + .dataType(DataTypes.IntegerType) + .withNullability(true) + .build(), + SparkMetadataColumn.builder() + .name(MetadataColumns.PARTITION_COLUMN_NAME) + .dataType(sparkPartitionType) + .withNullability(true) + .build(), + SparkMetadataColumn.builder() + .name(MetadataColumns.FILE_PATH.name()) + .dataType(DataTypes.StringType) + .withNullability(false) + .build(), + SparkMetadataColumn.builder() + .name(MetadataColumns.ROW_POSITION.name()) + .dataType(DataTypes.LongType) + .withNullability(false) + .build(), + SparkMetadataColumn.builder() + .name(MetadataColumns.IS_DELETED.name()) + .dataType(DataTypes.BooleanType) + .withNullability(false) + .build()); + + if (TableUtil.supportsRowLineage(icebergTable)) { + metadataColumns.add( + SparkMetadataColumn.builder() + .name(MetadataColumns.ROW_ID.name()) + .dataType(DataTypes.LongType) + .withNullability(true) + .preserveOnReinsert(true) + .preserveOnUpdate(true) + .preserveOnDelete(false) + .build()); + + metadataColumns.add( + SparkMetadataColumn.builder() + .name(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name()) + .dataType(DataTypes.LongType) + .withNullability(true) + .preserveOnReinsert(false) + .preserveOnUpdate(false) + .preserveOnDelete(false) + .build()); + } + + return metadataColumns.build().toArray(SparkMetadataColumn[]::new); + } + + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + if (options.containsKey(SparkReadOptions.SCAN_TASK_SET_ID)) { + return new SparkStagedScanBuilder(sparkSession(), icebergTable, options); + } + + if (refreshEagerly) { + icebergTable.refresh(); + } + + CaseInsensitiveStringMap scanOptions = + branch != null ? options : addSnapshotId(options, snapshotId); + return new SparkScanBuilder( + sparkSession(), icebergTable, branch, snapshotSchema(), scanOptions); + } + + @Override + public WriteBuilder newWriteBuilder(LogicalWriteInfo info) { + Preconditions.checkArgument( + snapshotId == null, "Cannot write to table at a specific snapshot: %s", snapshotId); + + if (icebergTable instanceof PositionDeletesTable) { + return new SparkPositionDeletesRewriteBuilder(sparkSession(), icebergTable, branch, info); + } else { + return new SparkWriteBuilder(sparkSession(), icebergTable, branch, info); + } + } + + @Override + public RowLevelOperationBuilder newRowLevelOperationBuilder(RowLevelOperationInfo info) { + return new SparkRowLevelOperationBuilder(sparkSession(), icebergTable, branch, info); + } + + @Override + public boolean canDeleteWhere(Predicate[] predicates) { + Preconditions.checkArgument( + snapshotId == null, "Cannot delete from table at a specific snapshot: %s", snapshotId); + + Expression deleteExpr = Expressions.alwaysTrue(); + + for (Predicate predicate : predicates) { + Expression expr = SparkV2Filters.convert(predicate); + if (expr != null) { + deleteExpr = Expressions.and(deleteExpr, expr); + } else { + return false; + } + } + + return canDeleteUsingMetadata(deleteExpr); + } + + // a metadata delete is possible iff matching files can be deleted entirely + private boolean canDeleteUsingMetadata(Expression deleteExpr) { + boolean caseSensitive = SparkUtil.caseSensitive(sparkSession()); + + if (ExpressionUtil.selectsPartitions(deleteExpr, table(), caseSensitive)) { + return true; + } + + TableScan scan = + table() + .newScan() + .filter(deleteExpr) + .caseSensitive(caseSensitive) + .includeColumnStats() + .ignoreResiduals(); + + if (branch != null) { + scan = scan.useRef(branch); + } + + try (CloseableIterable tasks = scan.planFiles()) { + Map evaluators = Maps.newHashMap(); + StrictMetricsEvaluator metricsEvaluator = + new StrictMetricsEvaluator(SnapshotUtil.schemaFor(table(), branch), deleteExpr); + + return Iterables.all( + tasks, + task -> { + DataFile file = task.file(); + PartitionSpec spec = task.spec(); + Evaluator evaluator = + evaluators.computeIfAbsent( + spec.specId(), + specId -> + new Evaluator( + spec.partitionType(), Projections.strict(spec).project(deleteExpr))); + return evaluator.eval(file.partition()) || metricsEvaluator.eval(file); + }); + + } catch (IOException ioe) { + LOG.warn("Failed to close task iterable", ioe); + return false; + } + } + + @Override + public void deleteWhere(Predicate[] predicates) { + Expression deleteExpr = SparkV2Filters.convert(predicates); + + if (deleteExpr == Expressions.alwaysFalse()) { + LOG.info("Skipping the delete operation as the condition is always false"); + return; + } + + DeleteFiles deleteFiles = + icebergTable + .newDelete() + .set("spark.app.id", sparkSession().sparkContext().applicationId()) + .deleteFromRowFilter(deleteExpr); + + if (SparkTableUtil.wapEnabled(table())) { + branch = SparkTableUtil.determineWriteBranch(sparkSession(), branch); + } + + if (branch != null) { + deleteFiles.toBranch(branch); + } + + if (!CommitMetadata.commitProperties().isEmpty()) { + CommitMetadata.commitProperties().forEach(deleteFiles::set); + } + + deleteFiles.commit(); + } + + @Override + public String toString() { + return icebergTable.toString(); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + // use only name in order to correctly invalidate Spark cache + SparkTable that = (SparkTable) other; + return icebergTable.name().equals(that.icebergTable.name()); + } + + @Override + public int hashCode() { + // use only name in order to correctly invalidate Spark cache + return icebergTable.name().hashCode(); + } + + private static CaseInsensitiveStringMap addSnapshotId( + CaseInsensitiveStringMap options, Long snapshotId) { + if (snapshotId != null) { + String snapshotIdFromOptions = options.get(SparkReadOptions.SNAPSHOT_ID); + String value = snapshotId.toString(); + 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); + } + + return options; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkView.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkView.java new file mode 100644 index 000000000000..47e57295363d --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkView.java @@ -0,0 +1,151 @@ +/* + * 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.FORMAT_VERSION; + +import java.util.Map; +import java.util.Set; +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.spark.SparkSchemaUtil; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.view.BaseView; +import org.apache.iceberg.view.SQLViewRepresentation; +import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewOperations; +import org.apache.spark.sql.types.StructType; + +public class SparkView implements org.apache.spark.sql.connector.catalog.View { + + public static final String QUERY_COLUMN_NAMES = "spark.query-column-names"; + public static final Set RESERVED_PROPERTIES = + ImmutableSet.of("provider", "location", FORMAT_VERSION, QUERY_COLUMN_NAMES); + + private final View icebergView; + private final String catalogName; + private StructType lazySchema = null; + + public SparkView(String catalogName, View icebergView) { + this.catalogName = catalogName; + this.icebergView = icebergView; + } + + public View view() { + return icebergView; + } + + @Override + public String name() { + return icebergView.name(); + } + + @Override + public String query() { + SQLViewRepresentation sqlRepr = icebergView.sqlFor("spark"); + Preconditions.checkState(sqlRepr != null, "Cannot load SQL for view %s", name()); + return sqlRepr.sql(); + } + + @Override + public String currentCatalog() { + return icebergView.currentVersion().defaultCatalog() != null + ? icebergView.currentVersion().defaultCatalog() + : catalogName; + } + + @Override + public String[] currentNamespace() { + return icebergView.currentVersion().defaultNamespace().levels(); + } + + @Override + public StructType schema() { + if (null == lazySchema) { + this.lazySchema = SparkSchemaUtil.convert(icebergView.schema()); + } + + return lazySchema; + } + + @Override + public String[] queryColumnNames() { + return icebergView.properties().containsKey(QUERY_COLUMN_NAMES) + ? icebergView.properties().get(QUERY_COLUMN_NAMES).split(",") + : new String[0]; + } + + @Override + public String[] columnAliases() { + return icebergView.schema().columns().stream() + .map(Types.NestedField::name) + .toArray(String[]::new); + } + + @Override + public String[] columnComments() { + return icebergView.schema().columns().stream() + .map(Types.NestedField::doc) + .toArray(String[]::new); + } + + @Override + public Map properties() { + ImmutableMap.Builder propsBuilder = ImmutableMap.builder(); + + propsBuilder.put("provider", "iceberg"); + propsBuilder.put("location", icebergView.location()); + + if (icebergView instanceof BaseView) { + ViewOperations ops = ((BaseView) icebergView).operations(); + propsBuilder.put(FORMAT_VERSION, String.valueOf(ops.current().formatVersion())); + } + + icebergView.properties().entrySet().stream() + .filter(entry -> !RESERVED_PROPERTIES.contains(entry.getKey())) + .forEach(propsBuilder::put); + + return propsBuilder.build(); + } + + @Override + public String toString() { + return icebergView.toString(); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + // use only name in order to correctly invalidate Spark cache + SparkView that = (SparkView) other; + return icebergView.name().equals(that.icebergView.name()); + } + + @Override + public int hashCode() { + // use only name in order to correctly invalidate Spark cache + return icebergView.name().hashCode(); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java new file mode 100644 index 000000000000..c9a94090ef89 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -0,0 +1,874 @@ +/* + * 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.IsolationLevel.SERIALIZABLE; +import static org.apache.iceberg.IsolationLevel.SNAPSHOT; + +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; +import java.util.stream.Collectors; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.IsolationLevel; +import org.apache.iceberg.OverwriteFiles; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.CleanableFailure; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.ClusteredDataWriter; +import org.apache.iceberg.io.DataWriteResult; +import org.apache.iceberg.io.FanoutDataWriter; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileWriter; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.PartitioningWriter; +import org.apache.iceberg.io.RollingDataWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.CommitMetadata; +import org.apache.iceberg.spark.FileRewriteCoordinator; +import org.apache.iceberg.spark.SparkWriteConf; +import org.apache.iceberg.spark.SparkWriteRequirements; +import org.apache.iceberg.util.ContentFileUtil; +import org.apache.iceberg.util.DataFileSet; +import org.apache.iceberg.util.DeleteFileSet; +import org.apache.spark.TaskContext; +import org.apache.spark.TaskContext$; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.executor.OutputMetrics; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.JoinedRow; +import org.apache.spark.sql.connector.distributions.Distribution; +import org.apache.spark.sql.connector.expressions.SortOrder; +import org.apache.spark.sql.connector.write.BatchWrite; +import org.apache.spark.sql.connector.write.DataWriter; +import org.apache.spark.sql.connector.write.DataWriterFactory; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.PhysicalWriteInfo; +import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering; +import org.apache.spark.sql.connector.write.Write; +import org.apache.spark.sql.connector.write.WriterCommitMessage; +import org.apache.spark.sql.connector.write.streaming.StreamingDataWriterFactory; +import org.apache.spark.sql.connector.write.streaming.StreamingWrite; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +abstract class SparkWrite implements Write, RequiresDistributionAndOrdering { + private static final Logger LOG = LoggerFactory.getLogger(SparkWrite.class); + + private final JavaSparkContext sparkContext; + private final SparkWriteConf writeConf; + private final Table table; + private final String queryId; + private final FileFormat format; + private final String applicationId; + private final boolean wapEnabled; + private final String wapId; + private final int outputSpecId; + private final String branch; + private final long targetFileSize; + private final Schema writeSchema; + private final StructType dsSchema; + private final Map extraSnapshotMetadata; + private final boolean useFanoutWriter; + private final SparkWriteRequirements writeRequirements; + private final Map writeProperties; + + private boolean cleanupOnAbort = false; + + SparkWrite( + SparkSession spark, + Table table, + SparkWriteConf writeConf, + LogicalWriteInfo writeInfo, + String applicationId, + Schema writeSchema, + StructType dsSchema, + SparkWriteRequirements writeRequirements) { + this.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + this.table = table; + this.writeConf = writeConf; + this.queryId = writeInfo.queryId(); + this.format = writeConf.dataFileFormat(); + this.applicationId = applicationId; + this.wapEnabled = writeConf.wapEnabled(); + this.wapId = writeConf.wapId(); + this.branch = writeConf.branch(); + this.targetFileSize = writeConf.targetDataFileSize(); + this.writeSchema = writeSchema; + this.dsSchema = dsSchema; + this.extraSnapshotMetadata = writeConf.extraSnapshotMetadata(); + this.useFanoutWriter = writeConf.useFanoutWriter(writeRequirements); + this.writeRequirements = writeRequirements; + this.outputSpecId = writeConf.outputSpecId(); + this.writeProperties = writeConf.writeProperties(); + } + + @Override + public Distribution requiredDistribution() { + Distribution distribution = writeRequirements.distribution(); + LOG.debug("Requesting {} as write distribution for table {}", distribution, table.name()); + return distribution; + } + + @Override + public boolean distributionStrictlyRequired() { + return false; + } + + @Override + public SortOrder[] requiredOrdering() { + SortOrder[] ordering = writeRequirements.ordering(); + LOG.debug("Requesting {} as write ordering for table {}", ordering, table.name()); + return ordering; + } + + @Override + public long advisoryPartitionSizeInBytes() { + long size = writeRequirements.advisoryPartitionSize(); + LOG.debug("Requesting {} bytes advisory partition size for table {}", size, table.name()); + return size; + } + + BatchWrite asBatchAppend() { + return new BatchAppend(); + } + + BatchWrite asDynamicOverwrite() { + return new DynamicOverwrite(); + } + + BatchWrite asOverwriteByFilter(Expression overwriteExpr) { + return new OverwriteByFilter(overwriteExpr); + } + + BatchWrite asCopyOnWriteOperation(SparkCopyOnWriteScan scan, IsolationLevel isolationLevel) { + return new CopyOnWriteOperation(scan, isolationLevel); + } + + BatchWrite asRewrite(String fileSetID) { + return new RewriteFiles(fileSetID); + } + + StreamingWrite asStreamingAppend() { + return new StreamingAppend(); + } + + StreamingWrite asStreamingOverwrite() { + return new StreamingOverwrite(); + } + + // the writer factory works for both batch and streaming + private WriterFactory createWriterFactory() { + // broadcast the table metadata as the writer factory will be sent to executors + Broadcast
    tableBroadcast = + sparkContext.broadcast(SerializableTableWithSize.copyOf(table)); + return new WriterFactory( + tableBroadcast, + queryId, + format, + outputSpecId, + targetFileSize, + writeSchema, + dsSchema, + useFanoutWriter, + writeProperties); + } + + private void commitOperation(SnapshotUpdate operation, String description) { + LOG.info("Committing {} to table {}", description, table); + if (applicationId != null) { + operation.set("spark.app.id", applicationId); + } + + if (!extraSnapshotMetadata.isEmpty()) { + extraSnapshotMetadata.forEach(operation::set); + } + + if (!CommitMetadata.commitProperties().isEmpty()) { + CommitMetadata.commitProperties().forEach(operation::set); + } + + if (wapEnabled && wapId != null) { + // write-audit-publish is enabled for this table and job + // stage the changes without changing the current snapshot + operation.set(SnapshotSummary.STAGED_WAP_ID_PROP, wapId); + operation.stageOnly(); + } + + if (branch != null) { + operation.toBranch(branch); + } + + try { + long start = System.currentTimeMillis(); + operation.commit(); // abort is automatically called if this fails + long duration = System.currentTimeMillis() - start; + LOG.info("Committed in {} ms", duration); + } catch (Exception e) { + cleanupOnAbort = e instanceof CleanableFailure; + throw e; + } + } + + private void abort(WriterCommitMessage[] messages) { + if (cleanupOnAbort) { + SparkCleanupUtil.deleteFiles("job abort", table.io(), Lists.newArrayList(files(messages))); + } else { + LOG.warn("Skipping cleanup of written files"); + } + } + + private DataFileSet files(WriterCommitMessage[] messages) { + DataFileSet files = DataFileSet.create(); + + for (WriterCommitMessage message : messages) { + if (message != null) { + TaskCommit taskCommit = (TaskCommit) message; + files.addAll(Arrays.asList(taskCommit.files())); + } + } + + return files; + } + + @Override + public String toString() { + return String.format("IcebergWrite(table=%s, format=%s)", table, format); + } + + private abstract class BaseBatchWrite implements BatchWrite { + @Override + public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { + return createWriterFactory(); + } + + @Override + public boolean useCommitCoordinator() { + return false; + } + + @Override + public void abort(WriterCommitMessage[] messages) { + SparkWrite.this.abort(messages); + } + + @Override + public String toString() { + return String.format("IcebergBatchWrite(table=%s, format=%s)", table, format); + } + } + + private class BatchAppend extends BaseBatchWrite { + @Override + public void commit(WriterCommitMessage[] messages) { + AppendFiles append = table.newAppend(); + + int numFiles = 0; + for (DataFile file : files(messages)) { + numFiles += 1; + append.appendFile(file); + } + + commitOperation( + append, String.format(Locale.ROOT, "append with %d new data files", numFiles)); + } + } + + private class DynamicOverwrite extends BaseBatchWrite { + @Override + public void commit(WriterCommitMessage[] messages) { + DataFileSet files = files(messages); + + if (files.isEmpty()) { + LOG.info("Dynamic overwrite is empty, skipping commit"); + return; + } + + ReplacePartitions dynamicOverwrite = table.newReplacePartitions(); + IsolationLevel isolationLevel = writeConf.isolationLevel(); + Long validateFromSnapshotId = writeConf.validateFromSnapshotId(); + + if (isolationLevel != null && validateFromSnapshotId != null) { + dynamicOverwrite.validateFromSnapshot(validateFromSnapshotId); + } + + if (isolationLevel == SERIALIZABLE) { + dynamicOverwrite.validateNoConflictingData(); + dynamicOverwrite.validateNoConflictingDeletes(); + } else if (isolationLevel == SNAPSHOT) { + dynamicOverwrite.validateNoConflictingDeletes(); + } + + int numFiles = 0; + for (DataFile file : files) { + numFiles += 1; + dynamicOverwrite.addFile(file); + } + + commitOperation( + dynamicOverwrite, + String.format( + Locale.ROOT, "dynamic partition overwrite with %d new data files", numFiles)); + } + } + + private class OverwriteByFilter extends BaseBatchWrite { + private final Expression overwriteExpr; + + private OverwriteByFilter(Expression overwriteExpr) { + this.overwriteExpr = overwriteExpr; + } + + @Override + public void commit(WriterCommitMessage[] messages) { + OverwriteFiles overwriteFiles = table.newOverwrite(); + overwriteFiles.overwriteByRowFilter(overwriteExpr); + + int numFiles = 0; + for (DataFile file : files(messages)) { + numFiles += 1; + overwriteFiles.addFile(file); + } + + IsolationLevel isolationLevel = writeConf.isolationLevel(); + Long validateFromSnapshotId = writeConf.validateFromSnapshotId(); + + if (isolationLevel != null && validateFromSnapshotId != null) { + overwriteFiles.validateFromSnapshot(validateFromSnapshotId); + } + + if (isolationLevel == SERIALIZABLE) { + overwriteFiles.validateNoConflictingDeletes(); + overwriteFiles.validateNoConflictingData(); + } else if (isolationLevel == SNAPSHOT) { + overwriteFiles.validateNoConflictingDeletes(); + } + + String commitMsg = + String.format( + Locale.ROOT, + "overwrite by filter %s with %d new data files", + overwriteExpr, + numFiles); + commitOperation(overwriteFiles, commitMsg); + } + } + + private class CopyOnWriteOperation extends BaseBatchWrite { + private final SparkCopyOnWriteScan scan; + private final IsolationLevel isolationLevel; + + private CopyOnWriteOperation(SparkCopyOnWriteScan scan, IsolationLevel isolationLevel) { + this.scan = scan; + this.isolationLevel = isolationLevel; + } + + private DataFileSet overwrittenFiles() { + if (scan == null) { + return DataFileSet.create(); + } else { + return scan.tasks().stream() + .map(FileScanTask::file) + .collect(Collectors.toCollection(DataFileSet::create)); + } + } + + private DeleteFileSet danglingDVs() { + if (scan == null) { + return DeleteFileSet.create(); + } else { + return scan.tasks().stream() + .flatMap(task -> task.deletes().stream().filter(ContentFileUtil::isDV)) + .collect(Collectors.toCollection(DeleteFileSet::create)); + } + } + + 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) { + OverwriteFiles overwriteFiles = table.newOverwrite(); + + DataFileSet overwrittenFiles = overwrittenFiles(); + int numOverwrittenFiles = overwrittenFiles.size(); + DeleteFileSet danglingDVs = danglingDVs(); + overwriteFiles.deleteFiles(overwrittenFiles, danglingDVs); + + int numAddedFiles = 0; + for (DataFile file : files(messages)) { + numAddedFiles += 1; + overwriteFiles.addFile(file); + } + + // the scan may be null if the optimizer replaces it with an empty relation (e.g. false cond) + // no validation is needed in this case as the command does not depend on the table state + if (scan != null) { + switch (isolationLevel) { + case SERIALIZABLE: + commitWithSerializableIsolation(overwriteFiles, numOverwrittenFiles, numAddedFiles); + break; + case SNAPSHOT: + commitWithSnapshotIsolation(overwriteFiles, numOverwrittenFiles, numAddedFiles); + break; + default: + throw new IllegalArgumentException("Unsupported isolation level: " + isolationLevel); + } + + } else { + commitOperation( + overwriteFiles, + String.format( + Locale.ROOT, "overwrite with %d new data files (no validation)", numAddedFiles)); + } + } + + private void commitWithSerializableIsolation( + OverwriteFiles overwriteFiles, int numOverwrittenFiles, int numAddedFiles) { + Long scanSnapshotId = scan.snapshotId(); + if (scanSnapshotId != null) { + overwriteFiles.validateFromSnapshot(scanSnapshotId); + } + + Expression conflictDetectionFilter = conflictDetectionFilter(); + overwriteFiles.conflictDetectionFilter(conflictDetectionFilter); + overwriteFiles.validateNoConflictingData(); + overwriteFiles.validateNoConflictingDeletes(); + + String commitMsg = + String.format( + Locale.ROOT, + "overwrite of %d data files with %d new data files, scanSnapshotId: %d, conflictDetectionFilter: %s", + numOverwrittenFiles, + numAddedFiles, + scanSnapshotId, + conflictDetectionFilter); + commitOperation(overwriteFiles, commitMsg); + } + + private void commitWithSnapshotIsolation( + OverwriteFiles overwriteFiles, int numOverwrittenFiles, int numAddedFiles) { + Long scanSnapshotId = scan.snapshotId(); + if (scanSnapshotId != null) { + overwriteFiles.validateFromSnapshot(scanSnapshotId); + } + + Expression conflictDetectionFilter = conflictDetectionFilter(); + overwriteFiles.conflictDetectionFilter(conflictDetectionFilter); + overwriteFiles.validateNoConflictingDeletes(); + + String commitMsg = + String.format( + Locale.ROOT, + "overwrite of %d data files with %d new data files", + numOverwrittenFiles, + numAddedFiles); + commitOperation(overwriteFiles, commitMsg); + } + } + + private class RewriteFiles extends BaseBatchWrite { + private final String fileSetID; + + private RewriteFiles(String fileSetID) { + this.fileSetID = fileSetID; + } + + @Override + public void commit(WriterCommitMessage[] messages) { + FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); + coordinator.stageRewrite(table, fileSetID, files(messages)); + } + } + + private abstract class BaseStreamingWrite implements StreamingWrite { + private static final String QUERY_ID_PROPERTY = "spark.sql.streaming.queryId"; + private static final String EPOCH_ID_PROPERTY = "spark.sql.streaming.epochId"; + + protected abstract String mode(); + + @Override + public StreamingDataWriterFactory createStreamingWriterFactory(PhysicalWriteInfo info) { + return createWriterFactory(); + } + + @Override + public boolean useCommitCoordinator() { + return false; + } + + @Override + public final void commit(long epochId, WriterCommitMessage[] messages) { + LOG.info("Committing epoch {} for query {} in {} mode", epochId, queryId, mode()); + + table.refresh(); + + Long lastCommittedEpochId = findLastCommittedEpochId(); + if (lastCommittedEpochId != null && epochId <= lastCommittedEpochId) { + LOG.info("Skipping epoch {} for query {} as it was already committed", epochId, queryId); + return; + } + + doCommit(epochId, messages); + } + + protected abstract void doCommit(long epochId, WriterCommitMessage[] messages); + + protected void commit(SnapshotUpdate snapshotUpdate, long epochId, String description) { + snapshotUpdate.set(QUERY_ID_PROPERTY, queryId); + snapshotUpdate.set(EPOCH_ID_PROPERTY, Long.toString(epochId)); + commitOperation(snapshotUpdate, description); + } + + private Long findLastCommittedEpochId() { + Snapshot snapshot = table.currentSnapshot(); + Long lastCommittedEpochId = null; + while (snapshot != null) { + Map summary = snapshot.summary(); + String snapshotQueryId = summary.get(QUERY_ID_PROPERTY); + if (queryId.equals(snapshotQueryId)) { + lastCommittedEpochId = Long.valueOf(summary.get(EPOCH_ID_PROPERTY)); + break; + } + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + return lastCommittedEpochId; + } + + @Override + public void abort(long epochId, WriterCommitMessage[] messages) { + SparkWrite.this.abort(messages); + } + + @Override + public String toString() { + return String.format("IcebergStreamingWrite(table=%s, format=%s)", table, format); + } + } + + private class StreamingAppend extends BaseStreamingWrite { + @Override + protected String mode() { + return "append"; + } + + @Override + protected void doCommit(long epochId, WriterCommitMessage[] messages) { + AppendFiles append = table.newFastAppend(); + int numFiles = 0; + for (DataFile file : files(messages)) { + append.appendFile(file); + numFiles++; + } + commit( + append, + epochId, + String.format(Locale.ROOT, "streaming append with %d new data files", numFiles)); + } + } + + private class StreamingOverwrite extends BaseStreamingWrite { + @Override + protected String mode() { + return "complete"; + } + + @Override + public void doCommit(long epochId, WriterCommitMessage[] messages) { + OverwriteFiles overwriteFiles = table.newOverwrite(); + overwriteFiles.overwriteByRowFilter(Expressions.alwaysTrue()); + int numFiles = 0; + for (DataFile file : files(messages)) { + overwriteFiles.addFile(file); + numFiles++; + } + commit( + overwriteFiles, + epochId, + String.format( + Locale.ROOT, "streaming complete overwrite with %d new data files", numFiles)); + } + } + + public static class TaskCommit implements WriterCommitMessage { + private final DataFile[] taskFiles; + + TaskCommit(DataFile[] taskFiles) { + this.taskFiles = taskFiles; + } + + // Reports bytesWritten and recordsWritten to the Spark output metrics. + // Can only be called in executor. + void reportOutputMetrics() { + long bytesWritten = 0L; + long recordsWritten = 0L; + for (DataFile dataFile : taskFiles) { + bytesWritten += dataFile.fileSizeInBytes(); + recordsWritten += dataFile.recordCount(); + } + + TaskContext taskContext = TaskContext$.MODULE$.get(); + if (taskContext != null) { + OutputMetrics outputMetrics = taskContext.taskMetrics().outputMetrics(); + outputMetrics.setBytesWritten(bytesWritten); + outputMetrics.setRecordsWritten(recordsWritten); + } + } + + DataFile[] files() { + return taskFiles; + } + } + + private static class WriterFactory implements DataWriterFactory, StreamingDataWriterFactory { + private final Broadcast
    tableBroadcast; + private final FileFormat format; + private final int outputSpecId; + private final long targetFileSize; + private final Schema writeSchema; + private final StructType dsSchema; + private final boolean useFanoutWriter; + private final String queryId; + private final Map writeProperties; + + protected WriterFactory( + Broadcast
    tableBroadcast, + String queryId, + FileFormat format, + int outputSpecId, + long targetFileSize, + Schema writeSchema, + StructType dsSchema, + boolean useFanoutWriter, + Map writeProperties) { + this.tableBroadcast = tableBroadcast; + this.format = format; + this.outputSpecId = outputSpecId; + this.targetFileSize = targetFileSize; + this.writeSchema = writeSchema; + this.dsSchema = dsSchema; + this.useFanoutWriter = useFanoutWriter; + this.queryId = queryId; + this.writeProperties = writeProperties; + } + + @Override + public DataWriter createWriter(int partitionId, long taskId) { + return createWriter(partitionId, taskId, 0); + } + + @Override + public DataWriter createWriter(int partitionId, long taskId, long epochId) { + Table table = tableBroadcast.value(); + PartitionSpec spec = table.specs().get(outputSpecId); + FileIO io = table.io(); + String operationId = queryId + "-" + epochId; + OutputFileFactory fileFactory = + OutputFileFactory.builderFor(table, partitionId, taskId) + .format(format) + .operationId(operationId) + .build(); + SparkFileWriterFactory writerFactory = + SparkFileWriterFactory.builderFor(table) + .dataFileFormat(format) + .dataSchema(writeSchema) + .dataSparkType(dsSchema) + .writeProperties(writeProperties) + .build(); + + Function rowLineageExtractor = new ExtractRowLineage(writeSchema); + + if (spec.isUnpartitioned()) { + return new UnpartitionedDataWriter( + writerFactory, fileFactory, io, spec, targetFileSize, rowLineageExtractor); + + } else { + return new PartitionedDataWriter( + writerFactory, + fileFactory, + io, + spec, + writeSchema, + dsSchema, + targetFileSize, + useFanoutWriter, + rowLineageExtractor); + } + } + } + + private static class UnpartitionedDataWriter extends DataWriterWithLineage { + private final FileWriter delegate; + private final FileIO io; + + private UnpartitionedDataWriter( + SparkFileWriterFactory writerFactory, + OutputFileFactory fileFactory, + FileIO io, + PartitionSpec spec, + long targetFileSize, + Function rowLineageExtractor) { + super(rowLineageExtractor); + this.delegate = + new RollingDataWriter<>(writerFactory, fileFactory, io, targetFileSize, spec, null); + this.io = io; + } + + @Override + public void write(InternalRow record) throws IOException { + write(null, record); + } + + @Override + public void write(InternalRow meta, InternalRow record) throws IOException { + delegate.write(decorateWithRowLineage(meta, record)); + } + + @Override + public WriterCommitMessage commit() throws IOException { + close(); + + DataWriteResult result = delegate.result(); + TaskCommit taskCommit = new TaskCommit(result.dataFiles().toArray(new DataFile[0])); + taskCommit.reportOutputMetrics(); + return taskCommit; + } + + @Override + public void abort() throws IOException { + close(); + + DataWriteResult result = delegate.result(); + SparkCleanupUtil.deleteTaskFiles(io, result.dataFiles()); + } + + @Override + public void close() throws IOException { + delegate.close(); + } + } + + private static class PartitionedDataWriter extends DataWriterWithLineage { + private final PartitioningWriter delegate; + private final FileIO io; + private final PartitionSpec spec; + private final PartitionKey partitionKey; + private final InternalRowWrapper internalRowWrapper; + + private PartitionedDataWriter( + SparkFileWriterFactory writerFactory, + OutputFileFactory fileFactory, + FileIO io, + PartitionSpec spec, + Schema dataSchema, + StructType dataSparkType, + long targetFileSize, + boolean fanoutEnabled, + Function rowLineageExtractor) { + super(rowLineageExtractor); + if (fanoutEnabled) { + this.delegate = new FanoutDataWriter<>(writerFactory, fileFactory, io, targetFileSize); + } else { + this.delegate = new ClusteredDataWriter<>(writerFactory, fileFactory, io, targetFileSize); + } + this.io = io; + this.spec = spec; + this.partitionKey = new PartitionKey(spec, dataSchema); + this.internalRowWrapper = new InternalRowWrapper(dataSparkType, dataSchema.asStruct()); + } + + @Override + public void write(InternalRow row) throws IOException { + write(null, row); + } + + @Override + public void write(InternalRow meta, InternalRow record) throws IOException { + partitionKey.partition(internalRowWrapper.wrap(record)); + delegate.write(decorateWithRowLineage(meta, record), spec, partitionKey); + } + + @Override + public WriterCommitMessage commit() throws IOException { + close(); + + DataWriteResult result = delegate.result(); + TaskCommit taskCommit = new TaskCommit(result.dataFiles().toArray(new DataFile[0])); + taskCommit.reportOutputMetrics(); + return taskCommit; + } + + @Override + public void abort() throws IOException { + close(); + + DataWriteResult result = delegate.result(); + SparkCleanupUtil.deleteTaskFiles(io, result.dataFiles()); + } + + @Override + public void close() throws IOException { + delegate.close(); + } + } + + private abstract static class DataWriterWithLineage implements DataWriter { + private final Function rowLineageExtractor; + + DataWriterWithLineage(Function rowLineageExtractor) { + Preconditions.checkArgument( + rowLineageExtractor != null, "Row lineage extractor cannot be null"); + this.rowLineageExtractor = rowLineageExtractor; + } + + protected InternalRow decorateWithRowLineage(InternalRow meta, InternalRow record) { + InternalRow rowLineage = rowLineageExtractor.apply(meta); + return rowLineage == null ? record : new JoinedRow(record, rowLineage); + } + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java new file mode 100644 index 000000000000..89af7740d988 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java @@ -0,0 +1,238 @@ +/* + * 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.IsolationLevel; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableUtil; +import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +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.SparkUtil; +import org.apache.iceberg.spark.SparkWriteConf; +import org.apache.iceberg.spark.SparkWriteRequirements; +import org.apache.iceberg.types.TypeUtil; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.read.Scan; +import org.apache.spark.sql.connector.write.BatchWrite; +import org.apache.spark.sql.connector.write.LogicalWriteInfo; +import org.apache.spark.sql.connector.write.RowLevelOperation.Command; +import org.apache.spark.sql.connector.write.SupportsDynamicOverwrite; +import org.apache.spark.sql.connector.write.SupportsOverwrite; +import org.apache.spark.sql.connector.write.Write; +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.StructType; + +class SparkWriteBuilder implements WriteBuilder, SupportsDynamicOverwrite, SupportsOverwrite { + private final SparkSession spark; + private final Table table; + private final SparkWriteConf writeConf; + private final LogicalWriteInfo writeInfo; + private final StructType dsSchema; + private final String overwriteMode; + private final String rewrittenFileSetId; + 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; + + 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.rewrittenFileSetId = writeConf.rewrittenFileSetId(); + } + + 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"); + Preconditions.checkState( + rewrittenFileSetId == null, "Cannot overwrite individual files and rewrite"); + + this.overwriteFiles = true; + this.copyOnWriteScan = (SparkCopyOnWriteScan) scan; + this.copyOnWriteCommand = command; + this.copyOnWriteIsolationLevel = 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"); + Preconditions.checkState( + rewrittenFileSetId == null, "Cannot overwrite dynamically and rewrite"); + + this.overwriteDynamic = true; + return this; + } + + @Override + public WriteBuilder overwrite(Filter[] filters) { + Preconditions.checkState( + !overwriteFiles, "Cannot overwrite individual files and using filters"); + Preconditions.checkState(rewrittenFileSetId == null, "Cannot overwrite and rewrite"); + + this.overwriteExpr = SparkFilters.convert(filters); + if (overwriteExpr == Expressions.alwaysTrue() && "dynamic".equals(overwriteMode)) { + // use the write option to override truncating the table. use dynamic overwrite instead. + this.overwriteDynamic = true; + } else { + Preconditions.checkState( + !overwriteDynamic, "Cannot overwrite dynamically and by filter: %s", overwriteExpr); + this.overwriteByFilter = true; + } + 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 || writeConf.rewrittenFileSetId() != null); + 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$); + } + + Schema writeSchema = + validateOrMergeWriteSchema(table, sparkWriteSchema, writeConf, writeRequiresRowLineage); + + SparkUtil.validatePartitionTransforms(table.spec()); + + // Get application id + String appId = spark.sparkContext().applicationId(); + + return new SparkWrite( + spark, + table, + writeConf, + writeInfo, + appId, + writeSchema, + sparkWriteSchema, + writeRequirements()) { + + @Override + public BatchWrite toBatch() { + if (rewrittenFileSetId != null) { + return asRewrite(rewrittenFileSetId); + } else if (overwriteByFilter) { + return asOverwriteByFilter(overwriteExpr); + } else if (overwriteDynamic) { + return asDynamicOverwrite(); + } else if (overwriteFiles) { + return asCopyOnWriteOperation(copyOnWriteScan, copyOnWriteIsolationLevel); + } else { + return asBatchAppend(); + } + } + + @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); + Preconditions.checkState( + rewrittenFileSetId == null, "Unsupported streaming operation: rewrite"); + + if (overwriteByFilter) { + return asStreamingOverwrite(); + } else { + return asStreamingAppend(); + } + } + }; + } + + private SparkWriteRequirements writeRequirements() { + if (overwriteFiles) { + return writeConf.copyOnWriteRequirements(copyOnWriteCommand); + } 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())); + } + + // reconvert the dsSchema without assignment to use the ids assigned by UpdateSchema + writeSchema = SparkSchemaUtil.convert(mergedSchema, dsSchema, caseSensitive); + + TypeUtil.validateWriteSchema( + mergedSchema, writeSchema, writeConf.checkNullability(), writeConf.checkOrdering()); + + // 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()); + } + + return writeSchema; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StagedSparkTable.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StagedSparkTable.java new file mode 100644 index 000000000000..b92c02d2b536 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StagedSparkTable.java @@ -0,0 +1,41 @@ +/* + * 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.Transaction; +import org.apache.spark.sql.connector.catalog.StagedTable; + +public class StagedSparkTable extends SparkTable implements StagedTable { + private final Transaction transaction; + + public StagedSparkTable(Transaction transaction) { + super(transaction.table(), false); + this.transaction = transaction; + } + + @Override + public void commitStagedChanges() { + transaction.commitTransaction(); + } + + @Override + public void abortStagedChanges() { + // TODO: clean up + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java new file mode 100644 index 000000000000..ccf523cb4b05 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java @@ -0,0 +1,52 @@ +/* + * 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.Map; +import java.util.OptionalLong; +import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.read.Statistics; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; + +class Stats implements Statistics { + private final OptionalLong sizeInBytes; + private final OptionalLong numRows; + private final Map colstats; + + Stats(long sizeInBytes, long numRows, Map colstats) { + this.sizeInBytes = OptionalLong.of(sizeInBytes); + this.numRows = OptionalLong.of(numRows); + this.colstats = colstats; + } + + @Override + public OptionalLong sizeInBytes() { + return sizeInBytes; + } + + @Override + public OptionalLong numRows() { + return numRows; + } + + @Override + public Map columnStats() { + return colstats; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java new file mode 100644 index 000000000000..8869fedc84ee --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java @@ -0,0 +1,161 @@ +/* + * 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 com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.io.InputStream; +import java.io.StringWriter; +import java.io.UncheckedIOException; +import java.util.Locale; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.JsonUtil; +import org.apache.spark.sql.connector.read.streaming.Offset; + +class StreamingOffset extends Offset { + static final StreamingOffset START_OFFSET = new StreamingOffset(-1L, -1, false); + + private static final int CURR_VERSION = 1; + private static final String VERSION = "version"; + private static final String SNAPSHOT_ID = "snapshot_id"; + private static final String POSITION = "position"; + private static final String SCAN_ALL_FILES = "scan_all_files"; + + private final long snapshotId; + private final long position; + private final boolean scanAllFiles; + + /** + * An implementation of Spark Structured Streaming Offset, to track the current processed files of + * Iceberg table. + * + * @param snapshotId The current processed snapshot id. + * @param position The position of last scanned file in snapshot. + * @param scanAllFiles whether to scan all files in a snapshot; for example, to read all data when + * starting a stream. + */ + StreamingOffset(long snapshotId, long position, boolean scanAllFiles) { + this.snapshotId = snapshotId; + this.position = position; + this.scanAllFiles = scanAllFiles; + } + + static StreamingOffset fromJson(String json) { + Preconditions.checkNotNull(json, "Cannot parse StreamingOffset JSON: null"); + + try { + JsonNode node = JsonUtil.mapper().readValue(json, JsonNode.class); + return fromJsonNode(node); + } catch (IOException e) { + throw new UncheckedIOException( + String.format("Failed to parse StreamingOffset from JSON string %s", json), e); + } + } + + static StreamingOffset fromJson(InputStream inputStream) { + Preconditions.checkNotNull(inputStream, "Cannot parse StreamingOffset from inputStream: null"); + + JsonNode node; + try { + node = JsonUtil.mapper().readValue(inputStream, JsonNode.class); + } catch (IOException e) { + throw new UncheckedIOException("Failed to read StreamingOffset from json", e); + } + + return fromJsonNode(node); + } + + @Override + public String json() { + StringWriter writer = new StringWriter(); + try { + JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + generator.writeStartObject(); + generator.writeNumberField(VERSION, CURR_VERSION); + generator.writeNumberField(SNAPSHOT_ID, snapshotId); + generator.writeNumberField(POSITION, position); + generator.writeBooleanField(SCAN_ALL_FILES, scanAllFiles); + generator.writeEndObject(); + generator.flush(); + + } catch (IOException e) { + throw new UncheckedIOException("Failed to write StreamingOffset to json", e); + } + + return writer.toString(); + } + + long snapshotId() { + return snapshotId; + } + + long position() { + return position; + } + + boolean shouldScanAllFiles() { + return scanAllFiles; + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof StreamingOffset) { + StreamingOffset offset = (StreamingOffset) obj; + return offset.snapshotId == snapshotId + && offset.position == position + && offset.scanAllFiles == scanAllFiles; + } else { + return false; + } + } + + @Override + public int hashCode() { + return Objects.hashCode(snapshotId, position, scanAllFiles); + } + + @Override + public String toString() { + return String.format( + Locale.ROOT, + "Streaming Offset[%d: position (%d) scan_all_files (%b)]", + snapshotId, + position, + scanAllFiles); + } + + private static StreamingOffset fromJsonNode(JsonNode node) { + // The version of StreamingOffset. The offset was created with a version number + // used to validate when deserializing from json string. + int version = JsonUtil.getInt(VERSION, node); + Preconditions.checkArgument( + version == CURR_VERSION, + "This version of Iceberg source only supports version %s. Version %s is not supported.", + CURR_VERSION, + version); + + long snapshotId = JsonUtil.getLong(SNAPSHOT_ID, node); + int position = JsonUtil.getInt(POSITION, node); + boolean shouldScanAllFiles = JsonUtil.getBool(SCAN_ALL_FILES, node); + + return new StreamingOffset(snapshotId, position, shouldScanAllFiles); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java new file mode 100644 index 000000000000..2d3c917e58f9 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java @@ -0,0 +1,416 @@ +/* + * 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.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.OffsetDateTime; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.BiConsumer; +import java.util.function.Function; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.variants.Variant; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.GenericArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.BinaryType; +import org.apache.spark.sql.types.BooleanType; +import org.apache.spark.sql.types.ByteType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DateType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.DoubleType; +import org.apache.spark.sql.types.FloatType; +import org.apache.spark.sql.types.IntegerType; +import org.apache.spark.sql.types.LongType; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.ShortType; +import org.apache.spark.sql.types.StringType; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.types.TimestampType; +import org.apache.spark.sql.types.VariantType; +import org.apache.spark.unsafe.types.CalendarInterval; +import org.apache.spark.unsafe.types.UTF8String; +import org.apache.spark.unsafe.types.VariantVal; + +class StructInternalRow extends InternalRow { + private final Types.StructType type; + private StructLike struct; + + StructInternalRow(Types.StructType type) { + this.type = type; + } + + private StructInternalRow(Types.StructType type, StructLike struct) { + this.type = type; + this.struct = struct; + } + + public StructInternalRow setStruct(StructLike newStruct) { + this.struct = newStruct; + return this; + } + + @Override + public int numFields() { + return struct.size(); + } + + @Override + public void setNullAt(int i) { + throw new UnsupportedOperationException("StructInternalRow is read-only"); + } + + @Override + public void update(int i, Object value) { + throw new UnsupportedOperationException("StructInternalRow is read-only"); + } + + @Override + public InternalRow copy() { + return this; + } + + @Override + public boolean isNullAt(int ordinal) { + return struct.get(ordinal, Object.class) == null; + } + + @Override + public boolean getBoolean(int ordinal) { + return struct.get(ordinal, Boolean.class); + } + + @Override + public byte getByte(int ordinal) { + return (byte) (int) struct.get(ordinal, Integer.class); + } + + @Override + public short getShort(int ordinal) { + return (short) (int) struct.get(ordinal, Integer.class); + } + + @Override + public int getInt(int ordinal) { + Object integer = struct.get(ordinal, Object.class); + + if (integer instanceof Integer) { + return (int) integer; + } else if (integer instanceof LocalDate) { + return (int) ((LocalDate) integer).toEpochDay(); + } else { + throw new IllegalStateException( + "Unknown type for int field. Type name: " + integer.getClass().getName()); + } + } + + @Override + public long getLong(int ordinal) { + Object longVal = struct.get(ordinal, Object.class); + + if (longVal instanceof Long) { + return (long) longVal; + } else if (longVal instanceof OffsetDateTime) { + return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000; + } else if (longVal instanceof LocalDate) { + return ((LocalDate) longVal).toEpochDay(); + } else { + throw new IllegalStateException( + "Unknown type for long field. Type name: " + longVal.getClass().getName()); + } + } + + @Override + public float getFloat(int ordinal) { + return struct.get(ordinal, Float.class); + } + + @Override + public double getDouble(int ordinal) { + return struct.get(ordinal, Double.class); + } + + @Override + public Decimal getDecimal(int ordinal, int precision, int scale) { + return isNullAt(ordinal) ? null : getDecimalInternal(ordinal); + } + + private Decimal getDecimalInternal(int ordinal) { + return Decimal.apply(struct.get(ordinal, BigDecimal.class)); + } + + @Override + public UTF8String getUTF8String(int ordinal) { + return isNullAt(ordinal) ? null : getUTF8StringInternal(ordinal); + } + + private UTF8String getUTF8StringInternal(int ordinal) { + CharSequence seq = struct.get(ordinal, CharSequence.class); + return UTF8String.fromString(seq.toString()); + } + + @Override + public byte[] getBinary(int ordinal) { + return isNullAt(ordinal) ? null : getBinaryInternal(ordinal); + } + + private byte[] getBinaryInternal(int ordinal) { + Object bytes = struct.get(ordinal, Object.class); + + // should only be either ByteBuffer or byte[] + if (bytes instanceof ByteBuffer) { + return ByteBuffers.toByteArray((ByteBuffer) bytes); + } else if (bytes instanceof byte[]) { + return (byte[]) bytes; + } else { + throw new IllegalStateException( + "Unknown type for binary field. Type name: " + bytes.getClass().getName()); + } + } + + @Override + public CalendarInterval getInterval(int ordinal) { + throw new UnsupportedOperationException("Unsupported type: interval"); + } + + @Override + public InternalRow getStruct(int ordinal, int numFields) { + return isNullAt(ordinal) ? null : getStructInternal(ordinal); + } + + private InternalRow getStructInternal(int ordinal) { + return new StructInternalRow( + type.fields().get(ordinal).type().asStructType(), struct.get(ordinal, StructLike.class)); + } + + @Override + public ArrayData getArray(int ordinal) { + return isNullAt(ordinal) ? null : getArrayInternal(ordinal); + } + + private ArrayData getArrayInternal(int ordinal) { + return collectionToArrayData( + type.fields().get(ordinal).type().asListType().elementType(), + struct.get(ordinal, Collection.class)); + } + + @Override + public MapData getMap(int ordinal) { + return isNullAt(ordinal) ? null : getMapInternal(ordinal); + } + + private MapData getMapInternal(int ordinal) { + return mapToMapData( + type.fields().get(ordinal).type().asMapType(), struct.get(ordinal, Map.class)); + } + + @Override + public VariantVal getVariant(int ordinal) { + return isNullAt(ordinal) ? null : getVariantInternal(ordinal); + } + + private VariantVal getVariantInternal(int ordinal) { + Object value = struct.get(ordinal, Object.class); + return toVariantVal(value); + } + + @Override + @SuppressWarnings("checkstyle:CyclomaticComplexity") + public Object get(int ordinal, DataType dataType) { + if (isNullAt(ordinal)) { + return null; + } + + if (dataType instanceof IntegerType) { + return getInt(ordinal); + } else if (dataType instanceof LongType) { + return getLong(ordinal); + } else if (dataType instanceof StringType) { + return getUTF8StringInternal(ordinal); + } else if (dataType instanceof FloatType) { + return getFloat(ordinal); + } else if (dataType instanceof DoubleType) { + return getDouble(ordinal); + } else if (dataType instanceof DecimalType) { + return getDecimalInternal(ordinal); + } else if (dataType instanceof BinaryType) { + return getBinaryInternal(ordinal); + } else if (dataType instanceof StructType) { + return getStructInternal(ordinal); + } else if (dataType instanceof ArrayType) { + return getArrayInternal(ordinal); + } else if (dataType instanceof MapType) { + return getMapInternal(ordinal); + } else if (dataType instanceof BooleanType) { + return getBoolean(ordinal); + } else if (dataType instanceof ByteType) { + return getByte(ordinal); + } else if (dataType instanceof ShortType) { + return getShort(ordinal); + } else if (dataType instanceof DateType) { + return getInt(ordinal); + } else if (dataType instanceof TimestampType) { + return getLong(ordinal); + } else if (dataType instanceof VariantType) { + return getVariantInternal(ordinal); + } + return null; + } + + private MapData mapToMapData(Types.MapType mapType, Map map) { + // make a defensive copy to ensure entries do not change + List> entries = ImmutableList.copyOf(map.entrySet()); + return new ArrayBasedMapData( + collectionToArrayData(mapType.keyType(), Lists.transform(entries, Map.Entry::getKey)), + collectionToArrayData(mapType.valueType(), Lists.transform(entries, Map.Entry::getValue))); + } + + private ArrayData collectionToArrayData(Type elementType, Collection values) { + switch (elementType.typeId()) { + case BOOLEAN: + case INTEGER: + case DATE: + case TIME: + case LONG: + case TIMESTAMP: + case FLOAT: + case DOUBLE: + return fillArray(values, array -> (pos, value) -> array[pos] = value); + case STRING: + return fillArray( + values, + array -> + (BiConsumer) + (pos, seq) -> array[pos] = UTF8String.fromString(seq.toString())); + case FIXED: + case BINARY: + return fillArray( + values, + array -> + (BiConsumer) + (pos, buf) -> array[pos] = ByteBuffers.toByteArray(buf)); + case DECIMAL: + return fillArray( + values, + array -> + (BiConsumer) (pos, dec) -> array[pos] = Decimal.apply(dec)); + case STRUCT: + return fillArray( + values, + array -> + (BiConsumer) + (pos, tuple) -> + array[pos] = new StructInternalRow(elementType.asStructType(), tuple)); + case LIST: + return fillArray( + values, + array -> + (BiConsumer>) + (pos, list) -> + array[pos] = + collectionToArrayData(elementType.asListType().elementType(), list)); + case MAP: + return fillArray( + values, + array -> + (BiConsumer>) + (pos, map) -> array[pos] = mapToMapData(elementType.asMapType(), map)); + case VARIANT: + return fillArray( + values, + array -> (BiConsumer) (pos, v) -> array[pos] = toVariantVal(v)); + default: + throw new UnsupportedOperationException("Unsupported array element type: " + elementType); + } + } + + private static VariantVal toVariantVal(Object value) { + if (value instanceof Variant) { + Variant variant = (Variant) value; + byte[] metadataBytes = new byte[variant.metadata().sizeInBytes()]; + ByteBuffer metadataBuffer = ByteBuffer.wrap(metadataBytes).order(ByteOrder.LITTLE_ENDIAN); + variant.metadata().writeTo(metadataBuffer, 0); + + byte[] valueBytes = new byte[variant.value().sizeInBytes()]; + ByteBuffer valueBuffer = ByteBuffer.wrap(valueBytes).order(ByteOrder.LITTLE_ENDIAN); + variant.value().writeTo(valueBuffer, 0); + + return new VariantVal(valueBytes, metadataBytes); + } + + throw new UnsupportedOperationException( + "Unsupported value for VARIANT in StructInternalRow: " + value.getClass()); + } + + @SuppressWarnings("unchecked") + private GenericArrayData fillArray( + Collection values, Function> makeSetter) { + Object[] array = new Object[values.size()]; + BiConsumer setter = makeSetter.apply(array); + + int index = 0; + for (Object value : values) { + if (value == null) { + array[index] = null; + } else { + setter.accept(index, (T) value); + } + + index += 1; + } + + return new GenericArrayData(array); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + StructInternalRow that = (StructInternalRow) other; + return type.equals(that.type) && struct.equals(that.struct); + } + + @Override + public int hashCode() { + return Objects.hash(type, struct); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/EqualityDeleteFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/EqualityDeleteFiles.java new file mode 100644 index 000000000000..754145f7d252 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/EqualityDeleteFiles.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class EqualityDeleteFiles extends CustomSumMetric { + + static final String NAME = "equalityDeleteFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of equality delete files"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/IndexedDeleteFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/IndexedDeleteFiles.java new file mode 100644 index 000000000000..7fc5b9066cdc --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/IndexedDeleteFiles.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class IndexedDeleteFiles extends CustomSumMetric { + + static final String NAME = "indexedDeleteFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of indexed delete files"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java new file mode 100644 index 000000000000..eeff4286d0ce --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumDeletes.java @@ -0,0 +1,48 @@ +/* + * 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.metrics; + +import java.text.NumberFormat; +import java.util.Locale; +import org.apache.spark.sql.connector.metric.CustomMetric; + +public class NumDeletes implements CustomMetric { + + public static final String DISPLAY_STRING = "number of row deletes applied"; + + @Override + public String name() { + return "numDeletes"; + } + + @Override + public String description() { + return DISPLAY_STRING; + } + + @Override + public String aggregateTaskMetrics(long[] taskMetrics) { + long sum = initialValue; + for (long taskMetric : taskMetrics) { + sum += taskMetric; + } + + return NumberFormat.getIntegerInstance(Locale.ROOT).format(sum); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java new file mode 100644 index 000000000000..f5c7a1f78cff --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/NumSplits.java @@ -0,0 +1,46 @@ +/* + * 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.metrics; + +import java.text.NumberFormat; +import java.util.Locale; +import org.apache.spark.sql.connector.metric.CustomMetric; + +public class NumSplits implements CustomMetric { + + @Override + public String name() { + return "numSplits"; + } + + @Override + public String description() { + return "number of file splits read"; + } + + @Override + public String aggregateTaskMetrics(long[] taskMetrics) { + long sum = initialValue; + for (long taskMetric : taskMetrics) { + sum += taskMetric; + } + + return NumberFormat.getIntegerInstance(Locale.ROOT).format(sum); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/PositionalDeleteFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/PositionalDeleteFiles.java new file mode 100644 index 000000000000..5de75776ea4f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/PositionalDeleteFiles.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class PositionalDeleteFiles extends CustomSumMetric { + + static final String NAME = "positionalDeleteFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of positional delete files"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java new file mode 100644 index 000000000000..21959cbf6c63 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class ResultDataFiles extends CustomSumMetric { + + static final String NAME = "resultDataFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of result data files"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDeleteFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDeleteFiles.java new file mode 100644 index 000000000000..9c6ad2ca328a --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDeleteFiles.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class ResultDeleteFiles extends CustomSumMetric { + + static final String NAME = "resultDeleteFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of result delete files"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java new file mode 100644 index 000000000000..a167904280e6 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataManifests.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class ScannedDataManifests extends CustomSumMetric { + + static final String NAME = "scannedDataManifests"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of scanned data manifests"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDeleteManifests.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDeleteManifests.java new file mode 100644 index 000000000000..1fa006b7b193 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDeleteManifests.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class ScannedDeleteManifests extends CustomSumMetric { + + static final String NAME = "scannedDeleteManifests"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of scanned delete manifests"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java new file mode 100644 index 000000000000..7fd17425313d --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataFiles.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class SkippedDataFiles extends CustomSumMetric { + + static final String NAME = "skippedDataFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of skipped data files"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java new file mode 100644 index 000000000000..b0eaeb5d87f2 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDataManifests.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class SkippedDataManifests extends CustomSumMetric { + + static final String NAME = "skippedDataManifests"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of skipped data manifests"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteFiles.java new file mode 100644 index 000000000000..70597be67113 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteFiles.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class SkippedDeleteFiles extends CustomSumMetric { + + static final String NAME = "skippedDeleteFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of skipped delete files"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteManifests.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteManifests.java new file mode 100644 index 000000000000..0336170b45a1 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteManifests.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class SkippedDeleteManifests extends CustomSumMetric { + + static final String NAME = "skippedDeleteManifests"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of skipped delete manifest"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskEqualityDeleteFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskEqualityDeleteFiles.java new file mode 100644 index 000000000000..ecd14bcca31d --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskEqualityDeleteFiles.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskEqualityDeleteFiles implements CustomTaskMetric { + private final long value; + + private TaskEqualityDeleteFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return EqualityDeleteFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskEqualityDeleteFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().equalityDeleteFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskEqualityDeleteFiles(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskIndexedDeleteFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskIndexedDeleteFiles.java new file mode 100644 index 000000000000..63b6767e955d --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskIndexedDeleteFiles.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskIndexedDeleteFiles implements CustomTaskMetric { + private final long value; + + private TaskIndexedDeleteFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return IndexedDeleteFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskIndexedDeleteFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().indexedDeleteFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskIndexedDeleteFiles(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumDeletes.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumDeletes.java new file mode 100644 index 000000000000..8c734ba9f022 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumDeletes.java @@ -0,0 +1,39 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskNumDeletes implements CustomTaskMetric { + private final long value; + + public TaskNumDeletes(long value) { + this.value = value; + } + + @Override + public String name() { + return "numDeletes"; + } + + @Override + public long value() { + return value; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumSplits.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumSplits.java new file mode 100644 index 000000000000..d8cbc4db05bb --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskNumSplits.java @@ -0,0 +1,39 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskNumSplits implements CustomTaskMetric { + private final long value; + + public TaskNumSplits(long value) { + this.value = value; + } + + @Override + public String name() { + return "numSplits"; + } + + @Override + public long value() { + return value; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskPositionalDeleteFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskPositionalDeleteFiles.java new file mode 100644 index 000000000000..805f22bf0d7c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskPositionalDeleteFiles.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskPositionalDeleteFiles implements CustomTaskMetric { + private final long value; + + private TaskPositionalDeleteFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return PositionalDeleteFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskPositionalDeleteFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().positionalDeleteFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskPositionalDeleteFiles(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java new file mode 100644 index 000000000000..a27142131403 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskResultDataFiles implements CustomTaskMetric { + private final long value; + + private TaskResultDataFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return ResultDataFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskResultDataFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().resultDataFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskResultDataFiles(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDeleteFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDeleteFiles.java new file mode 100644 index 000000000000..aea8ca07dd05 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDeleteFiles.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskResultDeleteFiles implements CustomTaskMetric { + private final long value; + + private TaskResultDeleteFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return ResultDeleteFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskResultDeleteFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().resultDeleteFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskResultDeleteFiles(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java new file mode 100644 index 000000000000..09dd0339910c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataManifests.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskScannedDataManifests implements CustomTaskMetric { + private final long value; + + private TaskScannedDataManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return ScannedDataManifests.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskScannedDataManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().scannedDataManifests(); + long value = counter != null ? counter.value() : 0L; + return new TaskScannedDataManifests(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDeleteManifests.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDeleteManifests.java new file mode 100644 index 000000000000..1766cf2f6835 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDeleteManifests.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskScannedDeleteManifests implements CustomTaskMetric { + private final long value; + + private TaskScannedDeleteManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return ScannedDeleteManifests.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskScannedDeleteManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().scannedDeleteManifests(); + long value = counter != null ? counter.value() : 0L; + return new TaskScannedDeleteManifests(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java new file mode 100644 index 000000000000..5165f9a3116c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataFiles.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskSkippedDataFiles implements CustomTaskMetric { + private final long value; + + private TaskSkippedDataFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return SkippedDataFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskSkippedDataFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().skippedDataFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskSkippedDataFiles(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java new file mode 100644 index 000000000000..86fef8c4118b --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDataManifests.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskSkippedDataManifests implements CustomTaskMetric { + private final long value; + + private TaskSkippedDataManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return SkippedDataManifests.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskSkippedDataManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().skippedDataManifests(); + long value = counter != null ? counter.value() : 0L; + return new TaskSkippedDataManifests(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteFiles.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteFiles.java new file mode 100644 index 000000000000..87579751742c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteFiles.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskSkippedDeleteFiles implements CustomTaskMetric { + private final long value; + + private TaskSkippedDeleteFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return SkippedDeleteFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskSkippedDeleteFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().skippedDeleteFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskSkippedDeleteFiles(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteManifests.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteManifests.java new file mode 100644 index 000000000000..4a9c71e0c1e4 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteManifests.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskSkippedDeleteManifests implements CustomTaskMetric { + private final long value; + + private TaskSkippedDeleteManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return SkippedDeleteManifests.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskSkippedDeleteManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().skippedDeleteManifests(); + long value = counter != null ? counter.value() : 0L; + return new TaskSkippedDeleteManifests(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataFileSize.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataFileSize.java new file mode 100644 index 000000000000..3f5a224425d8 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataFileSize.java @@ -0,0 +1,48 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskTotalDataFileSize implements CustomTaskMetric { + + private final long value; + + private TaskTotalDataFileSize(long value) { + this.value = value; + } + + @Override + public String name() { + return TotalDataFileSize.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskTotalDataFileSize from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().totalFileSizeInBytes(); + long value = counter != null ? counter.value() : 0L; + return new TaskTotalDataFileSize(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataManifests.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataManifests.java new file mode 100644 index 000000000000..6d8c3c24e460 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataManifests.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskTotalDataManifests implements CustomTaskMetric { + private final long value; + + private TaskTotalDataManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return TotalDataManifests.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskTotalDataManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().totalDataManifests(); + long value = counter != null ? counter.value() : 0L; + return new TaskTotalDataManifests(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteFileSize.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteFileSize.java new file mode 100644 index 000000000000..17ecec78da3f --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteFileSize.java @@ -0,0 +1,48 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskTotalDeleteFileSize implements CustomTaskMetric { + + private final long value; + + private TaskTotalDeleteFileSize(long value) { + this.value = value; + } + + @Override + public String name() { + return TotalDeleteFileSize.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskTotalDeleteFileSize from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().totalDeleteFileSizeInBytes(); + long value = counter != null ? counter.value() : 0L; + return new TaskTotalDeleteFileSize(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteManifests.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteManifests.java new file mode 100644 index 000000000000..ff55c1be89e3 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteManifests.java @@ -0,0 +1,47 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskTotalDeleteManifests implements CustomTaskMetric { + private final long value; + + private TaskTotalDeleteManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return TotalDeleteManifests.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskTotalDeleteManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().totalDeleteManifests(); + long value = counter != null ? counter.value() : 0L; + return new TaskTotalDeleteManifests(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java new file mode 100644 index 000000000000..32ac6fde8bf3 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalPlanningDuration.java @@ -0,0 +1,48 @@ +/* + * 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.metrics; + +import org.apache.iceberg.metrics.ScanReport; +import org.apache.iceberg.metrics.TimerResult; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskTotalPlanningDuration implements CustomTaskMetric { + + private final long value; + + private TaskTotalPlanningDuration(long value) { + this.value = value; + } + + @Override + public String name() { + return TotalPlanningDuration.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskTotalPlanningDuration from(ScanReport scanReport) { + TimerResult timerResult = scanReport.scanMetrics().totalPlanningDuration(); + long value = timerResult != null ? timerResult.totalDuration().toMillis() : -1; + return new TaskTotalPlanningDuration(value); + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataFileSize.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataFileSize.java new file mode 100644 index 000000000000..b1ff8a46368c --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataFileSize.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class TotalDataFileSize extends CustomSumMetric { + + static final String NAME = "totalDataFileSize"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "total data file size (bytes)"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataManifests.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataManifests.java new file mode 100644 index 000000000000..de8f04be7767 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataManifests.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class TotalDataManifests extends CustomSumMetric { + + static final String NAME = "totalDataManifest"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "total data manifests"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteFileSize.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteFileSize.java new file mode 100644 index 000000000000..da4303325273 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteFileSize.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class TotalDeleteFileSize extends CustomSumMetric { + + static final String NAME = "totalDeleteFileSize"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "total delete file size (bytes)"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteManifests.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteManifests.java new file mode 100644 index 000000000000..7442dfdb6ffb --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteManifests.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class TotalDeleteManifests extends CustomSumMetric { + + static final String NAME = "totalDeleteManifests"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "total delete manifests"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java new file mode 100644 index 000000000000..8b66eeac4046 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalPlanningDuration.java @@ -0,0 +1,36 @@ +/* + * 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.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class TotalPlanningDuration extends CustomSumMetric { + + static final String NAME = "totalPlanningDuration"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "total planning duration (ms)"; + } +} diff --git a/spark/v4.0/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/IcebergAnalysisException.java b/spark/v4.0/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/IcebergAnalysisException.java new file mode 100644 index 000000000000..1953d7986632 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/IcebergAnalysisException.java @@ -0,0 +1,37 @@ +/* + * 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.spark.QueryContext; +import org.apache.spark.sql.AnalysisException; +import scala.Option; +import scala.collection.immutable.Map$; + +public class IcebergAnalysisException extends AnalysisException { + public IcebergAnalysisException(String message) { + super( + message, + Option.empty(), + Option.empty(), + Option.empty(), + Option.empty(), + Map$.MODULE$.empty(), + new QueryContext[0]); + } +} diff --git a/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister new file mode 100644 index 000000000000..01a6c4e0670d --- /dev/null +++ b/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -0,0 +1,20 @@ +# +# 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. +# + +org.apache.iceberg.spark.source.IcebergSource diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala new file mode 100644 index 000000000000..5d5990c23b57 --- /dev/null +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/OrderAwareCoalesce.scala @@ -0,0 +1,48 @@ +/* + * 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.plans.logical + +import org.apache.spark.rdd.PartitionCoalescer +import org.apache.spark.rdd.PartitionGroup +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Attribute + +// this node doesn't extend RepartitionOperation on purpose to keep this logic isolated +// and ignore it in optimizer rules such as CollapseRepartition +case class OrderAwareCoalesce(numPartitions: Int, coalescer: PartitionCoalescer, child: LogicalPlan) + extends OrderPreservingUnaryNode { + + override def output: Seq[Attribute] = child.output + + override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = { + copy(child = newChild) + } +} + +class OrderAwareCoalescer(val groupSize: Int) extends PartitionCoalescer with Serializable { + + override def coalesce(maxPartitions: Int, parent: RDD[_]): Array[PartitionGroup] = { + val partitionBins = parent.partitions.grouped(groupSize) + partitionBins.map { partitions => + val group = new PartitionGroup() + group.partitions ++= partitions + group + }.toArray + } +} diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala new file mode 100644 index 000000000000..3ce3d3e82ad8 --- /dev/null +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SetWriteDistributionAndOrdering.scala @@ -0,0 +1,46 @@ +/* + * 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.plans.logical + +import org.apache.iceberg.DistributionMode +import org.apache.iceberg.NullOrder +import org.apache.iceberg.SortDirection +import org.apache.iceberg.expressions.Term +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits + +case class SetWriteDistributionAndOrdering( + table: Seq[String], + distributionMode: Option[DistributionMode], + sortOrder: Seq[(Term, SortDirection, NullOrder)]) + extends LeafCommand { + + import CatalogV2Implicits._ + + override lazy val output: Seq[Attribute] = Nil + + override def simpleString(maxFields: Int): String = { + val order = sortOrder + .map { case (term, direction, nullOrder) => + s"$term $direction $nullOrder" + } + .mkString(", ") + s"SetWriteDistributionAndOrdering ${table.quoted} $distributionMode $order" + } +} diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala new file mode 100644 index 000000000000..71b6107d048e --- /dev/null +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SortOrderParserUtil.scala @@ -0,0 +1,41 @@ +/* + * 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.plans.logical + +import org.apache.iceberg.NullOrder +import org.apache.iceberg.Schema +import org.apache.iceberg.SortDirection +import org.apache.iceberg.SortOrder +import org.apache.iceberg.expressions.Term + +class SortOrderParserUtil { + + def collectSortOrder( + tableSchema: Schema, + sortOrder: Seq[(Term, SortDirection, NullOrder)]): SortOrder = { + val orderBuilder = SortOrder.builderFor(tableSchema) + sortOrder.foreach { + case (term, SortDirection.ASC, nullOrder) => + orderBuilder.asc(term, nullOrder) + case (term, SortDirection.DESC, nullOrder) => + orderBuilder.desc(term, nullOrder) + } + orderBuilder.build(); + } +} diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala new file mode 100644 index 000000000000..f46127f818e5 --- /dev/null +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/catalyst/utils/PlanUtils.scala @@ -0,0 +1,41 @@ +/* + * 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.utils + +import org.apache.iceberg.spark.source.SparkTable +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import scala.annotation.tailrec + +object PlanUtils { + @tailrec + def isIcebergRelation(plan: LogicalPlan): Boolean = { + def isIcebergTable(relation: DataSourceV2Relation): Boolean = relation.table match { + case _: SparkTable => true + case _ => false + } + + plan match { + case s: SubqueryAlias => isIcebergRelation(s.child) + case r: DataSourceV2Relation => isIcebergTable(r) + case _ => false + } + } +} diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala new file mode 100644 index 000000000000..83dec98c0d9a --- /dev/null +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/OrderAwareCoalesceExec.scala @@ -0,0 +1,59 @@ +/* + * 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.execution + +import org.apache.spark.rdd.PartitionCoalescer +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.SortOrder +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.plans.physical.SinglePartition +import org.apache.spark.sql.catalyst.plans.physical.UnknownPartitioning + +case class OrderAwareCoalesceExec( + numPartitions: Int, + coalescer: PartitionCoalescer, + child: SparkPlan) + extends UnaryExecNode { + + override def output: Seq[Attribute] = child.output + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def outputPartitioning: Partitioning = { + if (numPartitions == 1) SinglePartition else UnknownPartitioning(numPartitions) + } + + protected override def doExecute(): RDD[InternalRow] = { + val result = child.execute() + if (numPartitions == 1 && result.getNumPartitions < 1) { + // make sure we don't output an RDD with 0 partitions, + // when claiming that we have a `SinglePartition` + // see CoalesceExec in Spark + new CoalesceExec.EmptyRDDWithPartitions(sparkContext, numPartitions) + } else { + result.coalesce(numPartitions, shuffle = false, Some(coalescer)) + } + } + + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = { + copy(child = newChild) + } +} diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala new file mode 100644 index 000000000000..0be425f7d796 --- /dev/null +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/execution/datasources/SparkExpressionConverter.scala @@ -0,0 +1,73 @@ +/* + * 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.execution.datasources + +import org.apache.iceberg.spark.SparkV2Filters +import org.apache.spark.sql.catalyst.analysis.IcebergAnalysisException +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.plans.logical.Filter +import org.apache.spark.sql.catalyst.plans.logical.LeafNode +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.classic.SparkSession +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy + +object SparkExpressionConverter { + + def convertToIcebergExpression( + sparkExpression: Expression): org.apache.iceberg.expressions.Expression = { + // Currently, it is a double conversion as we are converting Spark expression to Spark predicate + // and then converting Spark predicate to Iceberg expression. + // But these two conversions already exist and well tested. So, we are going with this approach. + DataSourceV2Strategy.translateFilterV2(sparkExpression) match { + case Some(filter) => + val converted = SparkV2Filters.convert(filter) + if (converted == null) { + throw new IllegalArgumentException( + s"Cannot convert Spark filter: $filter to Iceberg expression") + } + + converted + case _ => + throw new IllegalArgumentException( + s"Cannot translate Spark expression: $sparkExpression to data source filter") + } + } + + @throws[IcebergAnalysisException] + def collectResolvedSparkExpression( + session: SparkSession, + tableName: String, + where: String): Expression = { + val tableAttrs = session.table(tableName).queryExecution.analyzed.output + val unresolvedExpression = session.sessionState.sqlParser.parseExpression(where) + val filter = Filter(unresolvedExpression, DummyRelation(tableAttrs)) + val optimizedLogicalPlan = session.sessionState.executePlan(filter).optimizedPlan + optimizedLogicalPlan + .collectFirst { + case filter: Filter => filter.condition + case _: DummyRelation => Literal.TrueLiteral + case _: LocalRelation => Literal.FalseLiteral + } + .getOrElse(throw new IcebergAnalysisException("Failed to find filter expression")) + } + + case class DummyRelation(output: Seq[Attribute]) extends LeafNode +} diff --git a/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala new file mode 100644 index 000000000000..c63f3c9c5cf6 --- /dev/null +++ b/spark/v4.0/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala @@ -0,0 +1,122 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.spark.sql.stats + +import java.nio.ByteBuffer +import org.apache.datasketches.common.Family +import org.apache.datasketches.memory.Memory +import org.apache.datasketches.theta.CompactSketch +import org.apache.datasketches.theta.SetOperationBuilder +import org.apache.datasketches.theta.Sketch +import org.apache.datasketches.theta.UpdateSketch +import org.apache.iceberg.spark.SparkSchemaUtil +import org.apache.iceberg.types.Conversions +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.aggregate.ImperativeAggregate +import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate +import org.apache.spark.sql.catalyst.trees.UnaryLike +import org.apache.spark.sql.types.BinaryType +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.types.UTF8String + +/** + * ThetaSketchAgg generates Alpha family sketch with default seed. + * The values fed to the sketch are converted to bytes using Iceberg's single value serialization. + * The result returned is an array of bytes of Compact Theta sketch of Datasketches library, + * which should be deserialized to Compact sketch before using. + * + * See [[https://iceberg.apache.org/puffin-spec/]] for more information. + */ +case class ThetaSketchAgg( + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) + extends TypedImperativeAggregate[Sketch] + with UnaryLike[Expression] { + + private lazy val icebergType = SparkSchemaUtil.convert(child.dataType) + + def this(colName: String) = { + this(analysis.UnresolvedAttribute.quotedString(colName), 0, 0) + } + + override def dataType: DataType = BinaryType + + override def nullable: Boolean = false + + override def createAggregationBuffer(): Sketch = { + UpdateSketch.builder.setFamily(Family.ALPHA).build() + } + + override def update(buffer: Sketch, input: InternalRow): Sketch = { + val value = child.eval(input) + if (value != null) { + val icebergValue = toIcebergValue(value) + val byteBuffer = Conversions.toByteBuffer(icebergType, icebergValue) + buffer.asInstanceOf[UpdateSketch].update(byteBuffer) + } + buffer + } + + private def toIcebergValue(value: Any): Any = { + value match { + case s: UTF8String => s.toString + case d: Decimal => d.toJavaBigDecimal + case b: Array[Byte] => ByteBuffer.wrap(b) + case _ => value + } + } + + override def merge(buffer: Sketch, input: Sketch): Sketch = { + new SetOperationBuilder().buildUnion.union(buffer, input) + } + + override def eval(buffer: Sketch): Any = { + toBytes(buffer) + } + + override def serialize(buffer: Sketch): Array[Byte] = { + toBytes(buffer) + } + + override def deserialize(storageFormat: Array[Byte]): Sketch = { + CompactSketch.wrap(Memory.wrap(storageFormat)) + } + + override def withNewMutableAggBufferOffset( + newMutableAggBufferOffset: Int): ImperativeAggregate = { + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + } + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = { + copy(inputAggBufferOffset = newInputAggBufferOffset) + } + + override protected def withNewChildInternal(newChild: Expression): Expression = { + copy(child = newChild) + } + + private def toBytes(sketch: Sketch): Array[Byte] = { + val compactSketch = sketch.compact() + compactSketch.toByteArray + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/KryoHelpers.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/KryoHelpers.java new file mode 100644 index 000000000000..6d88aaa11813 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/KryoHelpers.java @@ -0,0 +1,51 @@ +/* + * 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; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import org.apache.spark.SparkConf; +import org.apache.spark.serializer.KryoSerializer; + +public class KryoHelpers { + + private KryoHelpers() {} + + @SuppressWarnings("unchecked") + public static T roundTripSerialize(T obj) throws IOException { + Kryo kryo = new KryoSerializer(new SparkConf()).newKryo(); + + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + + try (Output out = new Output(new ObjectOutputStream(bytes))) { + kryo.writeClassAndObject(out, obj); + } + + try (Input in = + new Input(new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray())))) { + return (T) kryo.readClassAndObject(in); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java new file mode 100644 index 000000000000..404ba7284606 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/SparkDistributedDataScanTestBase.java @@ -0,0 +1,95 @@ +/* + * 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; + +import static org.apache.iceberg.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; + +import java.util.Arrays; +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.internal.SQLConf; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class SparkDistributedDataScanTestBase + extends DataTableScanTestBase> { + + @Parameters(name = "formatVersion = {0}, dataMode = {1}, deleteMode = {2}") + public static List parameters() { + return Arrays.asList( + new Object[] {1, LOCAL, LOCAL}, + new Object[] {1, LOCAL, DISTRIBUTED}, + new Object[] {1, DISTRIBUTED, LOCAL}, + new Object[] {1, DISTRIBUTED, DISTRIBUTED}, + new Object[] {2, LOCAL, LOCAL}, + new Object[] {2, LOCAL, DISTRIBUTED}, + new Object[] {2, DISTRIBUTED, LOCAL}, + new Object[] {2, DISTRIBUTED, DISTRIBUTED}); + } + + protected static SparkSession spark = null; + + @Parameter(index = 1) + private PlanningMode dataMode; + + @Parameter(index = 2) + private PlanningMode deleteMode; + + @BeforeEach + public void configurePlanningModes() { + table + .updateProperties() + .set(TableProperties.DATA_PLANNING_MODE, dataMode.modeName()) + .set(TableProperties.DELETE_PLANNING_MODE, deleteMode.modeName()) + .commit(); + } + + @Override + protected BatchScan useRef(BatchScan scan, String ref) { + return scan.useRef(ref); + } + + @Override + protected BatchScan useSnapshot(BatchScan scan, long snapshotId) { + return scan.useSnapshot(snapshotId); + } + + @Override + protected BatchScan asOfTime(BatchScan scan, long timestampMillis) { + return scan.asOfTime(timestampMillis); + } + + @Override + protected BatchScan newScan() { + SparkReadConf readConf = new SparkReadConf(spark, table, ImmutableMap.of()); + return new SparkDistributedDataScan(spark, table, readConf); + } + + protected static SparkSession initSpark(String serializer) { + return SparkSession.builder() + .master("local[2]") + .config("spark.serializer", serializer) + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") + .getOrCreate(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java new file mode 100644 index 000000000000..e4979512a65f --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TaskCheckHelper.java @@ -0,0 +1,108 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Comparator; +import java.util.List; +import java.util.stream.Collectors; + +public final class TaskCheckHelper { + private TaskCheckHelper() {} + + public static void assertEquals( + ScanTaskGroup expected, ScanTaskGroup actual) { + List expectedTasks = getFileScanTasksInFilePathOrder(expected); + List actualTasks = getFileScanTasksInFilePathOrder(actual); + + assertThat(actualTasks) + .as("The number of file scan tasks should match") + .hasSameSizeAs(expectedTasks); + + for (int i = 0; i < expectedTasks.size(); i++) { + FileScanTask expectedTask = expectedTasks.get(i); + FileScanTask actualTask = actualTasks.get(i); + assertEquals(expectedTask, actualTask); + } + } + + public static void assertEquals(FileScanTask expected, FileScanTask actual) { + assertEquals(expected.file(), actual.file()); + + // PartitionSpec implements its own equals method + assertThat(actual.spec()).as("PartitionSpec doesn't match").isEqualTo(expected.spec()); + + assertThat(actual.start()).as("starting position doesn't match").isEqualTo(expected.start()); + + assertThat(actual.start()) + .as("the number of bytes to scan doesn't match") + .isEqualTo(expected.start()); + + // simplify comparison on residual expression via comparing toString + assertThat(actual.residual()) + .asString() + .as("Residual expression doesn't match") + .isEqualTo(expected.residual().toString()); + } + + public static void assertEquals(DataFile expected, DataFile actual) { + assertThat(actual.location()) + .as("Should match the serialized record path") + .isEqualTo(expected.location()); + assertThat(actual.format()) + .as("Should match the serialized record format") + .isEqualTo(expected.format()); + assertThat(actual.partition().get(0, Object.class)) + .as("Should match the serialized record partition") + .isEqualTo(expected.partition().get(0, Object.class)); + assertThat(actual.recordCount()) + .as("Should match the serialized record count") + .isEqualTo(expected.recordCount()); + assertThat(actual.fileSizeInBytes()) + .as("Should match the serialized record size") + .isEqualTo(expected.fileSizeInBytes()); + assertThat(actual.valueCounts()) + .as("Should match the serialized record value counts") + .isEqualTo(expected.valueCounts()); + assertThat(actual.nullValueCounts()) + .as("Should match the serialized record null value counts") + .isEqualTo(expected.nullValueCounts()); + assertThat(actual.lowerBounds()) + .as("Should match the serialized record lower bounds") + .isEqualTo(expected.lowerBounds()); + assertThat(actual.upperBounds()) + .as("Should match the serialized record upper bounds") + .isEqualTo(expected.upperBounds()); + assertThat(actual.keyMetadata()) + .as("Should match the serialized record key metadata") + .isEqualTo(expected.keyMetadata()); + assertThat(actual.splitOffsets()) + .as("Should match the serialized record offsets") + .isEqualTo(expected.splitOffsets()); + } + + private static List getFileScanTasksInFilePathOrder( + ScanTaskGroup taskGroup) { + return taskGroup.tasks().stream() + // use file path + start position to differentiate the tasks + .sorted(Comparator.comparing(o -> o.file().location() + "##" + o.start())) + .collect(Collectors.toList()); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java new file mode 100644 index 000000000000..57c4dc7cdf23 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java @@ -0,0 +1,175 @@ +/* + * 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; + +import static org.apache.iceberg.TaskCheckHelper.assertEquals; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.file.Path; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.SparkParquetWriters; +import org.apache.iceberg.types.Types; +import org.apache.spark.SparkConf; +import org.apache.spark.serializer.KryoSerializer; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestDataFileSerialization { + + private static final Schema DATE_SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get()), + optional(4, "double", Types.DoubleType.get())); + + private static final PartitionSpec PARTITION_SPEC = + PartitionSpec.builderFor(DATE_SCHEMA).identity("date").build(); + + private static final Map VALUE_COUNTS = Maps.newHashMap(); + private static final Map NULL_VALUE_COUNTS = Maps.newHashMap(); + private static final Map NAN_VALUE_COUNTS = Maps.newHashMap(); + private static final Map LOWER_BOUNDS = Maps.newHashMap(); + private static final Map UPPER_BOUNDS = Maps.newHashMap(); + + static { + VALUE_COUNTS.put(1, 5L); + VALUE_COUNTS.put(2, 3L); + VALUE_COUNTS.put(4, 2L); + NULL_VALUE_COUNTS.put(1, 0L); + NULL_VALUE_COUNTS.put(2, 2L); + NAN_VALUE_COUNTS.put(4, 1L); + LOWER_BOUNDS.put(1, longToBuffer(0L)); + UPPER_BOUNDS.put(1, longToBuffer(4L)); + } + + private static final DataFile DATA_FILE = + DataFiles.builder(PARTITION_SPEC) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(1234) + .withPartitionPath("date=2018-06-08") + .withMetrics( + new Metrics( + 5L, + null, + VALUE_COUNTS, + NULL_VALUE_COUNTS, + NAN_VALUE_COUNTS, + LOWER_BOUNDS, + UPPER_BOUNDS)) + .withSplitOffsets(ImmutableList.of(4L)) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(34)) + .withSortOrder(SortOrder.unsorted()) + .build(); + + @TempDir private Path temp; + + @Test + public void testDataFileKryoSerialization() throws Exception { + File data = File.createTempFile("junit", null, temp.toFile()); + assertThat(data.delete()).isTrue(); + Kryo kryo = new KryoSerializer(new SparkConf()).newKryo(); + + try (Output out = new Output(new FileOutputStream(data))) { + kryo.writeClassAndObject(out, DATA_FILE); + kryo.writeClassAndObject(out, DATA_FILE.copy()); + } + + try (Input in = new Input(new FileInputStream(data))) { + for (int i = 0; i < 2; i += 1) { + Object obj = kryo.readClassAndObject(in); + assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); + assertEquals(DATA_FILE, (DataFile) obj); + } + } + } + + @Test + public void testDataFileJavaSerialization() throws Exception { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(DATA_FILE); + out.writeObject(DATA_FILE.copy()); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + for (int i = 0; i < 2; i += 1) { + Object obj = in.readObject(); + assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); + assertEquals(DATA_FILE, (DataFile) obj); + } + } + } + + @Test + public void testParquetWriterSplitOffsets() throws IOException { + Iterable records = RandomData.generateSpark(DATE_SCHEMA, 1, 33L); + File parquetFile = + new File(temp.toFile(), FileFormat.PARQUET.addExtension(UUID.randomUUID().toString())); + FileAppender writer = + Parquet.write(Files.localOutput(parquetFile)) + .schema(DATE_SCHEMA) + .createWriterFunc( + msgType -> + SparkParquetWriters.buildWriter(SparkSchemaUtil.convert(DATE_SCHEMA), msgType)) + .build(); + try { + writer.addAll(records); + } finally { + writer.close(); + } + + Kryo kryo = new KryoSerializer(new SparkConf()).newKryo(); + File dataFile = File.createTempFile("junit", null, temp.toFile()); + try (Output out = new Output(new FileOutputStream(dataFile))) { + kryo.writeClassAndObject(out, writer.splitOffsets()); + } + try (Input in = new Input(new FileInputStream(dataFile))) { + kryo.readClassAndObject(in); + } + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestFileIOSerialization.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestFileIOSerialization.java new file mode 100644 index 000000000000..cdd2443cc0e0 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestFileIOSerialization.java @@ -0,0 +1,106 @@ +/* + * 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; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.source.SerializableTableWithSize; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestFileIOSerialization { + + private static final Configuration CONF = new Configuration(); + private static final HadoopTables TABLES = new HadoopTables(CONF); + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get()), + optional(4, "double", Types.DoubleType.get())); + + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("date").build(); + + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + + static { + CONF.set("k1", "v1"); + CONF.set("k2", "v2"); + } + + @TempDir private Path temp; + private Table table; + + @BeforeEach + public void initTable() throws IOException { + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + + File tableLocation = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableLocation.delete()).isTrue(); + + this.table = TABLES.create(SCHEMA, SPEC, SORT_ORDER, props, tableLocation.toString()); + } + + @Test + public void testHadoopFileIOKryoSerialization() throws IOException { + FileIO io = table.io(); + Configuration expectedConf = ((HadoopFileIO) io).conf(); + + Table serializableTable = SerializableTableWithSize.copyOf(table); + FileIO deserializedIO = KryoHelpers.roundTripSerialize(serializableTable.io()); + Configuration actualConf = ((HadoopFileIO) deserializedIO).conf(); + + assertThat(actualConf).containsExactlyInAnyOrderElementsOf(expectedConf); + } + + @Test + public void testHadoopFileIOJavaSerialization() throws IOException, ClassNotFoundException { + FileIO io = table.io(); + Configuration expectedConf = ((HadoopFileIO) io).conf(); + + Table serializableTable = SerializableTableWithSize.copyOf(table); + FileIO deserializedIO = TestHelpers.roundTripSerialize(serializableTable.io()); + Configuration actualConf = ((HadoopFileIO) deserializedIO).conf(); + + assertThat(actualConf).containsExactlyInAnyOrderElementsOf(expectedConf); + } + + private Map toMap(Configuration conf) { + Map map = Maps.newHashMapWithExpectedSize(conf.size()); + conf.forEach(entry -> map.put(entry.getKey(), entry.getValue())); + return map; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestHadoopMetricsContextSerialization.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestHadoopMetricsContextSerialization.java new file mode 100644 index 000000000000..a4643d7a087b --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestHadoopMetricsContextSerialization.java @@ -0,0 +1,56 @@ +/* + * 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; + +import java.io.IOException; +import org.apache.iceberg.hadoop.HadoopMetricsContext; +import org.apache.iceberg.io.FileIOMetricsContext; +import org.apache.iceberg.metrics.MetricsContext; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.Test; + +public class TestHadoopMetricsContextSerialization { + + @Test + public void testHadoopMetricsContextKryoSerialization() throws IOException { + MetricsContext metricsContext = new HadoopMetricsContext("s3"); + + metricsContext.initialize(Maps.newHashMap()); + + MetricsContext deserializedMetricContext = KryoHelpers.roundTripSerialize(metricsContext); + // statistics are properly re-initialized post de-serialization + deserializedMetricContext + .counter(FileIOMetricsContext.WRITE_BYTES, MetricsContext.Unit.BYTES) + .increment(); + } + + @Test + public void testHadoopMetricsContextJavaSerialization() + throws IOException, ClassNotFoundException { + MetricsContext metricsContext = new HadoopMetricsContext("s3"); + + metricsContext.initialize(Maps.newHashMap()); + + MetricsContext deserializedMetricContext = TestHelpers.roundTripSerialize(metricsContext); + // statistics are properly re-initialized post de-serialization + deserializedMetricContext + .counter(FileIOMetricsContext.WRITE_BYTES, MetricsContext.Unit.BYTES) + .increment(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java new file mode 100644 index 000000000000..1e09917d0305 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java @@ -0,0 +1,221 @@ +/* + * 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; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.file.Path; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.ManifestFile.PartitionFieldSummary; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.apache.spark.SparkConf; +import org.apache.spark.serializer.KryoSerializer; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestManifestFileSerialization { + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get()), + required(4, "double", Types.DoubleType.get())); + + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("double").build(); + + private static final DataFile FILE_A = + DataFiles.builder(SPEC) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(0) + .withPartition(TestHelpers.Row.of(1D)) + .withPartitionPath("double=1") + .withMetrics( + new Metrics( + 5L, + null, // no column sizes + ImmutableMap.of(1, 5L, 2, 3L), // value count + ImmutableMap.of(1, 0L, 2, 2L), // null count + ImmutableMap.of(), // nan count + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + private static final DataFile FILE_B = + DataFiles.builder(SPEC) + .withPath("/path/to/data-2.parquet") + .withFileSizeInBytes(0) + .withPartition(TestHelpers.Row.of(Double.NaN)) + .withPartitionPath("double=NaN") + .withMetrics( + new Metrics( + 1L, + null, // no column sizes + ImmutableMap.of(1, 1L, 4, 1L), // value count + ImmutableMap.of(1, 0L, 2, 0L), // null count + ImmutableMap.of(4, 1L), // nan count + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(1L)) // upper bounds + )) + .build(); + + private static final FileIO FILE_IO = new HadoopFileIO(new Configuration()); + + @TempDir private Path temp; + + @Test + public void testManifestFileKryoSerialization() throws IOException { + File data = File.createTempFile("junit", null, temp.toFile()); + assertThat(data.delete()).isTrue(); + + Kryo kryo = new KryoSerializer(new SparkConf()).newKryo(); + + ManifestFile manifest = writeManifest(FILE_A, FILE_B); + + try (Output out = new Output(new FileOutputStream(data))) { + kryo.writeClassAndObject(out, manifest); + kryo.writeClassAndObject(out, manifest.copy()); + kryo.writeClassAndObject(out, GenericManifestFile.copyOf(manifest).build()); + } + + try (Input in = new Input(new FileInputStream(data))) { + for (int i = 0; i < 3; i += 1) { + Object obj = kryo.readClassAndObject(in); + assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); + checkManifestFile(manifest, (ManifestFile) obj); + } + } + } + + @Test + public void testManifestFileJavaSerialization() throws Exception { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + + ManifestFile manifest = writeManifest(FILE_A, FILE_B); + + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(manifest); + out.writeObject(manifest.copy()); + out.writeObject(GenericManifestFile.copyOf(manifest).build()); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + for (int i = 0; i < 3; i += 1) { + Object obj = in.readObject(); + assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); + checkManifestFile(manifest, (ManifestFile) obj); + } + } + } + + private void checkManifestFile(ManifestFile expected, ManifestFile actual) { + assertThat(actual.path()).as("Path must match").isEqualTo(expected.path()); + assertThat(actual.length()).as("Length must match").isEqualTo(expected.length()); + assertThat(actual.partitionSpecId()) + .as("Spec id must match") + .isEqualTo(expected.partitionSpecId()); + assertThat(actual.snapshotId()).as("Snapshot id must match").isEqualTo(expected.snapshotId()); + assertThat(actual.hasAddedFiles()) + .as("Added files flag must match") + .isEqualTo(expected.hasAddedFiles()); + assertThat(actual.addedFilesCount()) + .as("Added files count must match") + .isEqualTo(expected.addedFilesCount()); + assertThat(actual.addedRowsCount()) + .as("Added rows count must match") + .isEqualTo(expected.addedRowsCount()); + assertThat(actual.hasExistingFiles()) + .as("Existing files flag must match") + .isEqualTo(expected.hasExistingFiles()); + assertThat(actual.existingFilesCount()) + .as("Existing files count must match") + .isEqualTo(expected.existingFilesCount()); + assertThat(actual.existingRowsCount()) + .as("Existing rows count must match") + .isEqualTo(expected.existingRowsCount()); + assertThat(actual.hasDeletedFiles()) + .as("Deleted files flag must match") + .isEqualTo(expected.hasDeletedFiles()); + assertThat(actual.deletedFilesCount()) + .as("Deleted files count must match") + .isEqualTo(expected.deletedFilesCount()); + assertThat(actual.deletedRowsCount()) + .as("Deleted rows count must match") + .isEqualTo(expected.deletedRowsCount()); + + PartitionFieldSummary expectedPartition = expected.partitions().get(0); + PartitionFieldSummary actualPartition = actual.partitions().get(0); + + assertThat(actualPartition.containsNull()) + .as("Null flag in partition must match") + .isEqualTo(expectedPartition.containsNull()); + assertThat(actualPartition.containsNaN()) + .as("NaN flag in partition must match") + .isEqualTo(expectedPartition.containsNaN()); + assertThat(actualPartition.lowerBound()) + .as("Lower bounds in partition must match") + .isEqualTo(expectedPartition.lowerBound()); + assertThat(actualPartition.upperBound()) + .as("Upper bounds in partition must match") + .isEqualTo(expectedPartition.upperBound()); + } + + private ManifestFile writeManifest(DataFile... files) throws IOException { + File manifestFile = File.createTempFile("input.m0", ".avro", temp.toFile()); + assertThat(manifestFile.delete()).isTrue(); + OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); + + ManifestWriter writer = ManifestFiles.write(SPEC, outputFile); + try { + for (DataFile file : files) { + writer.add(file); + } + } finally { + writer.close(); + } + + return writer.toManifestFile(); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java new file mode 100644 index 000000000000..66ed837eafb5 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java @@ -0,0 +1,197 @@ +/* + * 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; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.CloseableIterable; +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.TestBase; +import org.apache.iceberg.spark.source.ThreeColumnRecord; +import org.apache.iceberg.types.Types; +import org.apache.spark.SparkConf; +import org.apache.spark.serializer.KryoSerializer; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestScanTaskSerialization extends TestBase { + + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + private static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + + @TempDir private Path temp; + @TempDir private File tableDir; + + private String tableLocation = null; + + @BeforeEach + public void setupTableLocation() { + this.tableLocation = tableDir.toURI().toString(); + } + + @Test + public void testBaseCombinedScanTaskKryoSerialization() throws Exception { + BaseCombinedScanTask scanTask = prepareBaseCombinedScanTaskForSerDeTest(); + + File data = File.createTempFile("junit", null, temp.toFile()); + assertThat(data.delete()).isTrue(); + Kryo kryo = new KryoSerializer(new SparkConf()).newKryo(); + + try (Output out = new Output(new FileOutputStream(data))) { + kryo.writeClassAndObject(out, scanTask); + } + + try (Input in = new Input(new FileInputStream(data))) { + Object obj = kryo.readClassAndObject(in); + assertThat(obj) + .as("Should be a BaseCombinedScanTask") + .isInstanceOf(BaseCombinedScanTask.class); + TaskCheckHelper.assertEquals(scanTask, (BaseCombinedScanTask) obj); + } + } + + @Test + public void testBaseCombinedScanTaskJavaSerialization() throws Exception { + BaseCombinedScanTask scanTask = prepareBaseCombinedScanTaskForSerDeTest(); + + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(scanTask); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + Object obj = in.readObject(); + assertThat(obj) + .as("Should be a BaseCombinedScanTask") + .isInstanceOf(BaseCombinedScanTask.class); + TaskCheckHelper.assertEquals(scanTask, (BaseCombinedScanTask) obj); + } + } + + @Test + @SuppressWarnings("unchecked") + public void testBaseScanTaskGroupKryoSerialization() throws Exception { + BaseScanTaskGroup taskGroup = prepareBaseScanTaskGroupForSerDeTest(); + + assertThat(taskGroup.tasks()).as("Task group can't be empty").isNotEmpty(); + + File data = File.createTempFile("junit", null, temp.toFile()); + assertThat(data.delete()).isTrue(); + Kryo kryo = new KryoSerializer(new SparkConf()).newKryo(); + + try (Output out = new Output(Files.newOutputStream(data.toPath()))) { + kryo.writeClassAndObject(out, taskGroup); + } + + try (Input in = new Input(Files.newInputStream(data.toPath()))) { + Object obj = kryo.readClassAndObject(in); + assertThat(obj).as("should be a BaseScanTaskGroup").isInstanceOf(BaseScanTaskGroup.class); + TaskCheckHelper.assertEquals(taskGroup, (BaseScanTaskGroup) obj); + } + } + + @Test + @SuppressWarnings("unchecked") + public void testBaseScanTaskGroupJavaSerialization() throws Exception { + BaseScanTaskGroup taskGroup = prepareBaseScanTaskGroupForSerDeTest(); + + assertThat(taskGroup.tasks()).as("Task group can't be empty").isNotEmpty(); + + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(taskGroup); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + Object obj = in.readObject(); + assertThat(obj).as("should be a BaseScanTaskGroup").isInstanceOf(BaseScanTaskGroup.class); + TaskCheckHelper.assertEquals(taskGroup, (BaseScanTaskGroup) obj); + } + } + + private BaseCombinedScanTask prepareBaseCombinedScanTaskForSerDeTest() { + Table table = initTable(); + CloseableIterable tasks = table.newScan().planFiles(); + return new BaseCombinedScanTask(Lists.newArrayList(tasks)); + } + + private BaseScanTaskGroup prepareBaseScanTaskGroupForSerDeTest() { + Table table = initTable(); + CloseableIterable tasks = table.newScan().planFiles(); + return new BaseScanTaskGroup<>(ImmutableList.copyOf(tasks)); + } + + private Table initTable() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + List records1 = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), new ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")); + writeRecords(records1); + + List records2 = + Lists.newArrayList( + new ThreeColumnRecord(2, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(2, "DDDDDDDDDD", "DDDD")); + writeRecords(records2); + + table.refresh(); + + return table; + } + + private void writeRecords(List records) { + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); + writeDF(df); + } + + private void writeDF(Dataset df) { + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java new file mode 100644 index 000000000000..659507e4c5e3 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanDeletes.java @@ -0,0 +1,91 @@ +/* + * 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; + +import static org.apache.iceberg.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; + +import java.util.Arrays; +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.internal.SQLConf; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkDistributedDataScanDeletes + extends DeleteFileIndexTestBase> { + + @Parameters(name = "formatVersion = {0}, dataMode = {1}, deleteMode = {2}") + public static List parameters() { + return Arrays.asList( + new Object[] {2, LOCAL, LOCAL}, + new Object[] {2, LOCAL, DISTRIBUTED}, + new Object[] {2, DISTRIBUTED, LOCAL}, + new Object[] {2, LOCAL, DISTRIBUTED}, + new Object[] {3, LOCAL, LOCAL}, + new Object[] {3, LOCAL, DISTRIBUTED}, + new Object[] {3, DISTRIBUTED, LOCAL}, + new Object[] {3, DISTRIBUTED, DISTRIBUTED}); + } + + private static SparkSession spark = null; + + @Parameter(index = 1) + private PlanningMode dataMode; + + @Parameter(index = 2) + private PlanningMode deleteMode; + + @BeforeEach + public void configurePlanningModes() { + table + .updateProperties() + .set(TableProperties.DATA_PLANNING_MODE, dataMode.modeName()) + .set(TableProperties.DELETE_PLANNING_MODE, deleteMode.modeName()) + .commit(); + } + + @BeforeAll + public static void startSpark() { + TestSparkDistributedDataScanDeletes.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") + .getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestSparkDistributedDataScanDeletes.spark; + TestSparkDistributedDataScanDeletes.spark = null; + currentSpark.stop(); + } + + @Override + protected BatchScan newScan(Table table) { + SparkReadConf readConf = new SparkReadConf(spark, table, ImmutableMap.of()); + return new SparkDistributedDataScan(spark, table, readConf); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java new file mode 100644 index 000000000000..a218f965ea65 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanFilterFiles.java @@ -0,0 +1,85 @@ +/* + * 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; + +import static org.apache.iceberg.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; + +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.internal.SQLConf; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkDistributedDataScanFilterFiles + extends FilterFilesTestBase> { + + @Parameters(name = "formatVersion = {0}, dataMode = {1}, deleteMode = {2}") + public static Object[] parameters() { + return new Object[][] { + new Object[] {1, LOCAL, LOCAL}, + new Object[] {1, LOCAL, DISTRIBUTED}, + new Object[] {1, DISTRIBUTED, LOCAL}, + new Object[] {1, DISTRIBUTED, DISTRIBUTED}, + new Object[] {2, LOCAL, LOCAL}, + new Object[] {2, LOCAL, DISTRIBUTED}, + new Object[] {2, DISTRIBUTED, LOCAL}, + new Object[] {2, DISTRIBUTED, DISTRIBUTED} + }; + } + + private static SparkSession spark = null; + + @Parameter(index = 1) + private PlanningMode dataMode; + + @Parameter(index = 2) + private PlanningMode deleteMode; + + @BeforeAll + public static void startSpark() { + TestSparkDistributedDataScanFilterFiles.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") + .getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestSparkDistributedDataScanFilterFiles.spark; + TestSparkDistributedDataScanFilterFiles.spark = null; + currentSpark.stop(); + } + + @Override + protected BatchScan newScan(Table table) { + table + .updateProperties() + .set(TableProperties.DATA_PLANNING_MODE, dataMode.modeName()) + .set(TableProperties.DELETE_PLANNING_MODE, deleteMode.modeName()) + .commit(); + SparkReadConf readConf = new SparkReadConf(spark, table, ImmutableMap.of()); + return new SparkDistributedDataScan(spark, table, readConf); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java new file mode 100644 index 000000000000..b8bd6fb86747 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanJavaSerialization.java @@ -0,0 +1,40 @@ +/* + * 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; + +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; + +public class TestSparkDistributedDataScanJavaSerialization + extends SparkDistributedDataScanTestBase { + + @BeforeAll + public static void startSpark() { + SparkDistributedDataScanTestBase.spark = + initSpark("org.apache.spark.serializer.JavaSerializer"); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = SparkDistributedDataScanTestBase.spark; + SparkDistributedDataScanTestBase.spark = null; + currentSpark.stop(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java new file mode 100644 index 000000000000..08d66cccb627 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanKryoSerialization.java @@ -0,0 +1,40 @@ +/* + * 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; + +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; + +public class TestSparkDistributedDataScanKryoSerialization + extends SparkDistributedDataScanTestBase { + + @BeforeAll + public static void startSpark() { + SparkDistributedDataScanTestBase.spark = + initSpark("org.apache.spark.serializer.KryoSerializer"); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = SparkDistributedDataScanTestBase.spark; + SparkDistributedDataScanTestBase.spark = null; + currentSpark.stop(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java new file mode 100644 index 000000000000..2665d7ba8d3b --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestSparkDistributedDataScanReporting.java @@ -0,0 +1,86 @@ +/* + * 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; + +import static org.apache.iceberg.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; + +import java.util.Arrays; +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.internal.SQLConf; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkDistributedDataScanReporting + extends ScanPlanningAndReportingTestBase> { + + @Parameters(name = "formatVersion = {0}, dataMode = {1}, deleteMode = {2}") + public static List parameters() { + return Arrays.asList( + new Object[] {2, LOCAL, LOCAL}, + new Object[] {2, LOCAL, DISTRIBUTED}, + new Object[] {2, DISTRIBUTED, LOCAL}, + new Object[] {2, DISTRIBUTED, DISTRIBUTED}, + new Object[] {3, LOCAL, LOCAL}, + new Object[] {3, LOCAL, DISTRIBUTED}, + new Object[] {3, DISTRIBUTED, LOCAL}, + new Object[] {3, DISTRIBUTED, DISTRIBUTED}); + } + + private static SparkSession spark = null; + + @Parameter(index = 1) + private PlanningMode dataMode; + + @Parameter(index = 2) + private PlanningMode deleteMode; + + @BeforeAll + public static void startSpark() { + TestSparkDistributedDataScanReporting.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + .config(SQLConf.SHUFFLE_PARTITIONS().key(), "4") + .getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestSparkDistributedDataScanReporting.spark; + TestSparkDistributedDataScanReporting.spark = null; + currentSpark.stop(); + } + + @Override + protected BatchScan newScan(Table table) { + table + .updateProperties() + .set(TableProperties.DATA_PLANNING_MODE, dataMode.modeName()) + .set(TableProperties.DELETE_PLANNING_MODE, deleteMode.modeName()) + .commit(); + SparkReadConf readConf = new SparkReadConf(spark, table, ImmutableMap.of()); + return new SparkDistributedDataScan(spark, table, readConf); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestTableSerialization.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestTableSerialization.java new file mode 100644 index 000000000000..6e510299c632 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/TestTableSerialization.java @@ -0,0 +1,216 @@ +/* + * 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; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.source.SerializableTableWithSize; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestTableSerialization { + + @Parameters(name = "isObjectStoreEnabled = {0}") + public static List parameters() { + return Arrays.asList("true", "false"); + } + + private static final HadoopTables TABLES = new HadoopTables(); + + @Parameter private String isObjectStoreEnabled; + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get()), + optional(4, "double", Types.DoubleType.get())); + + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("date").build(); + + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + + @TempDir private Path temp; + private Table table; + + @BeforeEach + public void initTable() throws IOException { + Map props = + ImmutableMap.of("k1", "v1", TableProperties.OBJECT_STORE_ENABLED, isObjectStoreEnabled); + + File tableLocation = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableLocation.delete()).isTrue(); + + this.table = TABLES.create(SCHEMA, SPEC, SORT_ORDER, props, tableLocation.toString()); + } + + @TestTemplate + public void testCloseSerializableTableKryoSerialization() throws Exception { + for (Table tbl : tables()) { + Table spyTable = spy(tbl); + FileIO spyIO = spy(tbl.io()); + when(spyTable.io()).thenReturn(spyIO); + + Table serializableTable = SerializableTableWithSize.copyOf(spyTable); + + Table serializableTableCopy = spy(KryoHelpers.roundTripSerialize(serializableTable)); + FileIO spyFileIOCopy = spy(serializableTableCopy.io()); + when(serializableTableCopy.io()).thenReturn(spyFileIOCopy); + + ((AutoCloseable) serializableTable).close(); // mimics close on the driver + ((AutoCloseable) serializableTableCopy).close(); // mimics close on executors + + verify(spyIO, never()).close(); + verify(spyFileIOCopy, times(1)).close(); + } + } + + @TestTemplate + public void testCloseSerializableTableJavaSerialization() throws Exception { + for (Table tbl : tables()) { + Table spyTable = spy(tbl); + FileIO spyIO = spy(tbl.io()); + when(spyTable.io()).thenReturn(spyIO); + + Table serializableTable = SerializableTableWithSize.copyOf(spyTable); + + Table serializableTableCopy = spy(TestHelpers.roundTripSerialize(serializableTable)); + FileIO spyFileIOCopy = spy(serializableTableCopy.io()); + when(serializableTableCopy.io()).thenReturn(spyFileIOCopy); + + ((AutoCloseable) serializableTable).close(); // mimics close on the driver + ((AutoCloseable) serializableTableCopy).close(); // mimics close on executors + + verify(spyIO, never()).close(); + verify(spyFileIOCopy, times(1)).close(); + } + } + + @TestTemplate + public void testSerializableTableKryoSerialization() throws IOException { + Table serializableTable = SerializableTableWithSize.copyOf(table); + TestHelpers.assertSerializedAndLoadedMetadata( + table, KryoHelpers.roundTripSerialize(serializableTable)); + } + + @TestTemplate + public void testSerializableMetadataTableKryoSerialization() throws IOException { + for (MetadataTableType type : MetadataTableType.values()) { + TableOperations ops = ((HasTableOperations) table).operations(); + Table metadataTable = + MetadataTableUtils.createMetadataTableInstance(ops, table.name(), "meta", type); + Table serializableMetadataTable = SerializableTableWithSize.copyOf(metadataTable); + + TestHelpers.assertSerializedAndLoadedMetadata( + metadataTable, KryoHelpers.roundTripSerialize(serializableMetadataTable)); + } + } + + @TestTemplate + public void testSerializableTransactionTableKryoSerialization() throws IOException { + Transaction txn = table.newTransaction(); + + txn.updateProperties().set("k1", "v1").commit(); + + Table txnTable = txn.table(); + Table serializableTxnTable = SerializableTableWithSize.copyOf(txnTable); + + TestHelpers.assertSerializedMetadata( + txnTable, KryoHelpers.roundTripSerialize(serializableTxnTable)); + } + + @TestTemplate + public void testLocationProviderExceptionIsDeferred() { + Table spyTable = spy(table); + RuntimeException failure = new RuntimeException("location provider failure"); + when(spyTable.locationProvider()).thenThrow(failure); + + Table serializableTable = SerializableTableWithSize.copyOf(spyTable); + assertThat(serializableTable).isNotNull(); + + assertThatThrownBy(serializableTable::locationProvider).isSameAs(failure); + verify(spyTable, times(1)).locationProvider(); + } + + @TestTemplate + public void testLocationProviderExceptionJavaSerialization() + throws IOException, ClassNotFoundException { + Table spyTable = spy(table); + RuntimeException failure = new RuntimeException("location provider failure"); + when(spyTable.locationProvider()).thenThrow(failure); + + Table serializableTable = SerializableTableWithSize.copyOf(spyTable); + Table deserialized = TestHelpers.roundTripSerialize(serializableTable); + + assertThatThrownBy(deserialized::locationProvider) + .isInstanceOf(RuntimeException.class) + .hasMessage("location provider failure"); + } + + @TestTemplate + public void testLocationProviderExceptionKryoSerialization() throws IOException { + Table spyTable = spy(table); + RuntimeException failure = new RuntimeException("location provider failure"); + when(spyTable.locationProvider()).thenThrow(failure); + + Table serializableTable = SerializableTableWithSize.copyOf(spyTable); + Table deserialized = KryoHelpers.roundTripSerialize(serializableTable); + + assertThatThrownBy(deserialized::locationProvider) + .isInstanceOf(RuntimeException.class) + .hasMessage("location provider failure"); + } + + private List
    tables() { + List
    tables = Lists.newArrayList(); + + tables.add(table); + + for (MetadataTableType type : MetadataTableType.values()) { + Table metadataTable = MetadataTableUtils.createMetadataTableInstance(table, type); + tables.add(metadataTable); + } + + return tables; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java new file mode 100644 index 000000000000..dac09fea7562 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/ValidationHelpers.java @@ -0,0 +1,78 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public class ValidationHelpers { + + private ValidationHelpers() {} + + public static List dataSeqs(Long... seqs) { + return Arrays.asList(seqs); + } + + public static List fileSeqs(Long... seqs) { + return Arrays.asList(seqs); + } + + public static List snapshotIds(Long... ids) { + return Arrays.asList(ids); + } + + public static List files(ContentFile... files) { + return Arrays.stream(files).map(ContentFile::location).collect(Collectors.toList()); + } + + public static void validateDataManifest( + Table table, + ManifestFile manifest, + List dataSeqs, + List fileSeqs, + List snapshotIds, + List files) { + + List actualDataSeqs = Lists.newArrayList(); + List actualFileSeqs = Lists.newArrayList(); + List actualSnapshotIds = Lists.newArrayList(); + List actualFiles = Lists.newArrayList(); + + for (ManifestEntry entry : ManifestFiles.read(manifest, table.io()).entries()) { + actualDataSeqs.add(entry.dataSequenceNumber()); + actualFileSeqs.add(entry.fileSequenceNumber()); + actualSnapshotIds.add(entry.snapshotId()); + actualFiles.add(entry.file().location()); + } + + assertSameElements("data seqs", actualDataSeqs, dataSeqs); + assertSameElements("file seqs", actualFileSeqs, fileSeqs); + assertSameElements("snapshot IDs", actualSnapshotIds, snapshotIds); + assertSameElements("files", actualFiles, files); + } + + private static void assertSameElements(String context, List actual, List expected) { + String errorMessage = String.format("%s must match", context); + assertThat(actual).as(errorMessage).hasSameElementsAs(expected); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java new file mode 100644 index 000000000000..fe4d7bc01ba4 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java @@ -0,0 +1,59 @@ +/* + * 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 org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class CatalogTestBase extends TestBaseWithCatalog { + + // these parameters are broken out to avoid changes that need to modify lots of test suites + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties() + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties() + }, + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties() + }, + { + SparkCatalogConfig.REST.catalogName(), + SparkCatalogConfig.REST.implementation(), + ImmutableMap.builder() + .putAll(SparkCatalogConfig.REST.properties()) + .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) + .build() + } + }; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/Employee.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/Employee.java new file mode 100644 index 000000000000..9c57936d989e --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/Employee.java @@ -0,0 +1,66 @@ +/* + * 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.Objects; + +public class Employee { + private Integer id; + private String dep; + + public Employee() {} + + public Employee(Integer id, String dep) { + this.id = id; + this.dep = dep; + } + + public Integer getId() { + return id; + } + + public void setId(Integer id) { + this.id = id; + } + + public String getDep() { + return dep; + } + + public void setDep(String dep) { + this.dep = dep; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + Employee employee = (Employee) other; + return Objects.equals(id, employee.id) && Objects.equals(dep, employee.dep); + } + + @Override + public int hashCode() { + return Objects.hash(id, dep); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java new file mode 100644 index 000000000000..1d96d6f93959 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java @@ -0,0 +1,87 @@ +/* + * 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.Map; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public enum SparkCatalogConfig { + HIVE( + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default")), + HADOOP( + "testhadoop", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "hadoop", "cache-enabled", "false")), + REST( + "testrest", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "rest", "cache-enabled", "false")), + SPARK_SESSION( + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "parquet-enabled", "true", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + )), + SPARK_WITH_VIEWS( + "spark_with_views", + SparkCatalog.class.getName(), + ImmutableMap.of( + CatalogProperties.CATALOG_IMPL, + InMemoryCatalog.class.getName(), + "default-namespace", + "default", + "cache-enabled", + "false")), + SPARK_SESSION_WITH_VIEWS( + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of("type", "rest", "default-namespace", "default", "cache-enabled", "false")); + + private final String catalogName; + private final String implementation; + private final Map properties; + + SparkCatalogConfig(String catalogName, String implementation, Map properties) { + this.catalogName = catalogName; + this.implementation = implementation; + this.properties = properties; + } + + public String catalogName() { + return catalogName; + } + + public String implementation() { + return implementation; + } + + public Map properties() { + return properties; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java new file mode 100644 index 000000000000..2754e891a481 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java @@ -0,0 +1,95 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.spark.sql.Row; +import org.apache.spark.unsafe.types.VariantVal; + +public class SparkTestHelperBase { + protected static final Object ANY = new Object(); + + protected List rowsToJava(List rows) { + return rows.stream().map(this::toJava).collect(Collectors.toList()); + } + + private Object[] toJava(Row row) { + return IntStream.range(0, row.size()) + .mapToObj( + pos -> { + if (row.isNullAt(pos)) { + return null; + } + + Object value = row.get(pos); + if (value instanceof Row) { + return toJava((Row) value); + } else if (value instanceof scala.collection.Seq) { + return row.getList(pos); + } else if (value instanceof scala.collection.Map) { + return row.getJavaMap(pos); + } else { + return value; + } + }) + .toArray(Object[]::new); + } + + protected void assertEquals( + String context, List expectedRows, List actualRows) { + assertThat(actualRows) + .as("%s: number of results should match", context) + .hasSameSizeAs(expectedRows); + for (int row = 0; row < expectedRows.size(); row += 1) { + Object[] expected = expectedRows.get(row); + Object[] actual = actualRows.get(row); + assertThat(actual).as("Number of columns should match").hasSameSizeAs(expected); + assertEquals(context + ": row " + (row + 1), expected, actual); + } + } + + protected void assertEquals(String context, Object[] expectedRow, Object[] actualRow) { + assertThat(actualRow).as("Number of columns should match").hasSameSizeAs(expectedRow); + for (int col = 0; col < actualRow.length; col += 1) { + Object expectedValue = expectedRow[col]; + Object actualValue = actualRow[col]; + if (expectedValue != null && expectedValue.getClass().isArray()) { + String newContext = String.format("%s (nested col %d)", context, col + 1); + if (expectedValue instanceof byte[]) { + assertThat(actualValue).as(newContext).isEqualTo(expectedValue); + } else { + assertEquals(newContext, (Object[]) expectedValue, (Object[]) actualValue); + } + } else if (expectedValue instanceof VariantVal && actualValue instanceof VariantVal) { + // Spark VariantVal comparison is based on raw byte[] comparison, which can fail + // if Spark uses trailing null bytes. so, we compare their JSON representation instead. + assertThat(actualValue) + .asString() + .as("%s contents should match (VariantVal JSON)", context) + .isEqualTo((expectedValue).toString()); + } else if (expectedValue != ANY) { + assertThat(actualValue).as("%s contents should match", context).isEqualTo(expectedValue); + } + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SystemFunctionPushDownHelper.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SystemFunctionPushDownHelper.java new file mode 100644 index 000000000000..059325e02a34 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SystemFunctionPushDownHelper.java @@ -0,0 +1,127 @@ +/* + * 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 org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.spark.sql.SparkSession; + +public class SystemFunctionPushDownHelper { + public static final Types.StructType STRUCT = + Types.StructType.of( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "ts", Types.TimestampType.withZone()), + Types.NestedField.optional(3, "data", Types.StringType.get())); + + private SystemFunctionPushDownHelper() {} + + public static void createUnpartitionedTable(SparkSession spark, String tableName) { + sql(spark, "CREATE TABLE %s (id BIGINT, ts TIMESTAMP, data STRING) USING iceberg", tableName); + insertRecords(spark, tableName); + } + + public static void createPartitionedTable( + SparkSession spark, String tableName, String partitionCol) { + sql( + spark, + "CREATE TABLE %s (id BIGINT, ts TIMESTAMP, data STRING) USING iceberg PARTITIONED BY (%s)", + tableName, + partitionCol); + insertRecords(spark, tableName); + } + + private static void insertRecords(SparkSession spark, String tableName) { + sql( + spark, + "ALTER TABLE %s SET TBLPROPERTIES('%s' %s)", + tableName, + "read.split.target-size", + "10"); + sql( + spark, + "INSERT INTO TABLE %s VALUES %s", + tableName, + "(0, CAST('2017-11-22T09:20:44.294658+00:00' AS TIMESTAMP), 'data-0')"); + sql( + spark, + "INSERT INTO TABLE %s VALUES %s", + tableName, + "(1, CAST('2017-11-22T07:15:34.582910+00:00' AS TIMESTAMP), 'data-1')"); + sql( + spark, + "INSERT INTO TABLE %s VALUES %s", + tableName, + "(2, CAST('2017-11-22T06:02:09.243857+00:00' AS TIMESTAMP), 'data-2')"); + sql( + spark, + "INSERT INTO TABLE %s VALUES %s", + tableName, + "(3, CAST('2017-11-22T03:10:11.134509+00:00' AS TIMESTAMP), 'data-3')"); + sql( + spark, + "INSERT INTO TABLE %s VALUES %s", + tableName, + "(4, CAST('2017-11-22T00:34:00.184671+00:00' AS TIMESTAMP), 'data-4')"); + sql( + spark, + "INSERT INTO TABLE %s VALUES %s", + tableName, + "(5, CAST('2018-12-21T22:20:08.935889+00:00' AS TIMESTAMP), 'material-5')"); + sql( + spark, + "INSERT INTO TABLE %s VALUES %s", + tableName, + "(6, CAST('2018-12-21T21:55:30.589712+00:00' AS TIMESTAMP), 'material-6')"); + sql( + spark, + "INSERT INTO TABLE %s VALUES %s", + tableName, + "(7, CAST('2018-12-21T17:31:14.532797+00:00' AS TIMESTAMP), 'material-7')"); + sql( + spark, + "INSERT INTO TABLE %s VALUES %s", + tableName, + "(8, CAST('2018-12-21T15:21:51.237521+00:00' AS TIMESTAMP), 'material-8')"); + sql( + spark, + "INSERT INTO TABLE %s VALUES %s", + tableName, + "(9, CAST('2018-12-21T15:02:15.230570+00:00' AS TIMESTAMP), 'material-9')"); + } + + public static int timestampStrToYearOrdinal(String timestamp) { + return DateTimeUtil.microsToYears(DateTimeUtil.isoTimestamptzToMicros(timestamp)); + } + + public static int timestampStrToMonthOrdinal(String timestamp) { + return DateTimeUtil.microsToMonths(DateTimeUtil.isoTimestamptzToMicros(timestamp)); + } + + public static int timestampStrToDayOrdinal(String timestamp) { + return DateTimeUtil.microsToDays(DateTimeUtil.isoTimestamptzToMicros(timestamp)); + } + + public static int timestampStrToHourOrdinal(String timestamp) { + return DateTimeUtil.microsToHours(DateTimeUtil.isoTimestamptzToMicros(timestamp)); + } + + private static void sql(SparkSession spark, String sqlFormat, Object... args) { + spark.sql(String.format(sqlFormat, args)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestBase.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestBase.java new file mode 100644 index 000000000000..daf4e29ac075 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestBase.java @@ -0,0 +1,289 @@ +/* + * 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 static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.InetAddress; +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; +import java.util.Map; +import java.util.TimeZone; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.exceptions.AlreadyExistsException; +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.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.execution.QueryExecution; +import org.apache.spark.sql.execution.SparkPlan; +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.util.QueryExecutionListener; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; + +public abstract class TestBase extends SparkTestHelperBase { + + protected static TestHiveMetastore metastore = null; + protected static HiveConf hiveConf = null; + protected static SparkSession spark = null; + protected static JavaSparkContext sparkContext = null; + protected static HiveCatalog catalog = null; + + @BeforeAll + public static void startMetastoreAndSpark() { + TestBase.metastore = new TestHiveMetastore(); + metastore.start(); + TestBase.hiveConf = metastore.hiveConf(); + + TestBase.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) + .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") + .enableHiveSupport() + .getOrCreate(); + + TestBase.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + TestBase.catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + + try { + catalog.createNamespace(Namespace.of("default")); + } catch (AlreadyExistsException ignored) { + // the default namespace already exists. ignore the create error + } + } + + @AfterAll + public static void stopMetastoreAndSpark() throws Exception { + TestBase.catalog = null; + if (metastore != null) { + metastore.stop(); + TestBase.metastore = null; + } + if (spark != null) { + spark.stop(); + TestBase.spark = null; + TestBase.sparkContext = null; + } + } + + protected long waitUntilAfter(long timestampMillis) { + long current = System.currentTimeMillis(); + while (current <= timestampMillis) { + current = System.currentTimeMillis(); + } + return current; + } + + protected List sql(String query, Object... args) { + List rows = spark.sql(String.format(query, args)).collectAsList(); + if (rows.isEmpty()) { + return ImmutableList.of(); + } + + return rowsToJava(rows); + } + + protected Object scalarSql(String query, Object... args) { + List rows = sql(query, args); + assertThat(rows).as("Scalar SQL should return one row").hasSize(1); + Object[] row = Iterables.getOnlyElement(rows); + assertThat(row).as("Scalar SQL should return one value").hasSize(1); + return row[0]; + } + + protected Object[] row(Object... values) { + return values; + } + + protected static String dbPath(String dbName) { + return metastore.getDatabasePath(dbName); + } + + protected void withUnavailableFiles(Iterable> files, Action action) { + Iterable fileLocations = Iterables.transform(files, ContentFile::location); + withUnavailableLocations(fileLocations, action); + } + + private void move(String location, String newLocation) { + Path path = Paths.get(URI.create(location)); + Path tempPath = Paths.get(URI.create(newLocation)); + + try { + Files.move(path, tempPath); + } catch (IOException e) { + throw new UncheckedIOException("Failed to move: " + location, e); + } + } + + protected void withUnavailableLocations(Iterable locations, Action action) { + for (String location : locations) { + move(location, location + "_temp"); + } + + try { + action.invoke(); + } finally { + for (String location : locations) { + move(location + "_temp", location); + } + } + } + + protected void withDefaultTimeZone(String zoneId, Action action) { + TimeZone currentZone = TimeZone.getDefault(); + try { + TimeZone.setDefault(TimeZone.getTimeZone(zoneId)); + action.invoke(); + } finally { + TimeZone.setDefault(currentZone); + } + } + + protected void withSQLConf(Map conf, Action action) { + SQLConf sqlConf = SQLConf.get(); + + Map currentConfValues = Maps.newHashMap(); + conf.keySet() + .forEach( + confKey -> { + if (sqlConf.contains(confKey)) { + String currentConfValue = sqlConf.getConfString(confKey); + currentConfValues.put(confKey, currentConfValue); + } + }); + + conf.forEach( + (confKey, confValue) -> { + if (SQLConf.isStaticConfigKey(confKey)) { + throw new RuntimeException("Cannot modify the value of a static config: " + confKey); + } + sqlConf.setConfString(confKey, confValue); + }); + + try { + action.invoke(); + } finally { + conf.forEach( + (confKey, confValue) -> { + if (currentConfValues.containsKey(confKey)) { + sqlConf.setConfString(confKey, currentConfValues.get(confKey)); + } else { + sqlConf.unsetConf(confKey); + } + }); + } + } + + protected Dataset jsonToDF(String schema, String... records) { + Dataset jsonDF = spark.createDataset(ImmutableList.copyOf(records), Encoders.STRING()); + return spark.read().schema(schema).json(jsonDF); + } + + protected void append(String table, String... jsonRecords) { + try { + String schema = spark.table(table).schema().toDDL(); + Dataset df = jsonToDF(schema, jsonRecords); + df.coalesce(1).writeTo(table).append(); + } catch (NoSuchTableException e) { + throw new RuntimeException("Failed to write data", e); + } + } + + protected String tablePropsAsString(Map tableProps) { + StringBuilder stringBuilder = new StringBuilder(); + + for (Map.Entry property : tableProps.entrySet()) { + if (stringBuilder.length() > 0) { + stringBuilder.append(", "); + } + stringBuilder.append(String.format("'%s' '%s'", property.getKey(), property.getValue())); + } + + return stringBuilder.toString(); + } + + protected SparkPlan executeAndKeepPlan(String query, Object... args) { + return executeAndKeepPlan(() -> sql(query, args)); + } + + protected SparkPlan executeAndKeepPlan(Action action) { + AtomicReference executedPlanRef = new AtomicReference<>(); + + QueryExecutionListener listener = + new QueryExecutionListener() { + @Override + public void onSuccess(String funcName, QueryExecution qe, long durationNs) { + executedPlanRef.set(qe.executedPlan()); + } + + @Override + public void onFailure(String funcName, QueryExecution qe, Exception exception) {} + }; + + spark.listenerManager().register(listener); + + action.invoke(); + + try { + spark.sparkContext().listenerBus().waitUntilEmpty(); + } catch (TimeoutException e) { + throw new RuntimeException("Timeout while waiting for processing events", e); + } + + SparkPlan executedPlan = executedPlanRef.get(); + if (executedPlan instanceof AdaptiveSparkPlanExec) { + return ((AdaptiveSparkPlanExec) executedPlan).executedPlan(); + } else { + return executedPlan; + } + } + + @FunctionalInterface + protected interface Action { + void invoke(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java new file mode 100644 index 000000000000..7df9c75fb3dd --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java @@ -0,0 +1,196 @@ +/* + * 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 static org.apache.iceberg.CatalogProperties.CATALOG_IMPL; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_REST; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.util.Map; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.rest.RESTCatalog; +import org.apache.iceberg.rest.RESTCatalogServer; +import org.apache.iceberg.rest.RESTServerExtension; +import org.apache.iceberg.util.PropertyUtil; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TestBaseWithCatalog extends TestBase { + protected static File warehouse = null; + + @RegisterExtension + private static final RESTServerExtension REST_SERVER_EXTENSION = + new RESTServerExtension( + Map.of( + RESTCatalogServer.REST_PORT, + RESTServerExtension.FREE_PORT, + // In-memory sqlite database by default is private to the connection that created it. + // If more than 1 jdbc connection backed by in-memory sqlite is created behind one + // JdbcCatalog, then different jdbc connections could provide different views of table + // status even belonging to the same catalog. Reference: + // https://www.sqlite.org/inmemorydb.html + CatalogProperties.CLIENT_POOL_SIZE, + "1")); + + protected static RESTCatalog restCatalog; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties() + }, + }; + } + + @BeforeAll + public static void setUpAll() throws IOException { + TestBaseWithCatalog.warehouse = File.createTempFile("warehouse", null); + assertThat(warehouse.delete()).isTrue(); + restCatalog = REST_SERVER_EXTENSION.client(); + } + + @AfterAll + public static void tearDownAll() throws IOException { + if (warehouse != null && warehouse.exists()) { + Path warehousePath = new Path(warehouse.getAbsolutePath()); + FileSystem fs = warehousePath.getFileSystem(hiveConf); + assertThat(fs.delete(warehousePath, true)).as("Failed to delete " + warehousePath).isTrue(); + } + } + + @TempDir protected java.nio.file.Path temp; + + @Parameter(index = 0) + protected String catalogName; + + @Parameter(index = 1) + protected String implementation; + + @Parameter(index = 2) + protected Map catalogConfig; + + protected Catalog validationCatalog; + protected SupportsNamespaces validationNamespaceCatalog; + protected TableIdentifier tableIdent = TableIdentifier.of(Namespace.of("default"), "table"); + protected String tableName; + + private void configureValidationCatalog() { + if (catalogConfig.containsKey(ICEBERG_CATALOG_TYPE)) { + String catalogType = catalogConfig.get(ICEBERG_CATALOG_TYPE); + switch (catalogType) { + case ICEBERG_CATALOG_TYPE_HADOOP: + this.validationCatalog = + new HadoopCatalog(spark.sessionState().newHadoopConf(), "file:" + warehouse); + break; + case ICEBERG_CATALOG_TYPE_REST: + this.validationCatalog = restCatalog; + break; + case ICEBERG_CATALOG_TYPE_HIVE: + this.validationCatalog = catalog; + break; + default: + throw new IllegalArgumentException("Unknown catalog type: " + catalogType); + } + } else if (catalogConfig.containsKey(CATALOG_IMPL)) { + switch (catalogConfig.get(CATALOG_IMPL)) { + case "org.apache.iceberg.inmemory.InMemoryCatalog": + this.validationCatalog = new InMemoryCatalog(); + break; + default: + throw new IllegalArgumentException("Unknown catalog impl"); + } + } + this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; + } + + @BeforeEach + public void before() { + configureValidationCatalog(); + + spark.conf().set("spark.sql.catalog." + catalogName, implementation); + catalogConfig.forEach( + (key, value) -> spark.conf().set("spark.sql.catalog." + catalogName + "." + key, value)); + + if ("hadoop".equalsIgnoreCase(catalogConfig.get("type"))) { + spark.conf().set("spark.sql.catalog." + catalogName + ".warehouse", "file:" + warehouse); + } + + this.tableName = + (catalogName.equals("spark_catalog") ? "" : catalogName + ".") + "default.table"; + + sql("CREATE NAMESPACE IF NOT EXISTS default"); + } + + protected String tableName(String name) { + return (catalogName.equals("spark_catalog") ? "" : catalogName + ".") + "default." + name; + } + + protected String commitTarget() { + return tableName; + } + + protected String selectTarget() { + return tableName; + } + + protected boolean cachingCatalogEnabled() { + return PropertyUtil.propertyAsBoolean( + catalogConfig, CatalogProperties.CACHE_ENABLED, CatalogProperties.CACHE_ENABLED_DEFAULT); + } + + protected void configurePlanningMode(PlanningMode planningMode) { + configurePlanningMode(tableName, planningMode); + } + + protected void configurePlanningMode(String table, PlanningMode planningMode) { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s', '%s' '%s')", + table, + TableProperties.DATA_PLANNING_MODE, + planningMode.modeName(), + TableProperties.DELETE_PLANNING_MODE, + planningMode.modeName()); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java new file mode 100644 index 000000000000..bd9832f7d674 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestChangelogIterator.java @@ -0,0 +1,359 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import org.apache.iceberg.ChangelogOperation; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.Test; + +public class TestChangelogIterator extends SparkTestHelperBase { + private static final String DELETE = ChangelogOperation.DELETE.name(); + private static final String INSERT = ChangelogOperation.INSERT.name(); + private static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name(); + private static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name(); + + private static final StructType SCHEMA = + new StructType( + new StructField[] { + new StructField("id", DataTypes.IntegerType, false, Metadata.empty()), + new StructField("name", DataTypes.StringType, false, Metadata.empty()), + new StructField("data", DataTypes.StringType, true, Metadata.empty()), + new StructField( + MetadataColumns.CHANGE_TYPE.name(), DataTypes.StringType, false, Metadata.empty()), + new StructField( + MetadataColumns.CHANGE_ORDINAL.name(), + DataTypes.IntegerType, + false, + Metadata.empty()), + new StructField( + MetadataColumns.COMMIT_SNAPSHOT_ID.name(), + DataTypes.LongType, + false, + Metadata.empty()) + }); + private static final String[] IDENTIFIER_FIELDS = new String[] {"id", "name"}; + + private enum RowType { + DELETED, + INSERTED, + CARRY_OVER, + UPDATED + } + + @Test + public void testIterator() { + List permutations = Lists.newArrayList(); + // generate 24 permutations + permute( + Arrays.asList(RowType.DELETED, RowType.INSERTED, RowType.CARRY_OVER, RowType.UPDATED), + 0, + permutations); + assertThat(permutations).hasSize(24); + + for (Object[] permutation : permutations) { + validate(permutation); + } + } + + private void validate(Object[] permutation) { + List rows = Lists.newArrayList(); + List expectedRows = Lists.newArrayList(); + for (int i = 0; i < permutation.length; i++) { + rows.addAll(toOriginalRows((RowType) permutation[i], i)); + expectedRows.addAll(toExpectedRows((RowType) permutation[i], i)); + } + + Iterator iterator = + ChangelogIterator.computeUpdates(rows.iterator(), SCHEMA, IDENTIFIER_FIELDS); + List result = Lists.newArrayList(iterator); + assertEquals("Rows should match", expectedRows, rowsToJava(result)); + } + + private List toOriginalRows(RowType rowType, int index) { + switch (rowType) { + case DELETED: + return Lists.newArrayList( + new GenericRowWithSchema(new Object[] {index, "b", "data", DELETE, 0, 0}, null)); + case INSERTED: + return Lists.newArrayList( + new GenericRowWithSchema(new Object[] {index, "c", "data", INSERT, 0, 0}, null)); + case CARRY_OVER: + return Lists.newArrayList( + new GenericRowWithSchema(new Object[] {index, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {index, "d", "data", INSERT, 0, 0}, null)); + case UPDATED: + return Lists.newArrayList( + new GenericRowWithSchema(new Object[] {index, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {index, "a", "new_data", INSERT, 0, 0}, null)); + default: + throw new IllegalArgumentException("Unknown row type: " + rowType); + } + } + + private List toExpectedRows(RowType rowType, int order) { + switch (rowType) { + case DELETED: + List rows = Lists.newArrayList(); + rows.add(new Object[] {order, "b", "data", DELETE, 0, 0}); + return rows; + case INSERTED: + List insertedRows = Lists.newArrayList(); + insertedRows.add(new Object[] {order, "c", "data", INSERT, 0, 0}); + return insertedRows; + case CARRY_OVER: + return Lists.newArrayList(); + case UPDATED: + return Lists.newArrayList( + new Object[] {order, "a", "data", UPDATE_BEFORE, 0, 0}, + new Object[] {order, "a", "new_data", UPDATE_AFTER, 0, 0}); + default: + throw new IllegalArgumentException("Unknown row type: " + rowType); + } + } + + private void permute(List arr, int start, List pm) { + for (int i = start; i < arr.size(); i++) { + Collections.swap(arr, i, start); + permute(arr, start + 1, pm); + Collections.swap(arr, start, i); + } + if (start == arr.size() - 1) { + pm.add(arr.toArray()); + } + } + + @Test + public void testRowsWithNullValue() { + final List rowsWithNull = + Lists.newArrayList( + new GenericRowWithSchema(new Object[] {2, null, null, DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {3, null, null, INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {4, null, null, DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {4, null, null, INSERT, 0, 0}, null), + // mixed null and non-null value in non-identifier columns + new GenericRowWithSchema(new Object[] {5, null, null, DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {5, null, "data", INSERT, 0, 0}, null), + // mixed null and non-null value in identifier columns + new GenericRowWithSchema(new Object[] {6, null, null, DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {6, "name", null, INSERT, 0, 0}, null)); + + Iterator iterator = + ChangelogIterator.computeUpdates(rowsWithNull.iterator(), SCHEMA, IDENTIFIER_FIELDS); + List result = Lists.newArrayList(iterator); + + assertEquals( + "Rows should match", + Lists.newArrayList( + new Object[] {2, null, null, DELETE, 0, 0}, + new Object[] {3, null, null, INSERT, 0, 0}, + new Object[] {5, null, null, UPDATE_BEFORE, 0, 0}, + new Object[] {5, null, "data", UPDATE_AFTER, 0, 0}, + new Object[] {6, null, null, DELETE, 0, 0}, + new Object[] {6, "name", null, INSERT, 0, 0}), + rowsToJava(result)); + } + + @Test + public void testUpdatedRowsWithDuplication() { + List rowsWithDuplication = + Lists.newArrayList( + // two rows with same identifier fields(id, name) + new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "new_data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "new_data", INSERT, 0, 0}, null)); + + Iterator iterator = + ChangelogIterator.computeUpdates(rowsWithDuplication.iterator(), SCHEMA, IDENTIFIER_FIELDS); + + assertThatThrownBy(() -> Lists.newArrayList(iterator)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot compute updates because there are multiple rows with the same identifier fields([id,name]). Please make sure the rows are unique."); + + // still allow extra insert rows + rowsWithDuplication = + Lists.newArrayList( + new GenericRowWithSchema(new Object[] {1, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "new_data1", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "new_data2", INSERT, 0, 0}, null)); + + Iterator iterator1 = + ChangelogIterator.computeUpdates(rowsWithDuplication.iterator(), SCHEMA, IDENTIFIER_FIELDS); + + assertEquals( + "Rows should match.", + Lists.newArrayList( + new Object[] {1, "a", "data", UPDATE_BEFORE, 0, 0}, + new Object[] {1, "a", "new_data1", UPDATE_AFTER, 0, 0}, + new Object[] {1, "a", "new_data2", INSERT, 0, 0}), + rowsToJava(Lists.newArrayList(iterator1))); + } + + @Test + public void testCarryRowsRemoveWithDuplicates() { + // assume rows are sorted by id and change type + List rowsWithDuplication = + Lists.newArrayList( + // keep all delete rows for id 0 and id 1 since there is no insert row for them + new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {0, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "old_data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "a", "old_data", DELETE, 0, 0}, null), + // the same number of delete and insert rows for id 2 + new GenericRowWithSchema(new Object[] {2, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {2, "a", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {2, "a", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {2, "a", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {3, "a", "new_data", INSERT, 0, 0}, null)); + + List expectedRows = + Lists.newArrayList( + new Object[] {0, "a", "data", DELETE, 0, 0}, + new Object[] {0, "a", "data", DELETE, 0, 0}, + new Object[] {0, "a", "data", DELETE, 0, 0}, + new Object[] {1, "a", "old_data", DELETE, 0, 0}, + new Object[] {1, "a", "old_data", DELETE, 0, 0}, + new Object[] {3, "a", "new_data", INSERT, 0, 0}); + + validateIterators(rowsWithDuplication, expectedRows); + } + + @Test + public void testCarryRowsRemoveLessInsertRows() { + // less insert rows than delete rows + List rowsWithDuplication = + Lists.newArrayList( + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {2, "d", "data", INSERT, 0, 0}, null)); + + List expectedRows = + Lists.newArrayList( + new Object[] {1, "d", "data", DELETE, 0, 0}, + new Object[] {2, "d", "data", INSERT, 0, 0}); + + validateIterators(rowsWithDuplication, expectedRows); + } + + @Test + public void testCarryRowsRemoveMoreInsertRows() { + List rowsWithDuplication = + Lists.newArrayList( + new GenericRowWithSchema(new Object[] {0, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + // more insert rows than delete rows, should keep extra insert rows + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null)); + + List expectedRows = + Lists.newArrayList( + new Object[] {0, "d", "data", DELETE, 0, 0}, + new Object[] {1, "d", "data", INSERT, 0, 0}); + + validateIterators(rowsWithDuplication, expectedRows); + } + + @Test + public void testCarryRowsRemoveNoInsertRows() { + // no insert row + List rowsWithDuplication = + Lists.newArrayList( + // next two rows are identical + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null)); + + List expectedRows = + Lists.newArrayList( + new Object[] {1, "d", "data", DELETE, 0, 0}, + new Object[] {1, "d", "data", DELETE, 0, 0}); + + validateIterators(rowsWithDuplication, expectedRows); + } + + private void validateIterators(List rowsWithDuplication, List expectedRows) { + Iterator iterator = + ChangelogIterator.removeCarryovers(rowsWithDuplication.iterator(), SCHEMA); + List result = Lists.newArrayList(iterator); + + assertEquals("Rows should match.", expectedRows, rowsToJava(result)); + + iterator = ChangelogIterator.removeNetCarryovers(rowsWithDuplication.iterator(), SCHEMA); + result = Lists.newArrayList(iterator); + + assertEquals("Rows should match.", expectedRows, rowsToJava(result)); + } + + @Test + public void testRemoveNetCarryovers() { + List rowsWithDuplication = + Lists.newArrayList( + // this row are different from other rows, it is a net change, should be kept + new GenericRowWithSchema(new Object[] {0, "d", "data", DELETE, 0, 0}, null), + // a pair of delete and insert rows, should be removed + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 0, 0}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 0, 0}, null), + // 2 delete rows and 2 insert rows, should be removed + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 1, 1}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 1, 1}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 1, 1}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 1, 1}, null), + // a pair of insert and delete rows across snapshots, should be removed + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 2, 2}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", DELETE, 3, 3}, null), + // extra insert rows, they are net changes, should be kept + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 4, 4}, null), + new GenericRowWithSchema(new Object[] {1, "d", "data", INSERT, 4, 4}, null), + // different key, net changes, should be kept + new GenericRowWithSchema(new Object[] {2, "d", "data", DELETE, 4, 4}, null)); + + List expectedRows = + Lists.newArrayList( + new Object[] {0, "d", "data", DELETE, 0, 0}, + new Object[] {1, "d", "data", INSERT, 4, 4}, + new Object[] {1, "d", "data", INSERT, 4, 4}, + new Object[] {2, "d", "data", DELETE, 4, 4}); + + Iterator iterator = + ChangelogIterator.removeNetCarryovers(rowsWithDuplication.iterator(), SCHEMA); + List result = Lists.newArrayList(iterator); + + assertEquals("Rows should match.", expectedRows, rowsToJava(result)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java new file mode 100644 index 000000000000..085eedf45d1d --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java @@ -0,0 +1,275 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +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.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +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.Lists; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.iceberg.util.DataFileSet; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestFileRewriteCoordinator extends CatalogTestBase { + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testBinPackRewrite() throws NoSuchTableException, IOException { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + + Dataset df = newDF(1000); + df.coalesce(1).writeTo(tableName).append(); + df.coalesce(1).writeTo(tableName).append(); + df.coalesce(1).writeTo(tableName).append(); + df.coalesce(1).writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should produce 4 snapshots").hasSize(4); + + Dataset fileDF = + spark.read().format("iceberg").load(tableName(tableIdent.name() + ".files")); + List fileSizes = fileDF.select("file_size_in_bytes").as(Encoders.LONG()).collectAsList(); + long avgFileSize = fileSizes.stream().mapToLong(i -> i).sum() / fileSizes.size(); + + try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { + String fileSetID = UUID.randomUUID().toString(); + + ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + taskSetManager.stageTasks(table, fileSetID, Lists.newArrayList(fileScanTasks)); + + // read and pack original 4 files into 2 splits + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .option(SparkReadOptions.SPLIT_SIZE, Long.toString(avgFileSize * 2)) + .option(SparkReadOptions.FILE_OPEN_COST, "0") + .load(tableName); + + // write the packed data into new files where each split becomes a new file + scanDF + .writeTo(tableName) + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) + .append(); + + // commit the rewrite + FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get(); + Set rewrittenFiles = + taskSetManager.fetchTasks(table, fileSetID).stream() + .map(t -> t.asFileScanTask().file()) + .collect(Collectors.toCollection(DataFileSet::create)); + Set addedFiles = rewriteCoordinator.fetchNewFiles(table, fileSetID); + table.newRewrite().rewriteFiles(rewrittenFiles, addedFiles).commit(); + } + + table.refresh(); + + Map summary = table.currentSnapshot().summary(); + assertThat(summary) + .containsEntry("deleted-data-files", "4") + .containsEntry("added-data-files", "2"); + + Object rowCount = scalarSql("SELECT count(*) FROM %s", tableName); + assertThat(rowCount).as("Row count must match").isEqualTo(4000L); + } + + @TestTemplate + public void testSortRewrite() throws NoSuchTableException, IOException { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + + Dataset df = newDF(1000); + df.coalesce(1).writeTo(tableName).append(); + df.coalesce(1).writeTo(tableName).append(); + df.coalesce(1).writeTo(tableName).append(); + df.coalesce(1).writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should produce 4 snapshots").hasSize(4); + + try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { + String fileSetID = UUID.randomUUID().toString(); + + ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + taskSetManager.stageTasks(table, fileSetID, Lists.newArrayList(fileScanTasks)); + + // read original 4 files as 4 splits + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .option(SparkReadOptions.SPLIT_SIZE, "134217728") + .option(SparkReadOptions.FILE_OPEN_COST, "134217728") + .load(tableName); + + // make sure we disable AQE and set the number of shuffle partitions as the target num files + ImmutableMap sqlConf = + ImmutableMap.of( + "spark.sql.shuffle.partitions", "2", + "spark.sql.adaptive.enabled", "false"); + + withSQLConf( + sqlConf, + () -> { + try { + // write new files with sorted records + scanDF + .sort("id") + .writeTo(tableName) + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) + .append(); + } catch (NoSuchTableException e) { + throw new RuntimeException("Could not replace files", e); + } + }); + + // commit the rewrite + FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get(); + Set rewrittenFiles = + taskSetManager.fetchTasks(table, fileSetID).stream() + .map(t -> t.asFileScanTask().file()) + .collect(Collectors.toCollection(DataFileSet::create)); + Set addedFiles = rewriteCoordinator.fetchNewFiles(table, fileSetID); + table.newRewrite().rewriteFiles(rewrittenFiles, addedFiles).commit(); + } + + table.refresh(); + + Map summary = table.currentSnapshot().summary(); + assertThat(summary) + .containsEntry("deleted-data-files", "4") + .containsEntry("added-data-files", "2"); + + Object rowCount = scalarSql("SELECT count(*) FROM %s", tableName); + assertThat(rowCount).as("Row count must match").isEqualTo(4000L); + } + + @TestTemplate + public void testCommitMultipleRewrites() throws NoSuchTableException, IOException { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + + Dataset df = newDF(1000); + + // add first two files + df.coalesce(1).writeTo(tableName).append(); + df.coalesce(1).writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + + String firstFileSetID = UUID.randomUUID().toString(); + long firstFileSetSnapshotId = table.currentSnapshot().snapshotId(); + + ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + + try (CloseableIterable tasks = table.newScan().planFiles()) { + // stage first 2 files for compaction + taskSetManager.stageTasks(table, firstFileSetID, Lists.newArrayList(tasks)); + } + + // add two more files + df.coalesce(1).writeTo(tableName).append(); + df.coalesce(1).writeTo(tableName).append(); + + table.refresh(); + + String secondFileSetID = UUID.randomUUID().toString(); + + try (CloseableIterable tasks = + table.newScan().appendsAfter(firstFileSetSnapshotId).planFiles()) { + // stage 2 more files for compaction + taskSetManager.stageTasks(table, secondFileSetID, Lists.newArrayList(tasks)); + } + + ImmutableSet fileSetIDs = ImmutableSet.of(firstFileSetID, secondFileSetID); + + for (String fileSetID : fileSetIDs) { + // read and pack 2 files into 1 split + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .option(SparkReadOptions.SPLIT_SIZE, Long.MAX_VALUE) + .load(tableName); + + // write the combined data as one file + scanDF + .writeTo(tableName) + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) + .append(); + } + + // commit both rewrites at the same time + FileRewriteCoordinator rewriteCoordinator = FileRewriteCoordinator.get(); + Set rewrittenFiles = + fileSetIDs.stream() + .flatMap(fileSetID -> taskSetManager.fetchTasks(table, fileSetID).stream()) + .map(t -> t.asFileScanTask().file()) + .collect(Collectors.toSet()); + Set addedFiles = + fileSetIDs.stream() + .flatMap(fileSetID -> rewriteCoordinator.fetchNewFiles(table, fileSetID).stream()) + .collect(Collectors.toCollection(DataFileSet::create)); + table.newRewrite().rewriteFiles(rewrittenFiles, addedFiles).commit(); + + table.refresh(); + + assertThat(table.snapshots()).as("Should produce 5 snapshots").hasSize(5); + + Map summary = table.currentSnapshot().summary(); + assertThat(summary) + .containsEntry("deleted-data-files", "4") + .containsEntry("added-data-files", "2"); + + Object rowCount = scalarSql("SELECT count(*) FROM %s", tableName); + assertThat(rowCount).as("Row count must match").isEqualTo(4000L); + } + + private Dataset newDF(int numRecords) { + List data = Lists.newArrayListWithExpectedSize(numRecords); + for (int index = 0; index < numRecords; index++) { + data.add(new SimpleRecord(index, Integer.toString(index))); + } + return spark.createDataFrame(data, SimpleRecord.class); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java new file mode 100644 index 000000000000..5f160bcf10f8 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java @@ -0,0 +1,136 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.IcebergBuild; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.functions.IcebergVersionFunction; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException; +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.connector.catalog.FunctionCatalog; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestFunctionCatalog extends TestBaseWithCatalog { + private static final String[] EMPTY_NAMESPACE = new String[] {}; + private static final String[] SYSTEM_NAMESPACE = new String[] {"system"}; + private static final String[] DEFAULT_NAMESPACE = new String[] {"default"}; + private static final String[] DB_NAMESPACE = new String[] {"db"}; + private FunctionCatalog asFunctionCatalog; + + @BeforeEach + public void before() { + super.before(); + this.asFunctionCatalog = castToFunctionCatalog(catalogName); + sql("CREATE NAMESPACE IF NOT EXISTS %s", catalogName + ".default"); + } + + @AfterEach + public void dropDefaultNamespace() { + sql("DROP NAMESPACE IF EXISTS %s", catalogName + ".default"); + } + + @TestTemplate + public void testListFunctionsViaCatalog() throws NoSuchNamespaceException { + assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE)) + .anyMatch(func -> "iceberg_version".equals(func.name())); + + assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE)) + .anyMatch(func -> "iceberg_version".equals(func.name())); + + assertThat(asFunctionCatalog.listFunctions(DEFAULT_NAMESPACE)) + .as("Listing functions in an existing namespace that's not system should not throw") + .isEqualTo(new Identifier[0]); + + assertThatThrownBy(() -> asFunctionCatalog.listFunctions(DB_NAMESPACE)) + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageStartingWith("[SCHEMA_NOT_FOUND] The schema `db` cannot be found."); + } + + @TestTemplate + public void testLoadFunctions() throws NoSuchFunctionException { + for (String[] namespace : ImmutableList.of(EMPTY_NAMESPACE, SYSTEM_NAMESPACE)) { + Identifier identifier = Identifier.of(namespace, "iceberg_version"); + UnboundFunction func = asFunctionCatalog.loadFunction(identifier); + + assertThat(func) + .isNotNull() + .isInstanceOf(UnboundFunction.class) + .isExactlyInstanceOf(IcebergVersionFunction.class); + } + + assertThatThrownBy( + () -> + asFunctionCatalog.loadFunction(Identifier.of(DEFAULT_NAMESPACE, "iceberg_version"))) + .isInstanceOf(NoSuchFunctionException.class) + .hasMessageStartingWith( + String.format( + "[ROUTINE_NOT_FOUND] The routine default.iceberg_version cannot be found")); + + Identifier undefinedFunction = Identifier.of(SYSTEM_NAMESPACE, "undefined_function"); + assertThatThrownBy(() -> asFunctionCatalog.loadFunction(undefinedFunction)) + .isInstanceOf(NoSuchFunctionException.class) + .hasMessageStartingWith( + String.format( + "[ROUTINE_NOT_FOUND] The routine system.undefined_function cannot be found")); + + assertThatThrownBy(() -> sql("SELECT undefined_function(1, 2)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "[UNRESOLVED_ROUTINE] Cannot resolve routine `undefined_function` on search path"); + } + + @TestTemplate + public void testCallingFunctionInSQLEndToEnd() { + String buildVersion = IcebergBuild.version(); + + assertThat(scalarSql("SELECT %s.system.iceberg_version()", catalogName)) + .as( + "Should be able to use the Iceberg version function from the fully qualified system namespace") + .isEqualTo(buildVersion); + + assertThat(scalarSql("SELECT %s.iceberg_version()", catalogName)) + .as( + "Should be able to use the Iceberg version function when fully qualified without specifying a namespace") + .isEqualTo(buildVersion); + + sql("USE %s", catalogName); + + assertThat(scalarSql("SELECT system.iceberg_version()")) + .as( + "Should be able to call iceberg_version from system namespace without fully qualified name when using Iceberg catalog") + .isEqualTo(buildVersion); + + assertThat(scalarSql("SELECT iceberg_version()")) + .as( + "Should be able to call iceberg_version from empty namespace without fully qualified name when using Iceberg catalog") + .isEqualTo(buildVersion); + } + + private FunctionCatalog castToFunctionCatalog(String name) { + return (FunctionCatalog) spark.sessionState().catalogManager().catalog(name); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java new file mode 100644 index 000000000000..e4e66abfefa0 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java @@ -0,0 +1,186 @@ +/* + * 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 static org.apache.iceberg.NullOrder.NULLS_FIRST; +import static org.apache.iceberg.NullOrder.NULLS_LAST; +import static org.apache.iceberg.expressions.Expressions.and; +import static org.apache.iceberg.expressions.Expressions.bucket; +import static org.apache.iceberg.expressions.Expressions.day; +import static org.apache.iceberg.expressions.Expressions.equal; +import static org.apache.iceberg.expressions.Expressions.greaterThan; +import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.hour; +import static org.apache.iceberg.expressions.Expressions.in; +import static org.apache.iceberg.expressions.Expressions.lessThan; +import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.apache.iceberg.expressions.Expressions.month; +import static org.apache.iceberg.expressions.Expressions.notIn; +import static org.apache.iceberg.expressions.Expressions.truncate; +import static org.apache.iceberg.expressions.Expressions.year; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.CachingCatalog; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderParser; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSpark3Util extends TestBase { + @Test + public void testDescribeSortOrder() { + Schema schema = + new Schema( + required(1, "data", Types.StringType.get()), + required(2, "time", Types.TimestampType.withoutZone())); + + assertThat(Spark3Util.describe(buildSortOrder("Identity", schema, 1))) + .as("Sort order isn't correct.") + .isEqualTo("data DESC NULLS FIRST"); + + assertThat(Spark3Util.describe(buildSortOrder("bucket[1]", schema, 1))) + .as("Sort order isn't correct.") + .isEqualTo("bucket(1, data) DESC NULLS FIRST"); + + assertThat(Spark3Util.describe(buildSortOrder("truncate[3]", schema, 1))) + .as("Sort order isn't correct.") + .isEqualTo("truncate(data, 3) DESC NULLS FIRST"); + + assertThat(Spark3Util.describe(buildSortOrder("year", schema, 2))) + .as("Sort order isn't correct.") + .isEqualTo("years(time) DESC NULLS FIRST"); + + assertThat(Spark3Util.describe(buildSortOrder("month", schema, 2))) + .as("Sort order isn't correct.") + .isEqualTo("months(time) DESC NULLS FIRST"); + + assertThat(Spark3Util.describe(buildSortOrder("day", schema, 2))) + .as("Sort order isn't correct.") + .isEqualTo("days(time) DESC NULLS FIRST"); + + assertThat(Spark3Util.describe(buildSortOrder("hour", schema, 2))) + .as("Sort order isn't correct.") + .isEqualTo("hours(time) DESC NULLS FIRST"); + + assertThat(Spark3Util.describe(buildSortOrder("unknown", schema, 1))) + .as("Sort order isn't correct.") + .isEqualTo("unknown(data) DESC NULLS FIRST"); + + // multiple sort orders + SortOrder multiOrder = + SortOrder.builderFor(schema).asc("time", NULLS_FIRST).asc("data", NULLS_LAST).build(); + assertThat(Spark3Util.describe(multiOrder)) + .as("Sort order isn't correct.") + .isEqualTo("time ASC NULLS FIRST, data ASC NULLS LAST"); + } + + @Test + public void testDescribeSchema() { + Schema schema = + new Schema( + required(1, "data", Types.ListType.ofRequired(2, Types.StringType.get())), + optional( + 3, + "pairs", + Types.MapType.ofOptional(4, 5, Types.StringType.get(), Types.LongType.get())), + required(6, "time", Types.TimestampType.withoutZone()), + required(7, "v", Types.VariantType.get())); + + assertThat(Spark3Util.describe(schema)) + .as("Schema description isn't correct.") + .isEqualTo( + "struct not null,pairs: map,time: timestamp not null,v: variant not null>"); + } + + @Test + public void testLoadIcebergTable() throws Exception { + spark.conf().set("spark.sql.catalog.hive", SparkCatalog.class.getName()); + spark.conf().set("spark.sql.catalog.hive.type", "hive"); + spark.conf().set("spark.sql.catalog.hive.default-namespace", "default"); + + String tableFullName = "hive.default.tbl"; + sql("CREATE TABLE %s (c1 bigint, c2 string, c3 string) USING iceberg", tableFullName); + + Table table = Spark3Util.loadIcebergTable(spark, tableFullName); + assertThat(table.name()).isEqualTo(tableFullName); + } + + @Test + public void testLoadIcebergCatalog() throws Exception { + spark.conf().set("spark.sql.catalog.test_cat", SparkCatalog.class.getName()); + spark.conf().set("spark.sql.catalog.test_cat.type", "hive"); + Catalog catalog = Spark3Util.loadIcebergCatalog(spark, "test_cat"); + assertThat(catalog) + .as("Should retrieve underlying catalog class") + .isInstanceOf(CachingCatalog.class); + } + + @Test + public void testDescribeExpression() { + Expression refExpression = equal("id", 1); + assertThat(Spark3Util.describe(refExpression)).isEqualTo("id = 1"); + + Expression yearExpression = greaterThan(year("ts"), 10); + assertThat(Spark3Util.describe(yearExpression)).isEqualTo("year(ts) > 10"); + + Expression monthExpression = greaterThanOrEqual(month("ts"), 10); + assertThat(Spark3Util.describe(monthExpression)).isEqualTo("month(ts) >= 10"); + + Expression dayExpression = lessThan(day("ts"), 10); + assertThat(Spark3Util.describe(dayExpression)).isEqualTo("day(ts) < 10"); + + Expression hourExpression = lessThanOrEqual(hour("ts"), 10); + assertThat(Spark3Util.describe(hourExpression)).isEqualTo("hour(ts) <= 10"); + + Expression bucketExpression = in(bucket("id", 5), 3); + assertThat(Spark3Util.describe(bucketExpression)).isEqualTo("bucket[5](id) IN (3)"); + + Expression truncateExpression = notIn(truncate("name", 3), "abc"); + assertThat(Spark3Util.describe(truncateExpression)) + .isEqualTo("truncate[3](name) NOT IN ('abc')"); + + Expression andExpression = and(refExpression, yearExpression); + assertThat(Spark3Util.describe(andExpression)).isEqualTo("(id = 1 AND year(ts) > 10)"); + } + + private SortOrder buildSortOrder(String transform, Schema schema, int sourceId) { + String jsonString = + "{\n" + + " \"order-id\" : 10,\n" + + " \"fields\" : [ {\n" + + " \"transform\" : \"" + + transform + + "\",\n" + + " \"source-id\" : " + + sourceId + + ",\n" + + " \"direction\" : \"desc\",\n" + + " \"null-order\" : \"nulls-first\"\n" + + " } ]\n" + + "}"; + + return SortOrderParser.fromJson(schema, jsonString); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCachedTableCatalog.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCachedTableCatalog.java new file mode 100644 index 000000000000..228bf43b89b1 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCachedTableCatalog.java @@ -0,0 +1,105 @@ +/* + * 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 org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkCachedTableCatalog extends TestBaseWithCatalog { + + private static final SparkTableCache TABLE_CACHE = SparkTableCache.get(); + + @BeforeAll + public static void setupCachedTableCatalog() { + spark.conf().set("spark.sql.catalog.testcache", SparkCachedTableCatalog.class.getName()); + } + + @AfterAll + public static void unsetCachedTableCatalog() { + spark.conf().unset("spark.sql.catalog.testcache"); + } + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties() + }, + }; + } + + @TestTemplate + public void testTimeTravel() { + sql("CREATE TABLE %s (id INT, dep STRING) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + sql("INSERT INTO TABLE %s VALUES (1, 'hr')", tableName); + + table.refresh(); + Snapshot firstSnapshot = table.currentSnapshot(); + waitUntilAfter(firstSnapshot.timestampMillis()); + + sql("INSERT INTO TABLE %s VALUES (2, 'hr')", tableName); + + table.refresh(); + Snapshot secondSnapshot = table.currentSnapshot(); + waitUntilAfter(secondSnapshot.timestampMillis()); + + sql("INSERT INTO TABLE %s VALUES (3, 'hr')", tableName); + + table.refresh(); + + try { + TABLE_CACHE.add("key", table); + + assertEquals( + "Should have expected rows in 3rd snapshot", + ImmutableList.of(row(1, "hr"), row(2, "hr"), row(3, "hr")), + sql("SELECT * FROM testcache.key ORDER BY id")); + + assertEquals( + "Should have expected rows in 2nd snapshot", + ImmutableList.of(row(1, "hr"), row(2, "hr")), + sql( + "SELECT * FROM testcache.`key#at_timestamp_%s` ORDER BY id", + secondSnapshot.timestampMillis())); + + assertEquals( + "Should have expected rows in 1st snapshot", + ImmutableList.of(row(1, "hr")), + sql( + "SELECT * FROM testcache.`key#snapshot_id_%d` ORDER BY id", + firstSnapshot.snapshotId())); + + } finally { + TABLE_CACHE.remove("key"); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java new file mode 100644 index 000000000000..c71fc5954cc8 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java @@ -0,0 +1,196 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.atIndex; + +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.connector.catalog.Column; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableChange; +import org.apache.spark.sql.types.DataTypes; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkCatalogOperations extends CatalogTestBase { + private static final boolean USE_NULLABLE_QUERY_SCHEMA = + ThreadLocalRandom.current().nextBoolean(); + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "use-nullable-query-schema", Boolean.toString(USE_NULLABLE_QUERY_SCHEMA)) + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + ImmutableMap.of( + "type", + "hadoop", + "cache-enabled", + "false", + "use-nullable-query-schema", + Boolean.toString(USE_NULLABLE_QUERY_SCHEMA)) + }, + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + ImmutableMap.of( + "type", + "hive", + "default-namespace", + "default", + "parquet-enabled", + "true", + "cache-enabled", + "false", // Spark will delete tables using v1, leaving the cache out of sync + "use-nullable-query-schema", + Boolean.toString(USE_NULLABLE_QUERY_SCHEMA)), + } + }; + } + + @BeforeEach + public void createTable() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testAlterTable() throws NoSuchTableException { + BaseCatalog catalog = (BaseCatalog) spark.sessionState().catalogManager().catalog(catalogName); + Identifier identifier = Identifier.of(tableIdent.namespace().levels(), tableIdent.name()); + + String fieldName = "location"; + String propsKey = "note"; + String propsValue = "jazz"; + Table table = + catalog.alterTable( + identifier, + TableChange.addColumn(new String[] {fieldName}, DataTypes.StringType, true), + TableChange.setProperty(propsKey, propsValue)); + + assertThat(table).as("Should return updated table").isNotNull(); + + Column expectedField = Column.create(fieldName, DataTypes.StringType, true); + assertThat(table.columns()) + .as("Adding a column to a table should return the updated table with the new column") + .contains(expectedField, atIndex(2)); + + assertThat(table.properties()) + .as( + "Adding a property to a table should return the updated table with the new property with the new correct value") + .containsEntry(propsKey, propsValue); + } + + @TestTemplate + public void testInvalidateTable() { + // load table to CachingCatalog + sql("SELECT count(1) FROM %s", tableName); + + // recreate table from another catalog or program + Catalog anotherCatalog = validationCatalog; + Schema schema = anotherCatalog.loadTable(tableIdent).schema(); + anotherCatalog.dropTable(tableIdent); + anotherCatalog.createTable(tableIdent, schema); + + // invalidate and reload table + sql("REFRESH TABLE %s", tableName); + sql("SELECT count(1) FROM %s", tableName); + } + + @TestTemplate + public void testCTASUseNullableQuerySchema() { + sql("INSERT INTO %s VALUES(1, 'abc'), (2, null)", tableName); + + String ctasTableName = tableName("ctas_table"); + + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", ctasTableName, tableName); + + org.apache.iceberg.Table ctasTable = + validationCatalog.loadTable(TableIdentifier.parse("default.ctas_table")); + + Schema expectedSchema = + new Schema( + USE_NULLABLE_QUERY_SCHEMA + ? Types.NestedField.optional(1, "id", Types.LongType.get()) + : Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + assertThat(ctasTable.schema().asStruct()) + .as("Should have expected schema") + .isEqualTo(expectedSchema.asStruct()); + + sql("DROP TABLE IF EXISTS %s", ctasTableName); + } + + @TestTemplate + public void testRTASUseNullableQuerySchema() { + sql("INSERT INTO %s VALUES(1, 'abc'), (2, null)", tableName); + + String rtasTableName = tableName("rtas_table"); + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", rtasTableName); + + sql("REPLACE TABLE %s USING iceberg AS SELECT * FROM %s", rtasTableName, tableName); + + org.apache.iceberg.Table rtasTable = + validationCatalog.loadTable(TableIdentifier.parse("default.rtas_table")); + + Schema expectedSchema = + new Schema( + USE_NULLABLE_QUERY_SCHEMA + ? Types.NestedField.optional(1, "id", Types.LongType.get()) + : Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + assertThat(rtasTable.schema().asStruct()) + .as("Should have expected schema") + .isEqualTo(expectedSchema.asStruct()); + + assertEquals( + "Should have rows matching the source table", + sql("SELECT * FROM %s ORDER BY id", tableName), + sql("SELECT * FROM %s ORDER BY id", rtasTableName)); + + sql("DROP TABLE IF EXISTS %s", rtasTableName); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCompressionUtil.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCompressionUtil.java new file mode 100644 index 000000000000..aa329efbbad5 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkCompressionUtil.java @@ -0,0 +1,162 @@ +/* + * 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 static org.apache.iceberg.FileFormat.AVRO; +import static org.apache.iceberg.FileFormat.METADATA; +import static org.apache.iceberg.FileFormat.ORC; +import static org.apache.iceberg.FileFormat.PARQUET; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import org.apache.iceberg.FileFormat; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; +import org.apache.spark.internal.config.package$; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class TestSparkCompressionUtil { + + private SparkSession spark; + private SparkConf sparkConf; + + @BeforeEach + public void initSpark() { + this.spark = mock(SparkSession.class); + this.sparkConf = mock(SparkConf.class); + + SparkContext sparkContext = mock(SparkContext.class); + + when(spark.sparkContext()).thenReturn(sparkContext); + when(sparkContext.conf()).thenReturn(sparkConf); + } + + @Test + public void testParquetCompressionRatios() { + configureShuffle("lz4", true); + + double ratio1 = shuffleCompressionRatio(PARQUET, "zstd"); + assertThat(ratio1).isEqualTo(3.0); + + double ratio2 = shuffleCompressionRatio(PARQUET, "gzip"); + assertThat(ratio2).isEqualTo(3.0); + + double ratio3 = shuffleCompressionRatio(PARQUET, "snappy"); + assertThat(ratio3).isEqualTo(2.0); + } + + @Test + public void testOrcCompressionRatios() { + configureShuffle("lz4", true); + + double ratio1 = shuffleCompressionRatio(ORC, "zlib"); + assertThat(ratio1).isEqualTo(3.0); + + double ratio2 = shuffleCompressionRatio(ORC, "lz4"); + assertThat(ratio2).isEqualTo(2.0); + } + + @Test + public void testAvroCompressionRatios() { + configureShuffle("lz4", true); + + double ratio1 = shuffleCompressionRatio(AVRO, "gzip"); + assertThat(ratio1).isEqualTo(1.5); + + double ratio2 = shuffleCompressionRatio(AVRO, "zstd"); + assertThat(ratio2).isEqualTo(1.5); + } + + @Test + public void testCodecNameNormalization() { + configureShuffle("zStD", true); + double ratio = shuffleCompressionRatio(PARQUET, "ZstD"); + assertThat(ratio).isEqualTo(2.0); + } + + @Test + public void testUnknownCodecNames() { + configureShuffle("SOME_SPARK_CODEC", true); + + double ratio1 = shuffleCompressionRatio(PARQUET, "SOME_PARQUET_CODEC"); + assertThat(ratio1).isEqualTo(2.0); + + double ratio2 = shuffleCompressionRatio(ORC, "SOME_ORC_CODEC"); + assertThat(ratio2).isEqualTo(2.0); + + double ratio3 = shuffleCompressionRatio(AVRO, "SOME_AVRO_CODEC"); + assertThat(ratio3).isEqualTo(1.0); + } + + @Test + public void testOtherFileFormats() { + configureShuffle("lz4", true); + double ratio = shuffleCompressionRatio(METADATA, "zstd"); + assertThat(ratio).isEqualTo(1.0); + } + + @Test + public void testNullFileCodec() { + configureShuffle("lz4", true); + + double ratio1 = shuffleCompressionRatio(PARQUET, null); + assertThat(ratio1).isEqualTo(2.0); + + double ratio2 = shuffleCompressionRatio(ORC, null); + assertThat(ratio2).isEqualTo(2.0); + + double ratio3 = shuffleCompressionRatio(AVRO, null); + assertThat(ratio3).isEqualTo(1.0); + } + + @Test + public void testUncompressedShuffles() { + configureShuffle("zstd", false); + + double ratio1 = shuffleCompressionRatio(PARQUET, "zstd"); + assertThat(ratio1).isEqualTo(4.0); + + double ratio2 = shuffleCompressionRatio(ORC, "zlib"); + assertThat(ratio2).isEqualTo(4.0); + + double ratio3 = shuffleCompressionRatio(AVRO, "gzip"); + assertThat(ratio3).isEqualTo(2.0); + } + + @Test + public void testSparkDefaults() { + assertThat(package$.MODULE$.SHUFFLE_COMPRESS().defaultValueString()).isEqualTo("true"); + assertThat(package$.MODULE$.IO_COMPRESSION_CODEC().defaultValueString()).isEqualTo("lz4"); + } + + private void configureShuffle(String codec, boolean compress) { + when(sparkConf.getBoolean(eq("spark.shuffle.compress"), anyBoolean())).thenReturn(compress); + when(sparkConf.get(eq("spark.io.compression.codec"), anyString())).thenReturn(codec); + } + + private double shuffleCompressionRatio(FileFormat fileFormat, String codec) { + return SparkCompressionUtil.shuffleCompressionRatio(spark, fileFormat, codec); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java new file mode 100644 index 000000000000..ca86350346cd --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java @@ -0,0 +1,3029 @@ +/* + * 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 static org.apache.iceberg.TableProperties.DELETE_DISTRIBUTION_MODE; +import static org.apache.iceberg.TableProperties.MERGE_DISTRIBUTION_MODE; +import static org.apache.iceberg.TableProperties.SPARK_WRITE_PARTITIONED_FANOUT_ENABLED; +import static org.apache.iceberg.TableProperties.UPDATE_DISTRIBUTION_MODE; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE; +import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE; +import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.MERGE; +import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.UPDATE; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.spark.sql.connector.distributions.Distribution; +import org.apache.spark.sql.connector.distributions.Distributions; +import org.apache.spark.sql.connector.expressions.Expression; +import org.apache.spark.sql.connector.expressions.Expressions; +import org.apache.spark.sql.connector.expressions.SortDirection; +import org.apache.spark.sql.connector.expressions.SortOrder; +import org.apache.spark.sql.connector.write.RowLevelOperation.Command; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkDistributionAndOrderingUtil extends TestBaseWithCatalog { + + private static final Distribution UNSPECIFIED_DISTRIBUTION = Distributions.unspecified(); + private static final Distribution FILE_CLUSTERED_DISTRIBUTION = + Distributions.clustered( + new Expression[] {Expressions.column(MetadataColumns.FILE_PATH.name())}); + private static final Distribution SPEC_ID_PARTITION_CLUSTERED_DISTRIBUTION = + Distributions.clustered( + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME) + }); + private static final Distribution SPEC_ID_PARTITION_FILE_CLUSTERED_DISTRIBUTION = + Distributions.clustered( + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), + Expressions.column(MetadataColumns.FILE_PATH.name()) + }); + + private static final SortOrder[] EMPTY_ORDERING = new SortOrder[] {}; + private static final SortOrder[] FILE_POSITION_ORDERING = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING) + }; + private static final SortOrder[] SPEC_ID_PARTITION_ORDERING = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING) + }; + private static final SortOrder[] SPEC_ID_PARTITION_FILE_ORDERING = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING) + }; + private static final SortOrder[] SPEC_ID_PARTITION_FILE_POSITION_ORDERING = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING) + }; + + @AfterEach + public void dropTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + // ============================================================= + // Distribution and ordering for write operations such as APPEND + // ============================================================= + // + // UNPARTITIONED UNORDERED + // ------------------------------------------------------------------------- + // write mode is NOT SET -> unspecified distribution + empty ordering + // write mode is NONE -> unspecified distribution + empty ordering + // write mode is HASH -> unspecified distribution + empty ordering + // write mode is RANGE -> unspecified distribution + empty ordering + // + // UNPARTITIONED ORDERED BY id, data + // ------------------------------------------------------------------------- + // write mode is NOT SET -> ORDER BY id, data + // write mode is NONE -> unspecified distribution + LOCALLY ORDER BY id, data + // write mode is HASH -> unspecified distribution + LOCALLY ORDER BY id, data + // write mode is RANGE -> ORDER BY id, data + // + // PARTITIONED BY date, days(ts) UNORDERED + // ------------------------------------------------------------------------- + // write mode is NOT SET -> CLUSTER BY date, days(ts) + LOCALLY ORDER BY date, days(ts) + // write mode is NOT SET (fanout) -> CLUSTER BY date, days(ts) + empty ordering + // write mode is NONE -> unspecified distribution + LOCALLY ORDERED BY date, days(ts) + // write mode is NONE (fanout) -> unspecified distribution + empty ordering + // write mode is HASH -> CLUSTER BY date, days(ts) + LOCALLY ORDER BY date, days(ts) + // write mode is HASH (fanout) -> CLUSTER BY date, days(ts) + empty ordering + // write mode is RANGE -> ORDER BY date, days(ts) + // write mode is RANGE (fanout) -> RANGE DISTRIBUTE BY date, days(ts) + empty ordering + // + // PARTITIONED BY date ORDERED BY id + // ------------------------------------------------------------------------- + // write mode is NOT SET -> ORDER BY date, id + // write mode is NONE -> unspecified distribution + LOCALLY ORDERED BY date, id + // write mode is HASH -> CLUSTER BY date + LOCALLY ORDER BY date, id + // write mode is RANGE -> ORDER BY date, id + + @TestTemplate + public void testDefaultWriteUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + checkWriteDistributionAndOrdering(table, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashWriteUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + checkWriteDistributionAndOrdering(table, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangeWriteUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + checkWriteDistributionAndOrdering(table, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testDefaultWriteUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testHashWriteUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkWriteDistributionAndOrdering(table, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testRangeWriteUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testDefaultWritePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] {Expressions.identity("date"), Expressions.days("ts")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkWriteDistributionAndOrdering(table, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashWritePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] {Expressions.identity("date"), Expressions.days("ts")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkWriteDistributionAndOrdering(table, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangeWritePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + disableFanoutWriters(table); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkWriteDistributionAndOrdering(table, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testDefaultWritePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().desc("id").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.DESCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testHashWritePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + table.replaceSortOrder().asc("id").commit(); + + Expression[] expectedClustering = + new Expression[] {Expressions.identity("date"), Expressions.bucket(8, "data")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testRangeWritePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").commit(); + + disableFanoutWriters(table); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkWriteDistributionAndOrdering(table, expectedDistribution, expectedOrdering); + } + + // ============================================================= + // Distribution and ordering for copy-on-write DELETE operations + // ============================================================= + // + // UNPARTITIONED UNORDERED + // ------------------------------------------------------------------------- + // delete mode is NOT SET -> CLUSTER BY _file + empty ordering + // delete mode is NONE -> unspecified distribution + empty ordering + // delete mode is HASH -> CLUSTER BY _file + empty ordering + // delete mode is RANGE -> RANGE DISTRIBUTE BY _file, _pos + empty ordering + // + // UNPARTITIONED ORDERED BY id, data + // ------------------------------------------------------------------------- + // delete mode is NOT SET -> CLUSTER BY _file + LOCALLY ORDER BY id, data + // delete mode is NONE -> unspecified distribution + LOCALLY ORDER BY id, data + // delete mode is HASH -> CLUSTER BY _file + LOCALLY ORDER BY id, data + // delete mode is RANGE -> ORDER BY id, data + // + // PARTITIONED BY date, days(ts) UNORDERED + // ------------------------------------------------------------------------- + // delete mode is NOT SET -> CLUSTER BY date, days(ts) + LOCALLY ORDER BY date, days(ts) + // delete mode is NOT SET (fanout) -> CLUSTER BY date, days(ts) + empty ordering + // delete mode is NONE -> unspecified distribution + LOCALLY ORDERED BY date, days(ts) + // delete mode is NONE (fanout) -> unspecified distribution + empty ordering + // delete mode is HASH -> CLUSTER BY date, days(ts) + LOCALLY ORDER BY date, days(ts) + // delete mode is HASH (fanout) -> CLUSTER BY date, days(ts) + empty ordering + // delete mode is RANGE -> ORDER BY date, days(ts) + // delete mode is RANGE (fanout) -> RANGE DISTRIBUTE BY date, days(ts) + empty ordering + // + // PARTITIONED BY date ORDERED BY id + // ------------------------------------------------------------------------- + // delete mode is NOT SET -> CLUSTER BY date + LOCALLY ORDER BY date, id + // delete mode is NONE -> unspecified distribution + LOCALLY ORDERED BY date, id + // delete mode is HASH -> CLUSTER BY date + LOCALLY ORDER BY date, id + // delete mode is RANGE -> ORDER BY date, id + + @TestTemplate + public void testDefaultCopyOnWriteDeleteUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + checkCopyOnWriteDistributionAndOrdering( + table, DELETE, FILE_CLUSTERED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testNoneCopyOnWriteDeleteUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + checkCopyOnWriteDistributionAndOrdering( + table, DELETE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashCopyOnWriteDeleteUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + checkCopyOnWriteDistributionAndOrdering( + table, DELETE, FILE_CLUSTERED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangeCopyOnWriteDeleteUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + Distribution expectedDistribution = Distributions.ordered(FILE_POSITION_ORDERING); + + checkCopyOnWriteDistributionAndOrdering(table, DELETE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testDefaultCopyOnWriteDeleteUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, DELETE, FILE_CLUSTERED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testNoneCopyOnWriteDeleteUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, DELETE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testHashCopyOnWriteDeleteUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, DELETE, FILE_CLUSTERED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testRangeCopyOnWriteDeleteUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkCopyOnWriteDistributionAndOrdering(table, DELETE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testDefaultCopyOnWriteDeletePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] {Expressions.identity("date"), Expressions.days("ts")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering(table, DELETE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkCopyOnWriteDistributionAndOrdering(table, DELETE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testNoneCopyOnWriteDeletePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + disableFanoutWriters(table); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, DELETE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + + enableFanoutWriters(table); + + checkCopyOnWriteDistributionAndOrdering( + table, DELETE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashCopyOnWriteDeletePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] {Expressions.identity("date"), Expressions.days("ts")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering(table, DELETE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkCopyOnWriteDistributionAndOrdering(table, DELETE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangeCopyOnWriteDeletePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + disableFanoutWriters(table); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkCopyOnWriteDistributionAndOrdering(table, DELETE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkCopyOnWriteDistributionAndOrdering(table, DELETE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testDefaultCopyOnWriteDeletePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().desc("id").commit(); + + Expression[] expectedClustering = new Expression[] {Expressions.identity("date")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.DESCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering(table, DELETE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testNoneCopyOnWriteDeletePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + table.replaceSortOrder().desc("id").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.DESCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, DELETE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testHashCopyOnWriteDeletePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + table.replaceSortOrder().asc("id").commit(); + + Expression[] expectedClustering = + new Expression[] {Expressions.identity("date"), Expressions.bucket(8, "data")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering(table, DELETE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testRangeCopyOnWriteDeletePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + table.replaceSortOrder().asc("id").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkCopyOnWriteDistributionAndOrdering(table, DELETE, expectedDistribution, expectedOrdering); + } + + // ============================================================= + // Distribution and ordering for copy-on-write UPDATE operations + // ============================================================= + // + // UNPARTITIONED UNORDERED + // ------------------------------------------------------------------------- + // update mode is NOT SET -> CLUSTER BY _file + empty ordering + // update mode is NONE -> unspecified distribution + empty ordering + // update mode is HASH -> CLUSTER BY _file + empty ordering + // update mode is RANGE -> RANGE DISTRIBUTE BY _file, _pos + empty ordering + // + // UNPARTITIONED ORDERED BY id, data + // ------------------------------------------------------------------------- + // update mode is NOT SET -> CLUSTER BY _file + LOCALLY ORDER BY id, data + // update mode is NONE -> unspecified distribution + LOCALLY ORDER BY id, data + // update mode is HASH -> CLUSTER BY _file + LOCALLY ORDER BY id, data + // update mode is RANGE -> ORDER BY id, data + // + // PARTITIONED BY date, days(ts) UNORDERED + // ------------------------------------------------------------------------- + // update mode is NOT SET -> CLUSTER BY date, days(ts) + LOCALLY ORDER BY date, days(ts) + // update mode is NOT SET (fanout) -> CLUSTER BY _file + empty ordering + // update mode is NONE -> unspecified distribution + LOCALLY ORDERED BY date, days(ts) + // update mode is NONE (fanout) -> unspecified distribution + empty ordering + // update mode is HASH -> CLUSTER BY date, days(ts) + LOCALLY ORDER BY date, days(ts) + // update mode is HASH (fanout) -> CLUSTER BY date, days(ts) + empty ordering + // update mode is RANGE -> ORDER BY date, days(ts) + // update mode is RANGE (fanout) -> RANGE DISTRIBUTED BY date, days(ts) + empty ordering + // + // PARTITIONED BY date ORDERED BY id + // ------------------------------------------------------------------------- + // update mode is NOT SET -> CLUSTER BY date + LOCALLY ORDER BY date, id + // update mode is NONE -> unspecified distribution + LOCALLY ORDERED BY date, id + // update mode is HASH -> CLUSTER BY date + LOCALLY ORDER BY date, id + // update mode is RANGE -> ORDER BY date, id + + @TestTemplate + public void testDefaultCopyOnWriteUpdateUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + checkCopyOnWriteDistributionAndOrdering( + table, UPDATE, FILE_CLUSTERED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testNoneCopyOnWriteUpdateUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + checkCopyOnWriteDistributionAndOrdering( + table, UPDATE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashCopyOnWriteUpdateUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + checkCopyOnWriteDistributionAndOrdering( + table, UPDATE, FILE_CLUSTERED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangeCopyOnWriteUpdateUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + Distribution expectedDistribution = Distributions.ordered(FILE_POSITION_ORDERING); + + checkCopyOnWriteDistributionAndOrdering(table, UPDATE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testDefaultCopyOnWriteUpdateUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, UPDATE, FILE_CLUSTERED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testNoneCopyOnWriteUpdateUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, UPDATE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testHashCopyOnWriteUpdateUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, UPDATE, FILE_CLUSTERED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testRangeCopyOnWriteUpdateUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkCopyOnWriteDistributionAndOrdering(table, UPDATE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testDefaultCopyOnWriteUpdatePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] {Expressions.identity("date"), Expressions.days("ts")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering(table, UPDATE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkCopyOnWriteDistributionAndOrdering(table, UPDATE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testNoneCopyOnWriteUpdatePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + disableFanoutWriters(table); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, UPDATE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + + enableFanoutWriters(table); + + checkCopyOnWriteDistributionAndOrdering( + table, UPDATE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashCopyOnWriteUpdatePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] {Expressions.identity("date"), Expressions.days("ts")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering(table, UPDATE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkCopyOnWriteDistributionAndOrdering(table, UPDATE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangeCopyOnWriteUpdatePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + disableFanoutWriters(table); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkCopyOnWriteDistributionAndOrdering(table, UPDATE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkCopyOnWriteDistributionAndOrdering(table, UPDATE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testDefaultCopyOnWriteUpdatePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().desc("id").commit(); + + Expression[] expectedClustering = new Expression[] {Expressions.identity("date")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.DESCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering(table, UPDATE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testNoneCopyOnWriteUpdatePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + table.replaceSortOrder().desc("id").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.DESCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, UPDATE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testHashCopyOnWriteUpdatePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + table.replaceSortOrder().asc("id").commit(); + + Expression[] expectedClustering = + new Expression[] {Expressions.identity("date"), Expressions.bucket(8, "data")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering(table, UPDATE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testRangeCopyOnWriteUpdatePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + table.replaceSortOrder().asc("id").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkCopyOnWriteDistributionAndOrdering(table, UPDATE, expectedDistribution, expectedOrdering); + } + + // ============================================================= + // Distribution and ordering for copy-on-write MERGE operations + // ============================================================= + // + // UNPARTITIONED UNORDERED + // ------------------------------------------------------------------------- + // merge mode is NOT SET -> use write distribution and ordering + // merge mode is NONE -> unspecified distribution + empty ordering + // merge mode is HASH -> unspecified distribution + empty ordering + // merge mode is RANGE -> unspecified distribution + empty ordering + // + // UNPARTITIONED ORDERED BY id, data + // ------------------------------------------------------------------------- + // merge mode is NOT SET -> use write distribution and ordering + // merge mode is NONE -> unspecified distribution + LOCALLY ORDER BY id, data + // merge mode is HASH -> unspecified distribution + LOCALLY ORDER BY id, data + // merge mode is RANGE -> ORDER BY id, data + // + // PARTITIONED BY date, days(ts) UNORDERED + // ------------------------------------------------------------------------- + // merge mode is NOT SET -> CLUSTER BY date, days(ts) + LOCALLY ORDER BY date, days(ts) + // merge mode is NOT SET (fanout) -> CLUSTER BY date, days(ts) + empty ordering + // merge mode is NONE -> unspecified distribution + LOCALLY ORDERED BY date, days(ts) + // merge mode is NONE (fanout) -> unspecified distribution + empty ordering + // merge mode is HASH -> CLUSTER BY date, days(ts) + LOCALLY ORDER BY date, days(ts) + // merge mode is HASH (fanout) -> CLUSTER BY date, days(ts) + empty ordering + // merge mode is RANGE -> ORDER BY date, days(ts) + // merge mode is RANGE (fanout) -> RANGE DISTRIBUTE BY date, days(ts) + empty ordering + // + // PARTITIONED BY date ORDERED BY id + // ------------------------------------------------------------------------- + // merge mode is NOT SET -> CLUSTER BY date + LOCALLY ORDER BY date, id + // merge mode is NONE -> unspecified distribution + LOCALLY ORDERED BY date, id + // merge mode is HASH -> CLUSTER BY date + LOCALLY ORDER BY date, id + // merge mode is RANGE -> ORDERED BY date, id + + @TestTemplate + public void testDefaultCopyOnWriteMergeUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testNoneCopyOnWriteMergeUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashCopyOnWriteMergeUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangeCopyOnWriteMergeUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testDefaultCopyOnWriteMergeUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testNoneCopyOnWriteMergeUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, MERGE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testHashCopyOnWriteMergeUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, MERGE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testRangeCopyOnWriteMergeUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testDefaultCopyOnWriteMergePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] {Expressions.identity("date"), Expressions.days("ts")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testNoneCopyOnWriteMergePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + disableFanoutWriters(table); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, MERGE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + + enableFanoutWriters(table); + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashCopyOnWriteMergePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] {Expressions.identity("date"), Expressions.days("ts")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangeCopyOnWriteMergePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + disableFanoutWriters(table); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testDefaultCopyOnWriteMergePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().desc("id").commit(); + + Expression[] expectedClustering = new Expression[] {Expressions.identity("date")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.DESCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testNoneCopyOnWriteMergePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + table.replaceSortOrder().desc("id").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.DESCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering( + table, MERGE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testHashCopyOnWriteMergePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + table.replaceSortOrder().asc("id").commit(); + + Expression[] expectedClustering = + new Expression[] {Expressions.identity("date"), Expressions.bucket(8, "data")}; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testRangeCopyOnWriteMergePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + table.replaceSortOrder().asc("id").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + Distribution expectedDistribution = Distributions.ordered(expectedOrdering); + + checkCopyOnWriteDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + } + + // =================================================================================== + // Distribution and ordering for merge-on-read DELETE operations with position deletes + // =================================================================================== + // + // UNPARTITIONED (ORDERED & UNORDERED) + // ------------------------------------------------------------------------- + // delete mode is NOT SET -> CLUSTER BY _spec_id, _partition, _file + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos + // delete mode is NOT SET (fanout) -> CLUSTER BY _spec_id, _partition, _file + empty ordering + // delete mode is NONE -> unspecified distribution + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos + // delete mode is NOT SET (fanout) -> CLUSTER BY _spec_id, _partition, _file + empty ordering + // delete mode is HASH -> CLUSTER BY _spec_id, _partition, _file + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos + // delete mode is HASH (fanout) -> CLUSTER BY _spec_id, _partition, _file + empty ordering + // delete mode is RANGE -> RANGE DISTRIBUTE BY _spec_id, _partition, _file + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos + // delete mode is RANGE (fanout) -> RANGE DISTRIBUTE BY _spec_id, _partition, _file + + // empty ordering + // + // PARTITIONED BY date, days(ts) (ORDERED & UNORDERED) + // ------------------------------------------------------------------------- + // delete mode is NOT SET -> CLUSTER BY _spec_id, _partition + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos + // delete mode is NOT SET (fanout) -> CLUSTER BY _spec_id, _partition + empty ordering + // delete mode is NONE -> unspecified distribution + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos + // delete mode is NONE (fanout) -> unspecified distribution + empty ordering + // delete mode is HASH -> CLUSTER BY _spec_id, _partition + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos + // delete mode is HASH (fanout) -> CLUSTER BY _spec_id, _partition + empty ordering + // delete mode is RANGE -> RANGE DISTRIBUTE BY _spec_id, _partition + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos + // delete mode is RANGE (fanout) -> RANGE DISTRIBUTE BY _spec_id, _partition + empty ordering + + @TestTemplate + public void testDefaultPositionDeltaDeleteUnpartitionedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + disableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, + DELETE, + SPEC_ID_PARTITION_FILE_CLUSTERED_DISTRIBUTION, + SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, DELETE, SPEC_ID_PARTITION_FILE_CLUSTERED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testNonePositionDeltaDeleteUnpartitionedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + disableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, DELETE, UNSPECIFIED_DISTRIBUTION, SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, DELETE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashPositionDeltaDeleteUnpartitionedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + disableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, + DELETE, + SPEC_ID_PARTITION_FILE_CLUSTERED_DISTRIBUTION, + SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, DELETE, SPEC_ID_PARTITION_FILE_CLUSTERED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangePositionDeltaDeleteUnpartitionedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + disableFanoutWriters(table); + + Distribution expectedDistribution = Distributions.ordered(SPEC_ID_PARTITION_FILE_ORDERING); + + checkPositionDeltaDistributionAndOrdering( + table, DELETE, expectedDistribution, SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering(table, DELETE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testDefaultPositionDeltaDeletePartitionedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + disableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, + DELETE, + SPEC_ID_PARTITION_CLUSTERED_DISTRIBUTION, + SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, DELETE, SPEC_ID_PARTITION_CLUSTERED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testNonePositionDeltaDeletePartitionedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + disableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, DELETE, UNSPECIFIED_DISTRIBUTION, SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, DELETE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashPositionDeltaDeletePartitionedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + disableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, + DELETE, + SPEC_ID_PARTITION_CLUSTERED_DISTRIBUTION, + SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, DELETE, SPEC_ID_PARTITION_CLUSTERED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangePositionDeltaDeletePartitionedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + disableFanoutWriters(table); + + Distribution expectedDistribution = Distributions.ordered(SPEC_ID_PARTITION_ORDERING); + + checkPositionDeltaDistributionAndOrdering( + table, DELETE, expectedDistribution, SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering(table, DELETE, expectedDistribution, EMPTY_ORDERING); + } + + // =================================================================================== + // Distribution and ordering for merge-on-read UPDATE operations with position deletes + // =================================================================================== + // + // IMPORTANT: updates are represented as delete and insert + // IMPORTANT: metadata columns like _spec_id and _partition are NULL for new insert rows + // + // UNPARTITIONED UNORDERED + // ------------------------------------------------------------------------- + // update mode is NOT SET -> CLUSTER BY _spec_id, _partition, _file + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos + // update mode is NOT SET (fanout) -> CLUSTER BY _spec_id, _partition, _file + empty ordering + // update mode is NONE -> unspecified distribution + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos + // update mode is NONE (fanout) -> unspecified distribution + empty ordering + // update mode is HASH -> CLUSTER BY _spec_id, _partition, _file + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos + // update mode is HASH (fanout) -> CLUSTER BY _spec_id, _partition, _file + empty ordering + // update mode is RANGE -> RANGE DISTRIBUTE BY _spec_id, _partition, _file + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos + // update mode is RANGE (fanout) -> RANGE DISTRIBUTE BY _spec_id, _partition, _file + empty + // ordering + // + // UNPARTITIONED ORDERED BY id, data + // ------------------------------------------------------------------------- + // update mode is NOT SET -> CLUSTER BY _spec_id, _partition, _file + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, id, data + // update mode is NONE -> unspecified distribution + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, id, data + // update mode is HASH -> CLUSTER BY _spec_id, _partition, _file + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, id, data + // update mode is RANGE -> RANGE DISTRIBUTE BY _spec_id, _partition, _file, id, data + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, id, data + // + // PARTITIONED BY date, days(ts) UNORDERED + // ------------------------------------------------------------------------- + // update mode is NOT SET -> CLUSTER BY _spec_id, _partition, date, days(ts) + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, date, days(ts) + // update mode is NOT SET (fanout) -> CLUSTER BY _spec_id, _partition, date, days(ts) + + // empty ordering + // update mode is NONE -> unspecified distribution + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, date, days(ts) + // update mode is NONE (fanout) -> unspecified distribution + empty ordering + // update mode is HASH -> CLUSTER BY _spec_id, _partition, date, days(ts) + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, date, days(ts) + // update mode is HASH (fanout) -> CLUSTER BY _spec_id, _partition, date, days(ts) + + // empty ordering + // update mode is RANGE -> RANGE DISTRIBUTE BY _spec_id, _partition, date, days(ts) + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, date, days(ts) + // update mode is RANGE (fanout) -> RANGE DISTRIBUTE BY _spec_id, _partition, date, days(ts) + + // empty ordering + // + // PARTITIONED BY date ORDERED BY id + // ------------------------------------------------------------------------- + // update mode is NOT SET -> CLUSTER BY _spec_id, _partition, date + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, date, id + // update mode is NONE -> unspecified distribution + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, date, id + // update mode is HASH -> CLUSTER BY _spec_id, _partition, date + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, date, id + // update mode is RANGE -> RANGE DISTRIBUTE BY _spec_id, _partition, date, id + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, date, id + + @TestTemplate + public void testDefaultPositionDeltaUpdateUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + disableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, + UPDATE, + SPEC_ID_PARTITION_FILE_CLUSTERED_DISTRIBUTION, + SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, SPEC_ID_PARTITION_FILE_CLUSTERED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testNonePositionDeltaUpdateUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + disableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, UNSPECIFIED_DISTRIBUTION, SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashPositionDeltaUpdateUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + disableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, + UPDATE, + SPEC_ID_PARTITION_FILE_CLUSTERED_DISTRIBUTION, + SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, SPEC_ID_PARTITION_FILE_CLUSTERED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangePositionDeltaUpdateUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + disableFanoutWriters(table); + + Distribution expectedDistribution = Distributions.ordered(SPEC_ID_PARTITION_FILE_ORDERING); + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, expectedDistribution, SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering(table, UPDATE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testDefaultPositionDeltaUpdateUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, SPEC_ID_PARTITION_FILE_CLUSTERED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testNonePositionDeltaUpdateUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testHashPositionDeltaUpdateUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, SPEC_ID_PARTITION_FILE_CLUSTERED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testRangePositionDeltaUpdateUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + SortOrder[] expectedDistributionOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + Distribution expectedDistribution = Distributions.ordered(expectedDistributionOrdering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testDefaultPositionDeltaUpdatePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), + Expressions.identity("date"), + Expressions.bucket(8, "data") + }; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering(table, UPDATE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testNonePositionDeltaUpdatePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + disableFanoutWriters(table); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashPositionDeltaUpdatePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), + Expressions.identity("date"), + Expressions.bucket(8, "data") + }; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering(table, UPDATE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangePositionDeltaUpdatePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + disableFanoutWriters(table); + + SortOrder[] expectedDistributionOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING) + }; + Distribution expectedDistribution = Distributions.ordered(expectedDistributionOrdering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering(table, UPDATE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testDefaultPositionDeltaUpdatePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").commit(); + + Expression[] expectedClustering = + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), + Expressions.identity("date"), + Expressions.bucket(8, "data") + }; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testNonePositionDeltaUpdatePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").commit(); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testHashPositionDeltaUpdatePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").commit(); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + Expression[] expectedClustering = + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), + Expressions.identity("date"), + Expressions.bucket(8, "data") + }; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testRangePositionDeltaUpdatePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").commit(); + + table.updateProperties().set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + SortOrder[] expectedDistributionOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + Distribution expectedDistribution = Distributions.ordered(expectedDistributionOrdering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, UPDATE, expectedDistribution, expectedOrdering); + } + + // ================================================================================== + // Distribution and ordering for merge-on-read MERGE operations with position deletes + // ================================================================================== + // + // IMPORTANT: updates are represented as delete and insert + // IMPORTANT: metadata columns like _spec_id and _partition are NULL for new rows + // + // UNPARTITIONED UNORDERED + // ------------------------------------------------------------------------- + // merge mode is NOT SET -> CLUSTER BY _spec_id, _partition, _file + + // LOCALLY ORDER BY _spec_id, _partition, _file, _pos + // merge mode is NOT SET (fanout) -> CLUSTER BY _spec_id, _partition, _file + empty ordering + // merge mode is NONE -> unspecified distribution + + // LOCALLY ORDER BY _spec_id, _partition, _file, _pos + // merge mode is NONE (fanout) -> unspecified distribution + empty ordering + // merge mode is HASH -> CLUSTER BY _spec_id, _partition, _file + + // LOCALLY ORDER BY _spec_id, _partition, _file, _pos + // merge mode is HASH (fanout) -> CLUSTER BY _spec_id, _partition, _file + + // empty ordering + // merge mode is RANGE -> RANGE DISTRIBUTE BY _spec_id, _partition, _file + + // LOCALLY ORDER BY _spec_id, _partition, _file, _pos + // merge mode is RANGE (fanout) -> RANGE DISTRIBUTE BY _spec_id, _partition, _file + + // empty ordering + // + // UNPARTITIONED ORDERED BY id, data + // ------------------------------------------------------------------------- + // merge mode is NOT SET -> CLUSTER BY _spec_id, _partition, _file + + // LOCALLY ORDER BY _spec_id, _partition, _file, _pos, id, data + // merge mode is NONE -> unspecified distribution + + // LOCALLY ORDER BY _spec_id, _partition, _file, _pos, id, data + // merge mode is HASH -> CLUSTER BY _spec_id, _partition, _file + + // LOCALLY ORDER BY _spec_id, _partition, _file, _pos, id, data + // merge mode is RANGE -> RANGE DISTRIBUTE BY _spec_id, _partition, _file, id, data + + // LOCALLY ORDER BY _spec_id, _partition, _file, _pos, id, data + // + // PARTITIONED BY date, days(ts) UNORDERED + // ------------------------------------------------------------------------- + // merge mode is NOT SET -> CLUSTER BY _spec_id, _partition, date, days(ts) + + // LOCALLY ORDER BY _spec_id, _partition, _file, _pos, date, days(ts) + // merge mode is NOT SET (fanout) -> CLUSTER BY _spec_id, _partition, date, days(ts) + + // empty ordering + // merge mode is NONE -> unspecified distribution + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, date, days(ts) + // merge mode is NONE (fanout) -> unspecified distribution + empty ordering + // merge mode is HASH -> CLUSTER BY _spec_id, _partition, date, days(ts) + + // LOCALLY ORDER BY _spec_id, _partition, _file, _pos, date, days(ts) + // merge mode is HASH (fanout) -> CLUSTER BY _spec_id, _partition, date, days(ts) + + // empty ordering + // merge mode is RANGE -> RANGE DISTRIBUTE BY _spec_id, _partition, date, days(ts) + + // LOCALLY ORDER BY _spec_id, _partition, _file, _pos, date, days(ts) + // merge mode is RANGE (fanout) -> RANGE DISTRIBUTE BY _spec_id, _partition, date, days(ts) + + // empty ordering + // + // PARTITIONED BY date ORDERED BY id + // ------------------------------------------------------------------------- + // merge mode is NOT SET -> CLUSTER BY _spec_id, _partition, date + + // LOCALLY ORDER BY _spec_id, _partition, _file, _pos, date, id + // merge mode is NONE -> unspecified distribution + + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, date, id + // merge mode is HASH -> CLUSTER BY _spec_id, _partition, date + + // LOCALLY ORDER BY _spec_id, _partition, _file, _pos, date, id + // merge mode is RANGE -> RANGE DISTRIBUTE BY _spec_id, _partition, date, id + // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, date, id + + @TestTemplate + public void testDefaultPositionDeltaMergeUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), + Expressions.column(MetadataColumns.FILE_PATH.name()) + }; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + checkPositionDeltaDistributionAndOrdering( + table, MERGE, expectedDistribution, SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testNonePositionDeltaMergeUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + disableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, MERGE, UNSPECIFIED_DISTRIBUTION, SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, MERGE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashPositionDeltaMergeUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), + Expressions.column(MetadataColumns.FILE_PATH.name()) + }; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + checkPositionDeltaDistributionAndOrdering( + table, MERGE, expectedDistribution, SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangePositionDeltaMergeUnpartitionedUnsortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + disableFanoutWriters(table); + + SortOrder[] expectedDistributionOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING) + }; + Distribution expectedDistribution = Distributions.ordered(expectedDistributionOrdering); + + checkPositionDeltaDistributionAndOrdering( + table, MERGE, expectedDistribution, SPEC_ID_PARTITION_FILE_POSITION_ORDERING); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testDefaultPositionDeltaMergeUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + Expression[] expectedClustering = + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), + Expressions.column(MetadataColumns.FILE_PATH.name()) + }; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testNonePositionDeltaMergeUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, MERGE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testHashPositionDeltaMergeUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + Expression[] expectedClustering = + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), + Expressions.column(MetadataColumns.FILE_PATH.name()) + }; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testRangePositionDeltaMergeUnpartitionedSortedTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + table.replaceSortOrder().asc("id").asc("data").commit(); + + SortOrder[] expectedDistributionOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + Distribution expectedDistribution = Distributions.ordered(expectedDistributionOrdering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("data"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testDefaultPositionDeltaMergePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), + Expressions.identity("date"), + Expressions.days("ts") + }; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testNonePositionDeltaMergePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + disableFanoutWriters(table); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, MERGE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering( + table, MERGE, UNSPECIFIED_DISTRIBUTION, EMPTY_ORDERING); + } + + @TestTemplate + public void testHashPositionDeltaMergePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + disableFanoutWriters(table); + + Expression[] expectedClustering = + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), + Expressions.identity("date"), + Expressions.days("ts") + }; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testRangePositionDeltaMergePartitionedUnsortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + disableFanoutWriters(table); + + SortOrder[] expectedDistributionOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + Distribution expectedDistribution = Distributions.ordered(expectedDistributionOrdering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.days("ts"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + + enableFanoutWriters(table); + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, EMPTY_ORDERING); + } + + @TestTemplate + public void testNonePositionDeltaMergePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE).commit(); + + table.replaceSortOrder().desc("id").commit(); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.DESCENDING) + }; + + checkPositionDeltaDistributionAndOrdering( + table, MERGE, UNSPECIFIED_DISTRIBUTION, expectedOrdering); + } + + @TestTemplate + public void testDefaultPositionDeltaMergePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").commit(); + + Expression[] expectedClustering = + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), + Expressions.identity("date"), + Expressions.bucket(8, "data") + }; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testHashPositionDeltaMergePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, bucket(8, data))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH).commit(); + + table.replaceSortOrder().asc("id").commit(); + + Expression[] expectedClustering = + new Expression[] { + Expressions.column(MetadataColumns.SPEC_ID.name()), + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), + Expressions.identity("date"), + Expressions.bucket(8, "data") + }; + Distribution expectedDistribution = Distributions.clustered(expectedClustering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.bucket(8, "data"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + } + + @TestTemplate + public void testRangePositionDeltaMergePartitionedSortedTable() { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE).commit(); + + table.replaceSortOrder().asc("id").commit(); + + SortOrder[] expectedDistributionOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + Distribution expectedDistribution = Distributions.ordered(expectedDistributionOrdering); + + SortOrder[] expectedOrdering = + new SortOrder[] { + Expressions.sort( + Expressions.column(MetadataColumns.SPEC_ID.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.PARTITION_COLUMN_NAME), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.FILE_PATH.name()), SortDirection.ASCENDING), + Expressions.sort( + Expressions.column(MetadataColumns.ROW_POSITION.name()), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("date"), SortDirection.ASCENDING), + Expressions.sort(Expressions.column("id"), SortDirection.ASCENDING) + }; + + checkPositionDeltaDistributionAndOrdering(table, MERGE, expectedDistribution, expectedOrdering); + } + + private void checkWriteDistributionAndOrdering( + Table table, Distribution expectedDistribution, SortOrder[] expectedOrdering) { + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + + SparkWriteRequirements requirements = writeConf.writeRequirements(); + + Distribution distribution = requirements.distribution(); + assertThat(distribution).as("Distribution must match").isEqualTo(expectedDistribution); + + SortOrder[] ordering = requirements.ordering(); + assertThat(ordering).as("Ordering must match").isEqualTo(expectedOrdering); + } + + private void checkCopyOnWriteDistributionAndOrdering( + Table table, + Command command, + Distribution expectedDistribution, + SortOrder[] expectedOrdering) { + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + + SparkWriteRequirements requirements = writeConf.copyOnWriteRequirements(command); + + Distribution distribution = requirements.distribution(); + assertThat(distribution).as("Distribution must match").isEqualTo(expectedDistribution); + + SortOrder[] ordering = requirements.ordering(); + assertThat(ordering).as("Ordering must match").isEqualTo(expectedOrdering); + } + + private void checkPositionDeltaDistributionAndOrdering( + Table table, + Command command, + Distribution expectedDistribution, + SortOrder[] expectedOrdering) { + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + + SparkWriteRequirements requirements = writeConf.positionDeltaRequirements(command); + + Distribution distribution = requirements.distribution(); + assertThat(distribution).as("Distribution must match").isEqualTo(expectedDistribution); + + SortOrder[] ordering = requirements.ordering(); + assertThat(ordering).as("Ordering must match").isEqualTo(expectedOrdering); + } + + private void disableFanoutWriters(Table table) { + table.updateProperties().set(SPARK_WRITE_PARTITIONED_FANOUT_ENABLED, "false").commit(); + } + + private void enableFanoutWriters(Table table) { + table.updateProperties().set(SPARK_WRITE_PARTITIONED_FANOUT_ENABLED, "true").commit(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java new file mode 100644 index 000000000000..a411c3fc703e --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkExecutorCache.java @@ -0,0 +1,677 @@ +/* + * 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 static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; +import static org.assertj.core.api.Assertions.assertThat; + +import com.github.benmanes.caffeine.cache.Cache; +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.ContentScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Files; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.SeekableInputStream; +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.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.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.SparkExecutorCache.CacheValue; +import org.apache.iceberg.spark.SparkExecutorCache.Conf; +import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.Pair; +import org.apache.spark.SparkEnv; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.storage.memory.MemoryStore; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestSparkExecutorCache extends TestBaseWithCatalog { + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + protected static Object[][] parameters() { + return new Object[][] { + { + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", + "hive", + CatalogProperties.FILE_IO_IMPL, + CustomFileIO.class.getName(), + "default-namespace", + "default") + }, + }; + } + + private static final String UPDATES_VIEW_NAME = "updates"; + private static final AtomicInteger JOB_COUNTER = new AtomicInteger(); + private static final Map INPUT_FILES = + Collections.synchronizedMap(Maps.newHashMap()); + + private String targetTableName; + private TableIdentifier targetTableIdent; + + @BeforeEach + public void configureTargetTableName() { + String name = "target_exec_cache_" + JOB_COUNTER.incrementAndGet(); + this.targetTableName = tableName(name); + this.targetTableIdent = TableIdentifier.of(Namespace.of("default"), name); + } + + @AfterEach + public void releaseResources() { + sql("DROP TABLE IF EXISTS %s", targetTableName); + sql("DROP TABLE IF EXISTS %s", UPDATES_VIEW_NAME); + INPUT_FILES.clear(); + } + + @TestTemplate + public void testCacheValueWeightOverflow() { + CacheValue cacheValue = new CacheValue("v", Integer.MAX_VALUE + 1L); + assertThat(cacheValue.weight()).isEqualTo(Integer.MAX_VALUE); + } + + @TestTemplate + public void testCacheEnabledConfig() { + withSQLConf( + ImmutableMap.of(SparkSQLProperties.EXECUTOR_CACHE_ENABLED, "true"), + () -> { + Conf conf = new Conf(); + assertThat(conf.cacheEnabled()).isTrue(); + }); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.EXECUTOR_CACHE_ENABLED, "false"), + () -> { + Conf conf = new Conf(); + assertThat(conf.cacheEnabled()).isFalse(); + }); + } + + @TestTemplate + public void testTimeoutConfig() { + withSQLConf( + ImmutableMap.of(SparkSQLProperties.EXECUTOR_CACHE_TIMEOUT, "10s"), + () -> { + Conf conf = new Conf(); + assertThat(conf.timeout()).hasSeconds(10); + }); + + withSQLConf( + ImmutableMap.of(SparkSQLProperties.EXECUTOR_CACHE_TIMEOUT, "2m"), + () -> { + Conf conf = new Conf(); + assertThat(conf.timeout()).hasMinutes(2); + }); + } + + @TestTemplate + public void testMaxEntrySizeConfig() { + withSQLConf( + ImmutableMap.of(SparkSQLProperties.EXECUTOR_CACHE_MAX_ENTRY_SIZE, "128"), + () -> { + Conf conf = new Conf(); + assertThat(conf.maxEntrySize()).isEqualTo(128L); + }); + } + + @TestTemplate + public void testMaxTotalSizeConfig() { + withSQLConf( + ImmutableMap.of(SparkSQLProperties.EXECUTOR_CACHE_MAX_TOTAL_SIZE, "512"), + () -> { + Conf conf = new Conf(); + assertThat(conf.maxTotalSize()).isEqualTo(512L); + }); + } + + @TestTemplate + public void testDeleteFilesCacheDisabledConfig() throws Exception { + createAndInitTable(TableProperties.DELETE_MODE, COPY_ON_WRITE); + Table table = validationCatalog.loadTable(targetTableIdent); + + withSQLConf( + ImmutableMap.of( + SparkSQLProperties.EXECUTOR_CACHE_ENABLED, "true", + SparkSQLProperties.EXECUTOR_CACHE_DELETE_FILES_ENABLED, "false"), + () -> { + SparkReadConf readConf = new SparkReadConf(spark, table, Collections.emptyMap()); + assertThat(readConf.cacheDeleteFilesOnExecutors()).isFalse(); + }); + + withSQLConf( + ImmutableMap.of( + SparkSQLProperties.EXECUTOR_CACHE_ENABLED, "true", + SparkSQLProperties.EXECUTOR_CACHE_DELETE_FILES_ENABLED, "true"), + () -> { + SparkReadConf readConf = new SparkReadConf(spark, table, Collections.emptyMap()); + assertThat(readConf.cacheDeleteFilesOnExecutors()).isTrue(); + }); + + withSQLConf( + ImmutableMap.of( + SparkSQLProperties.EXECUTOR_CACHE_ENABLED, "false", + SparkSQLProperties.EXECUTOR_CACHE_DELETE_FILES_ENABLED, "true"), + () -> { + SparkReadConf readConf = new SparkReadConf(spark, table, Collections.emptyMap()); + assertThat(readConf.cacheDeleteFilesOnExecutors()).isFalse(); + }); + } + + @TestTemplate + public void testConcurrentAccess() throws InterruptedException { + SparkExecutorCache cache = SparkExecutorCache.getOrCreate(); + + String table1 = "table1"; + String table2 = "table2"; + + Set loadedInternalKeys = Sets.newHashSet(); + + String key1 = "key1"; + String key2 = "key2"; + + long valueSize = 100L; + + int threadCount = 10; + ExecutorService executorService = Executors.newFixedThreadPool(threadCount); + + for (int threadNumber = 0; threadNumber < threadCount; threadNumber++) { + String group = threadNumber % 2 == 0 ? table1 : table2; + executorService.submit( + () -> { + for (int batch = 0; batch < 3; batch++) { + cache.getOrLoad( + group, + key1, + () -> { + String internalKey = toInternalKey(group, key1); + synchronized (loadedInternalKeys) { + // verify only one load was done for this key + assertThat(loadedInternalKeys.contains(internalKey)).isFalse(); + loadedInternalKeys.add(internalKey); + } + return "value1"; + }, + valueSize); + + cache.getOrLoad( + group, + key2, + () -> { + String internalKey = toInternalKey(group, key2); + synchronized (loadedInternalKeys) { + // verify only one load was done for this key + assertThat(loadedInternalKeys.contains(internalKey)).isFalse(); + loadedInternalKeys.add(internalKey); + } + return "value2"; + }, + valueSize); + } + }); + } + + executorService.shutdown(); + assertThat(executorService.awaitTermination(1, TimeUnit.MINUTES)).isTrue(); + + cache.invalidate(table1); + cache.invalidate(table2); + + // all keys must be invalidated + Cache state = fetchInternalCacheState(); + Set liveKeys = state.asMap().keySet(); + assertThat(liveKeys).noneMatch(key -> key.startsWith(table1) || key.startsWith(table2)); + } + + @TestTemplate + public void testCopyOnWriteDelete() throws Exception { + checkDelete(COPY_ON_WRITE); + } + + @TestTemplate + public void testMergeOnReadDelete() throws Exception { + checkDelete(MERGE_ON_READ); + } + + private void checkDelete(RowLevelOperationMode mode) throws Exception { + List deleteFiles = createAndInitTable(TableProperties.DELETE_MODE, mode); + + sql("DELETE FROM %s WHERE id = 1 OR id = 4", targetTableName); + + // there are 2 data files and 2 delete files that apply to both of them + // in CoW, the target table will be scanned 2 times (main query + runtime filter) + // the runtime filter may invalidate the cache so check at least some requests were hits + // in MoR, the target table will be scanned only once + // so each delete file must be opened once + int maxRequestCount = mode == COPY_ON_WRITE ? 3 : 1; + assertThat(deleteFiles).allMatch(deleteFile -> streamCount(deleteFile) <= maxRequestCount); + + // verify the final set of records is correct + assertEquals( + "Should have expected rows", + ImmutableList.of(), + sql("SELECT * FROM %s ORDER BY id ASC", targetTableName)); + } + + @TestTemplate + public void testCopyOnWriteUpdate() throws Exception { + checkUpdate(COPY_ON_WRITE); + } + + @TestTemplate + public void testMergeOnReadUpdate() throws Exception { + checkUpdate(MERGE_ON_READ); + } + + private void checkUpdate(RowLevelOperationMode mode) throws Exception { + List deleteFiles = createAndInitTable(TableProperties.UPDATE_MODE, mode); + + Dataset updateDS = spark.createDataset(ImmutableList.of(1, 4), Encoders.INT()); + updateDS.createOrReplaceTempView(UPDATES_VIEW_NAME); + + sql("UPDATE %s SET id = -1 WHERE id IN (SELECT * FROM %s)", targetTableName, UPDATES_VIEW_NAME); + + // there are 2 data files and 2 delete files that apply to both of them + // in CoW, the target table will be scanned 3 times (2 in main query + runtime filter) + // the runtime filter may invalidate the cache so check at least some requests were hits + // in MoR, the target table will be scanned only once + // so each delete file must be opened once + int maxRequestCount = mode == COPY_ON_WRITE ? 5 : 1; + assertThat(deleteFiles).allMatch(deleteFile -> streamCount(deleteFile) <= maxRequestCount); + + // verify the final set of records is correct + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(-1, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC", targetTableName)); + } + + @TestTemplate + public void testCopyOnWriteMerge() throws Exception { + checkMerge(COPY_ON_WRITE); + } + + @TestTemplate + public void testMergeOnReadMerge() throws Exception { + checkMerge(MERGE_ON_READ); + } + + private void checkMerge(RowLevelOperationMode mode) throws Exception { + List deleteFiles = createAndInitTable(TableProperties.MERGE_MODE, mode); + + Dataset updateDS = spark.createDataset(ImmutableList.of(1, 4), Encoders.INT()); + updateDS.createOrReplaceTempView(UPDATES_VIEW_NAME); + + sql( + "MERGE INTO %s t USING %s s " + + "ON t.id == s.value " + + "WHEN MATCHED THEN " + + " UPDATE SET id = 100 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, dep) VALUES (-1, 'unknown')", + targetTableName, UPDATES_VIEW_NAME); + + // there are 2 data files and 2 delete files that apply to both of them + // in CoW, the target table will be scanned 2 times (main query + runtime filter) + // the runtime filter may invalidate the cache so check at least some requests were hits + // in MoR, the target table will be scanned only once + // so each delete file must be opened once + int maxRequestCount = mode == COPY_ON_WRITE ? 3 : 1; + assertThat(deleteFiles).allMatch(deleteFile -> streamCount(deleteFile) <= maxRequestCount); + + // verify the final set of records is correct + assertEquals( + "Should have expected rows", + ImmutableList.of(row(100, "hr"), row(100, "hr")), + sql("SELECT * FROM %s ORDER BY id ASC", targetTableName)); + } + + @TestTemplate + public void testCopyOnWriteDeleteWithDeleteFileCacheDisabled() { + checkDeleteWithDeleteFilesCacheDisabled(COPY_ON_WRITE); + } + + @TestTemplate + public void testMergeOnReadDeleteWithDeleteFileCacheDisabled() { + checkDeleteWithDeleteFilesCacheDisabled(MERGE_ON_READ); + } + + private void checkDeleteWithDeleteFilesCacheDisabled(RowLevelOperationMode mode) { + withSQLConf( + ImmutableMap.of( + SparkSQLProperties.EXECUTOR_CACHE_ENABLED, "true", + SparkSQLProperties.EXECUTOR_CACHE_DELETE_FILES_ENABLED, "false"), + () -> { + try { + List deleteFiles = createAndInitTable(TableProperties.DELETE_MODE, mode); + + sql("DELETE FROM %s WHERE id = 1 OR id = 4", targetTableName); + + // When cache is disabled, delete files should be opened multiple times + // The cached CoW test has a maximum of 3 scans, so we expect more than that when + // disabled. + // The cached MoR test has a maximum of 1 scan, so we expect more than that when + // disabled. + int expectedMinStreamCount = mode == COPY_ON_WRITE ? 4 : 2; + assertThat(deleteFiles) + .allMatch(deleteFile -> streamCount(deleteFile) >= expectedMinStreamCount); + + assertEquals( + "Should have expected rows", + ImmutableList.of(), + sql("SELECT * FROM %s ORDER BY id ASC", targetTableName)); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + @TestTemplate + public void testCopyOnWriteUpdateWithDeleteFilesCacheDisabled() throws Exception { + checkUpdateWithDeleteFilesCacheDisabled(COPY_ON_WRITE); + } + + @TestTemplate + public void testMergeOnReadUpdateWithDeleteFilesCacheDisabled() throws Exception { + checkUpdateWithDeleteFilesCacheDisabled(MERGE_ON_READ); + } + + private void checkUpdateWithDeleteFilesCacheDisabled(RowLevelOperationMode mode) + throws Exception { + withSQLConf( + ImmutableMap.of( + SparkSQLProperties.EXECUTOR_CACHE_ENABLED, "true", + SparkSQLProperties.EXECUTOR_CACHE_DELETE_FILES_ENABLED, "false"), + () -> { + try { + List deleteFiles = createAndInitTable(TableProperties.UPDATE_MODE, mode); + + Dataset updateDS = spark.createDataset(ImmutableList.of(1, 4), Encoders.INT()); + updateDS.createOrReplaceTempView(UPDATES_VIEW_NAME); + + sql( + "UPDATE %s SET id = -1 WHERE id IN (SELECT * FROM %s)", + targetTableName, UPDATES_VIEW_NAME); + + // When cache is disabled, delete files should be opened multiple times + // Both CoW and MoR should open delete files at least 2 times without caching + int expectedMinStreamCount = 2; + assertThat(deleteFiles) + .allMatch(deleteFile -> streamCount(deleteFile) >= expectedMinStreamCount); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(-1, "hr"), row(-1, "hr")), + sql("SELECT * FROM %s", targetTableName)); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + @TestTemplate + public void testCopyOnWriteMergeWithDeleteFilesCacheDisabled() throws Exception { + checkMergeWithDeleteFilesCacheDisabled(COPY_ON_WRITE); + } + + @TestTemplate + public void testMergeOnReadMergeWithDeleteFilesCacheDisabled() throws Exception { + checkMergeWithDeleteFilesCacheDisabled(MERGE_ON_READ); + } + + private void checkMergeWithDeleteFilesCacheDisabled(RowLevelOperationMode mode) throws Exception { + withSQLConf( + ImmutableMap.of( + SparkSQLProperties.EXECUTOR_CACHE_ENABLED, "true", + SparkSQLProperties.EXECUTOR_CACHE_DELETE_FILES_ENABLED, "false"), + () -> { + try { + List deleteFiles = createAndInitTable(TableProperties.MERGE_MODE, mode); + + Dataset updateDS = spark.createDataset(ImmutableList.of(1, 4), Encoders.INT()); + updateDS.createOrReplaceTempView(UPDATES_VIEW_NAME); + + sql( + "MERGE INTO %s t USING %s s " + + "ON t.id == s.value " + + "WHEN MATCHED THEN " + + " UPDATE SET id = 100 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, dep) VALUES (-1, 'unknown')", + targetTableName, UPDATES_VIEW_NAME); + + // When the cache is disabled, delete files are opened more often because each Spark + // task reads them. + // The cached CoW MERGE test allows up to 3 scans, so we require at least 4 to confirm + // the cache is disabled. + // For MoR MERGE, the cached test allows 1 scan, so we require at least 2 to confirm the + // cache is disabled. + int expectedMinStreamCount = mode == COPY_ON_WRITE ? 4 : 2; + assertThat(deleteFiles) + .allMatch(deleteFile -> streamCount(deleteFile) >= expectedMinStreamCount); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(100, "hr"), row(100, "hr")), + sql("SELECT * FROM %s", targetTableName)); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + private int streamCount(DeleteFile deleteFile) { + CustomInputFile inputFile = INPUT_FILES.get(deleteFile.location()); + return inputFile.streamCount(); + } + + private List createAndInitTable(String operation, RowLevelOperationMode mode) + throws Exception { + sql( + "CREATE TABLE %s (id INT, dep STRING) " + + "USING iceberg " + + "TBLPROPERTIES ('%s' '%s', '%s' '%s', '%s' '%s')", + targetTableName, + TableProperties.WRITE_METADATA_LOCATION, + temp.toString().replaceFirst("file:", ""), + TableProperties.WRITE_DATA_LOCATION, + temp.toString().replaceFirst("file:", ""), + operation, + mode.modeName()); + + append(targetTableName, new Employee(0, "hr"), new Employee(1, "hr"), new Employee(2, "hr")); + append(targetTableName, new Employee(3, "hr"), new Employee(4, "hr"), new Employee(5, "hr")); + + Table table = validationCatalog.loadTable(targetTableIdent); + + List> posDeletes = + dataFiles(table).stream() + .map(dataFile -> Pair.of((CharSequence) dataFile.location(), 0L)) + .collect(Collectors.toList()); + Pair posDeleteResult = writePosDeletes(table, posDeletes); + DeleteFile posDeleteFile = posDeleteResult.first(); + CharSequenceSet referencedDataFiles = posDeleteResult.second(); + + DeleteFile eqDeleteFile = writeEqDeletes(table, "id", 2, 5); + + table + .newRowDelta() + .validateFromSnapshot(table.currentSnapshot().snapshotId()) + .validateDataFilesExist(referencedDataFiles) + .addDeletes(posDeleteFile) + .addDeletes(eqDeleteFile) + .commit(); + + sql("REFRESH TABLE %s", targetTableName); + + // invalidate the memory store to destroy all currently live table broadcasts + SparkEnv sparkEnv = SparkEnv.get(); + MemoryStore memoryStore = sparkEnv.blockManager().memoryStore(); + memoryStore.clear(); + + return ImmutableList.of(posDeleteFile, eqDeleteFile); + } + + private DeleteFile writeEqDeletes(Table table, String col, Object... values) throws IOException { + Schema deleteSchema = table.schema().select(col); + + Record delete = GenericRecord.create(deleteSchema); + List deletes = Lists.newArrayList(); + for (Object value : values) { + deletes.add(delete.copy(col, value)); + } + + OutputFile out = Files.localOutput(new File(temp.toFile(), "eq-deletes-" + UUID.randomUUID())); + return FileHelpers.writeDeleteFile(table, out, null, deletes, deleteSchema); + } + + private Pair writePosDeletes( + Table table, List> deletes) throws IOException { + OutputFile out = Files.localOutput(new File(temp.toFile(), "pos-deletes-" + UUID.randomUUID())); + return FileHelpers.writeDeleteFile(table, out, null, deletes); + } + + private void append(String target, Employee... employees) throws NoSuchTableException { + List input = Arrays.asList(employees); + Dataset inputDF = spark.createDataFrame(input, Employee.class); + inputDF.coalesce(1).writeTo(target).append(); + } + + private Collection dataFiles(Table table) { + try (CloseableIterable tasks = table.newScan().planFiles()) { + return ImmutableList.copyOf(Iterables.transform(tasks, ContentScanTask::file)); + } catch (IOException e) { + throw new RuntimeIOException(e); + } + } + + @SuppressWarnings("unchecked") + private static Cache fetchInternalCacheState() { + try { + Field stateField = SparkExecutorCache.class.getDeclaredField("state"); + stateField.setAccessible(true); + SparkExecutorCache cache = SparkExecutorCache.get(); + return (Cache) stateField.get(cache); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private static String toInternalKey(String group, String key) { + return group + "_" + key; + } + + public static class CustomFileIO implements FileIO { + + public CustomFileIO() {} + + @Override + public InputFile newInputFile(String path) { + return INPUT_FILES.computeIfAbsent(path, key -> new CustomInputFile(path)); + } + + @Override + public OutputFile newOutputFile(String path) { + return Files.localOutput(path); + } + + @Override + public void deleteFile(String path) { + File file = new File(path); + if (!file.delete()) { + throw new RuntimeIOException("Failed to delete file: " + path); + } + } + } + + public static class CustomInputFile implements InputFile { + private final InputFile delegate; + private final AtomicInteger streamCount; + + public CustomInputFile(String path) { + this.delegate = Files.localInput(path); + this.streamCount = new AtomicInteger(); + } + + @Override + public long getLength() { + return delegate.getLength(); + } + + @Override + public SeekableInputStream newStream() { + streamCount.incrementAndGet(); + return delegate.newStream(); + } + + public int streamCount() { + return streamCount.get(); + } + + @Override + public String location() { + return delegate.location(); + } + + @Override + public boolean exists() { + return delegate.exists(); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java new file mode 100644 index 000000000000..49c38b34d34a --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkFilters.java @@ -0,0 +1,201 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.sql.Date; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.temporal.ChronoUnit; +import java.util.Map; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.spark.sql.sources.And; +import org.apache.spark.sql.sources.EqualNullSafe; +import org.apache.spark.sql.sources.EqualTo; +import org.apache.spark.sql.sources.GreaterThan; +import org.apache.spark.sql.sources.GreaterThanOrEqual; +import org.apache.spark.sql.sources.In; +import org.apache.spark.sql.sources.IsNotNull; +import org.apache.spark.sql.sources.IsNull; +import org.apache.spark.sql.sources.LessThan; +import org.apache.spark.sql.sources.LessThanOrEqual; +import org.apache.spark.sql.sources.Not; +import org.junit.jupiter.api.Test; + +public class TestSparkFilters { + + @Test + public void testQuotedAttributes() { + Map attrMap = Maps.newHashMap(); + attrMap.put("id", "id"); + attrMap.put("`i.d`", "i.d"); + attrMap.put("`i``d`", "i`d"); + attrMap.put("`d`.b.`dd```", "d.b.dd`"); + attrMap.put("a.`aa```.c", "a.aa`.c"); + + attrMap.forEach( + (quoted, unquoted) -> { + IsNull isNull = IsNull.apply(quoted); + Expression expectedIsNull = Expressions.isNull(unquoted); + Expression actualIsNull = SparkFilters.convert(isNull); + assertThat(actualIsNull) + .asString() + .as("IsNull must match") + .isEqualTo(expectedIsNull.toString()); + + IsNotNull isNotNull = IsNotNull.apply(quoted); + Expression expectedIsNotNull = Expressions.notNull(unquoted); + Expression actualIsNotNull = SparkFilters.convert(isNotNull); + assertThat(actualIsNotNull) + .asString() + .as("IsNotNull must match") + .isEqualTo(expectedIsNotNull.toString()); + + LessThan lt = LessThan.apply(quoted, 1); + Expression expectedLt = Expressions.lessThan(unquoted, 1); + Expression actualLt = SparkFilters.convert(lt); + assertThat(actualLt) + .asString() + .as("LessThan must match") + .isEqualTo(expectedLt.toString()); + + LessThanOrEqual ltEq = LessThanOrEqual.apply(quoted, 1); + Expression expectedLtEq = Expressions.lessThanOrEqual(unquoted, 1); + Expression actualLtEq = SparkFilters.convert(ltEq); + assertThat(actualLtEq) + .asString() + .as("LessThanOrEqual must match") + .isEqualTo(expectedLtEq.toString()); + + GreaterThan gt = GreaterThan.apply(quoted, 1); + Expression expectedGt = Expressions.greaterThan(unquoted, 1); + Expression actualGt = SparkFilters.convert(gt); + assertThat(actualGt) + .asString() + .as("GreaterThan must match") + .isEqualTo(expectedGt.toString()); + + GreaterThanOrEqual gtEq = GreaterThanOrEqual.apply(quoted, 1); + Expression expectedGtEq = Expressions.greaterThanOrEqual(unquoted, 1); + Expression actualGtEq = SparkFilters.convert(gtEq); + assertThat(actualGtEq) + .asString() + .as("GreaterThanOrEqual must match") + .isEqualTo(expectedGtEq.toString()); + + EqualTo eq = EqualTo.apply(quoted, 1); + Expression expectedEq = Expressions.equal(unquoted, 1); + Expression actualEq = SparkFilters.convert(eq); + assertThat(actualEq).asString().as("EqualTo must match").isEqualTo(expectedEq.toString()); + + EqualNullSafe eqNullSafe = EqualNullSafe.apply(quoted, 1); + Expression expectedEqNullSafe = Expressions.equal(unquoted, 1); + Expression actualEqNullSafe = SparkFilters.convert(eqNullSafe); + assertThat(actualEqNullSafe) + .asString() + .as("EqualNullSafe must match") + .isEqualTo(expectedEqNullSafe.toString()); + + In in = In.apply(quoted, new Integer[] {1}); + Expression expectedIn = Expressions.in(unquoted, 1); + Expression actualIn = SparkFilters.convert(in); + assertThat(actualIn).asString().as("In must match").isEqualTo(expectedIn.toString()); + }); + } + + @Test + public void testTimestampFilterConversion() { + Instant instant = Instant.parse("2018-10-18T00:00:57.907Z"); + Timestamp timestamp = Timestamp.from(instant); + long epochMicros = ChronoUnit.MICROS.between(Instant.EPOCH, instant); + + Expression instantExpression = SparkFilters.convert(GreaterThan.apply("x", instant)); + Expression timestampExpression = SparkFilters.convert(GreaterThan.apply("x", timestamp)); + Expression rawExpression = Expressions.greaterThan("x", epochMicros); + + assertThat(timestampExpression) + .asString() + .as("Generated Timestamp expression should be correct") + .isEqualTo(rawExpression.toString()); + + assertThat(instantExpression) + .asString() + .as("Generated Instant expression should be correct") + .isEqualTo(rawExpression.toString()); + } + + @Test + public void testLocalDateTimeFilterConversion() { + LocalDateTime ldt = LocalDateTime.parse("2018-10-18T00:00:57"); + long epochMicros = + ChronoUnit.MICROS.between(LocalDateTime.ofInstant(Instant.EPOCH, ZoneId.of("UTC")), ldt); + + Expression instantExpression = SparkFilters.convert(GreaterThan.apply("x", ldt)); + Expression rawExpression = Expressions.greaterThan("x", epochMicros); + + assertThat(instantExpression) + .asString() + .as("Generated Instant expression should be correct") + .isEqualTo(rawExpression.toString()); + } + + @Test + public void testDateFilterConversion() { + LocalDate localDate = LocalDate.parse("2018-10-18"); + Date date = Date.valueOf(localDate); + long epochDay = localDate.toEpochDay(); + + Expression localDateExpression = SparkFilters.convert(GreaterThan.apply("x", localDate)); + Expression dateExpression = SparkFilters.convert(GreaterThan.apply("x", date)); + Expression rawExpression = Expressions.greaterThan("x", epochDay); + + assertThat(localDateExpression) + .asString() + .as("Generated localdate expression should be correct") + .isEqualTo(rawExpression.toString()); + + assertThat(dateExpression) + .asString() + .as("Generated date expression should be correct") + .isEqualTo(rawExpression.toString()); + } + + @Test + public void testNestedInInsideNot() { + Not filter = + Not.apply(And.apply(EqualTo.apply("col1", 1), In.apply("col2", new Integer[] {1, 2}))); + Expression converted = SparkFilters.convert(filter); + assertThat(converted).as("Expression should not be converted").isNull(); + } + + @Test + public void testNotIn() { + Not filter = Not.apply(In.apply("col", new Integer[] {1, 2})); + Expression actual = SparkFilters.convert(filter); + Expression expected = + Expressions.and(Expressions.notNull("col"), Expressions.notIn("col", 1, 2)); + assertThat(actual).asString().as("Expressions should match").isEqualTo(expected.toString()); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java new file mode 100644 index 000000000000..d5f407a715ef --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkSchemaUtil.java @@ -0,0 +1,272 @@ +/* + * 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 static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.TimeZone; +import java.util.stream.Stream; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.expressions.AttributeReference; +import org.apache.spark.sql.catalyst.expressions.MetadataAttribute; +import org.apache.spark.sql.catalyst.types.DataTypeUtils; +import org.apache.spark.sql.catalyst.util.ResolveDefaultColumnsUtils$; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +public class TestSparkSchemaUtil { + private static final Schema TEST_SCHEMA = + new Schema( + optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); + + private static final Schema TEST_SCHEMA_WITH_METADATA_COLS = + new Schema( + optional(1, "id", Types.IntegerType.get()), + optional(2, "data", Types.StringType.get()), + MetadataColumns.FILE_PATH, + MetadataColumns.ROW_POSITION); + + @Test + public void testEstimateSizeMaxValue() { + assertThat(SparkSchemaUtil.estimateSize(null, Long.MAX_VALUE)) + .as("estimateSize returns Long max value") + .isEqualTo(Long.MAX_VALUE); + } + + @Test + public void testEstimateSizeWithOverflow() { + long tableSize = + SparkSchemaUtil.estimateSize(SparkSchemaUtil.convert(TEST_SCHEMA), Long.MAX_VALUE - 1); + assertThat(tableSize).as("estimateSize handles overflow").isEqualTo(Long.MAX_VALUE); + } + + @Test + public void testEstimateSize() { + long tableSize = SparkSchemaUtil.estimateSize(SparkSchemaUtil.convert(TEST_SCHEMA), 1); + assertThat(tableSize).as("estimateSize matches with expected approximation").isEqualTo(24); + } + + @Test + public void testSchemaConversionWithMetaDataColumnSchema() { + StructType structType = SparkSchemaUtil.convert(TEST_SCHEMA_WITH_METADATA_COLS); + List attrRefs = + scala.collection.JavaConverters.seqAsJavaList(DataTypeUtils.toAttributes(structType)); + for (AttributeReference attrRef : attrRefs) { + if (MetadataColumns.isMetadataColumn(attrRef.name())) { + assertThat(MetadataAttribute.unapply(attrRef).isDefined()) + .as("metadata columns should have __metadata_col in attribute metadata") + .isTrue(); + } else { + assertThat(MetadataAttribute.unapply(attrRef).isDefined()) + .as("non metadata columns should not have __metadata_col in attribute metadata") + .isFalse(); + } + } + } + + @Test + public void testSchemaConversionWithOnlyWriteDefault() { + Schema schema = + new Schema( + Types.NestedField.optional("field") + .withId(1) + .ofType(Types.StringType.get()) + .withWriteDefault(Literal.of("write_only")) + .build()); + + StructType sparkSchema = SparkSchemaUtil.convert(schema); + Metadata metadata = sparkSchema.fields()[0].metadata(); + + assertThat( + metadata.contains( + ResolveDefaultColumnsUtils$.MODULE$.CURRENT_DEFAULT_COLUMN_METADATA_KEY())) + .as("Field with only write default should have CURRENT_DEFAULT metadata") + .isTrue(); + assertThat( + metadata.contains( + ResolveDefaultColumnsUtils$.MODULE$.EXISTS_DEFAULT_COLUMN_METADATA_KEY())) + .as("Field with only write default should not have EXISTS_DEFAULT metadata") + .isFalse(); + assertThat( + metadata.getString( + ResolveDefaultColumnsUtils$.MODULE$.CURRENT_DEFAULT_COLUMN_METADATA_KEY())) + .as("Spark metadata CURRENT_DEFAULT should contain correctly formatted literal") + .isEqualTo("'write_only'"); + } + + @Test + public void testSchemaConversionWithOnlyInitialDefault() { + Schema schema = + new Schema( + Types.NestedField.optional("field") + .withId(1) + .ofType(Types.IntegerType.get()) + .withInitialDefault(Literal.of(42)) + .build()); + + StructType sparkSchema = SparkSchemaUtil.convert(schema); + Metadata metadata = sparkSchema.fields()[0].metadata(); + + assertThat( + metadata.contains( + ResolveDefaultColumnsUtils$.MODULE$.CURRENT_DEFAULT_COLUMN_METADATA_KEY())) + .as("Field with only initial default should not have CURRENT_DEFAULT metadata") + .isFalse(); + assertThat( + metadata.contains( + ResolveDefaultColumnsUtils$.MODULE$.EXISTS_DEFAULT_COLUMN_METADATA_KEY())) + .as("Field with only initial default should have EXISTS_DEFAULT metadata") + .isTrue(); + assertThat( + metadata.getString( + ResolveDefaultColumnsUtils$.MODULE$.EXISTS_DEFAULT_COLUMN_METADATA_KEY())) + .as("Spark metadata EXISTS_DEFAULT should contain correctly formatted literal") + .isEqualTo("42"); + } + + @ParameterizedTest(name = "{0} with writeDefault={1}, initialDefault={2}") + @MethodSource("schemaConversionWithDefaultsTestCases") + public void testSchemaConversionWithDefaultsForPrimitiveTypes( + Type type, + Literal writeDefault, + Literal initialDefault, + String expectedCurrentDefaultValue, + String expectedExistsDefaultValue) { + TimeZone systemTimeZone = TimeZone.getDefault(); + try { + TimeZone.setDefault(TimeZone.getTimeZone("UTC")); + Schema schema = + new Schema( + Types.NestedField.optional("field") + .withId(1) + .ofType(type) + .withWriteDefault(writeDefault) + .withInitialDefault(initialDefault) + .build()); + + StructType sparkSchema = SparkSchemaUtil.convert(schema); + StructField defaultField = sparkSchema.fields()[0]; + Metadata metadata = defaultField.metadata(); + + assertThat( + metadata.contains( + ResolveDefaultColumnsUtils$.MODULE$.CURRENT_DEFAULT_COLUMN_METADATA_KEY())) + .as("Field of type %s should have CURRENT_DEFAULT metadata", type) + .isTrue(); + assertThat( + metadata.contains( + ResolveDefaultColumnsUtils$.MODULE$.EXISTS_DEFAULT_COLUMN_METADATA_KEY())) + .as("Field of type %s should have EXISTS_DEFAULT metadata", type) + .isTrue(); + assertThat( + metadata.getString( + ResolveDefaultColumnsUtils$.MODULE$.CURRENT_DEFAULT_COLUMN_METADATA_KEY())) + .as( + "Spark metadata CURRENT_DEFAULT for type %s should contain correctly formatted literal", + type) + .isEqualTo(expectedCurrentDefaultValue); + assertThat( + metadata.getString( + ResolveDefaultColumnsUtils$.MODULE$.EXISTS_DEFAULT_COLUMN_METADATA_KEY())) + .as( + "Spark metadata EXISTS_DEFAULT for type %s should contain correctly formatted literal", + type) + .isEqualTo(expectedExistsDefaultValue); + } finally { + TimeZone.setDefault(systemTimeZone); + } + } + + private static Stream schemaConversionWithDefaultsTestCases() { + return Stream.of( + Arguments.of(Types.IntegerType.get(), Literal.of(1), Literal.of(2), "1", "2"), + Arguments.of( + Types.StringType.get(), + Literal.of("write_default"), + Literal.of("initial_default"), + "'write_default'", + "'initial_default'"), + Arguments.of( + Types.UUIDType.get(), + Literal.of("f79c3e09-677c-4bbd-a479-3f349cb785e7").to(Types.UUIDType.get()), + Literal.of("f79c3e09-677c-4bbd-a479-3f349cb685e7").to(Types.UUIDType.get()), + "'f79c3e09-677c-4bbd-a479-3f349cb785e7'", + "'f79c3e09-677c-4bbd-a479-3f349cb685e7'"), + Arguments.of(Types.BooleanType.get(), Literal.of(true), Literal.of(false), "true", "false"), + Arguments.of(Types.IntegerType.get(), Literal.of(42), Literal.of(10), "42", "10"), + Arguments.of(Types.LongType.get(), Literal.of(100L), Literal.of(50L), "100L", "50L"), + Arguments.of( + Types.FloatType.get(), + Literal.of(3.14f), + Literal.of(1.5f), + "CAST('3.14' AS FLOAT)", + "CAST('1.5' AS FLOAT)"), + Arguments.of( + Types.DoubleType.get(), Literal.of(2.718), Literal.of(1.414), "2.718D", "1.414D"), + Arguments.of( + Types.DecimalType.of(10, 2), + Literal.of(new BigDecimal("99.99")), + Literal.of(new BigDecimal("11.11")), + "99.99BD", + "11.11BD"), + Arguments.of( + Types.DateType.get(), + Literal.of("2024-01-01").to(Types.DateType.get()), + Literal.of("2023-01-01").to(Types.DateType.get()), + "DATE '2024-01-01'", + "DATE '2023-01-01'"), + Arguments.of( + Types.TimestampType.withZone(), + Literal.of("2017-11-30T10:30:07.123456+00:00").to(Types.TimestampType.withZone()), + Literal.of("2017-11-29T10:30:07.123456+00:00").to(Types.TimestampType.withZone()), + "TIMESTAMP '2017-11-30 10:30:07.123456'", + "TIMESTAMP '2017-11-29 10:30:07.123456'"), + Arguments.of( + Types.TimestampType.withoutZone(), + Literal.of("2017-11-30T10:30:07.123456").to(Types.TimestampType.withoutZone()), + Literal.of("2017-11-29T10:30:07.123456").to(Types.TimestampType.withoutZone()), + "TIMESTAMP_NTZ '2017-11-30 10:30:07.123456'", + "TIMESTAMP_NTZ '2017-11-29 10:30:07.123456'"), + Arguments.of( + Types.BinaryType.get(), + Literal.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b})), + Literal.of(ByteBuffer.wrap(new byte[] {0x01, 0x02})), + "X'0A0B'", + "X'0102'"), + Arguments.of( + Types.FixedType.ofLength(4), + Literal.of("test".getBytes()), + Literal.of("init".getBytes()), + "X'74657374'", + "X'696E6974'")); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkSessionCatalog.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkSessionCatalog.java new file mode 100644 index 000000000000..85408ffff1d9 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkSessionCatalog.java @@ -0,0 +1,101 @@ +/* + * 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 static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class TestSparkSessionCatalog extends TestBase { + private final String envHmsUriKey = "spark.hadoop." + METASTOREURIS.varname; + private final String catalogHmsUriKey = "spark.sql.catalog.spark_catalog.uri"; + private final String hmsUri = hiveConf.get(METASTOREURIS.varname); + + @BeforeAll + public static void setUpCatalog() { + spark + .conf() + .set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog"); + spark.conf().set("spark.sql.catalog.spark_catalog.type", "hive"); + } + + @BeforeEach + public void setupHmsUri() { + spark.sessionState().catalogManager().reset(); + spark.conf().set(envHmsUriKey, hmsUri); + spark.conf().set(catalogHmsUriKey, hmsUri); + } + + @Test + public void testValidateHmsUri() { + // HMS uris match + assertThat(spark.sessionState().catalogManager().v2SessionCatalog().defaultNamespace()) + .containsExactly("default"); + + // HMS uris doesn't match + spark.sessionState().catalogManager().reset(); + String catalogHmsUri = "RandomString"; + spark.conf().set(envHmsUriKey, hmsUri); + spark.conf().set(catalogHmsUriKey, catalogHmsUri); + + assertThatThrownBy(() -> spark.sessionState().catalogManager().v2SessionCatalog()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + String.format( + "Inconsistent Hive metastore URIs: %s (Spark session) != %s (spark_catalog)", + hmsUri, catalogHmsUri)); + + // no env HMS uri, only catalog HMS uri + spark.sessionState().catalogManager().reset(); + spark.conf().set(catalogHmsUriKey, hmsUri); + spark.conf().unset(envHmsUriKey); + assertThat(spark.sessionState().catalogManager().v2SessionCatalog().defaultNamespace()) + .containsExactly("default"); + + // no catalog HMS uri, only env HMS uri + spark.sessionState().catalogManager().reset(); + spark.conf().set(envHmsUriKey, hmsUri); + spark.conf().unset(catalogHmsUriKey); + assertThat(spark.sessionState().catalogManager().v2SessionCatalog().defaultNamespace()) + .containsExactly("default"); + } + + @Test + public void testLoadFunction() { + String functionClass = "org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper"; + + // load permanent UDF in Hive via FunctionCatalog + spark.sql(String.format("CREATE FUNCTION perm_upper AS '%s'", functionClass)); + assertThat(scalarSql("SELECT perm_upper('xyz')")) + .as("Load permanent UDF in Hive") + .isEqualTo("XYZ"); + + // load temporary UDF in Hive via FunctionCatalog + spark.sql(String.format("CREATE TEMPORARY FUNCTION temp_upper AS '%s'", functionClass)); + assertThat(scalarSql("SELECT temp_upper('xyz')")) + .as("Load temporary UDF in Hive") + .isEqualTo("XYZ"); + + // TODO: fix loading Iceberg built-in functions in SessionCatalog + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java new file mode 100644 index 000000000000..93e4c8968715 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java @@ -0,0 +1,106 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.KryoHelpers; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.MetricsModes; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkTableUtil.SparkPartition; +import org.junit.jupiter.api.Test; + +public class TestSparkTableUtil { + @Test + public void testSparkPartitionOKryoSerialization() throws IOException { + Map values = ImmutableMap.of("id", "2"); + String uri = "s3://bucket/table/data/id=2"; + String format = "parquet"; + SparkPartition sparkPartition = new SparkPartition(values, uri, format); + + SparkPartition deserialized = KryoHelpers.roundTripSerialize(sparkPartition); + assertThat(sparkPartition).isEqualTo(deserialized); + } + + @Test + public void testSparkPartitionJavaSerialization() throws IOException, ClassNotFoundException { + Map values = ImmutableMap.of("id", "2"); + String uri = "s3://bucket/table/data/id=2"; + String format = "parquet"; + SparkPartition sparkPartition = new SparkPartition(values, uri, format); + + SparkPartition deserialized = TestHelpers.roundTripSerialize(sparkPartition); + assertThat(sparkPartition).isEqualTo(deserialized); + } + + @Test + public void testMetricsConfigKryoSerialization() throws Exception { + Map metricsConfig = + ImmutableMap.of( + TableProperties.DEFAULT_WRITE_METRICS_MODE, + "counts", + TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "col1", + "full", + TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "col2", + "truncate(16)"); + + MetricsConfig config = MetricsConfig.fromProperties(metricsConfig); + MetricsConfig deserialized = KryoHelpers.roundTripSerialize(config); + + assertThat(deserialized.columnMode("col1")) + .asString() + .isEqualTo(MetricsModes.Full.get().toString()); + assertThat(deserialized.columnMode("col2")) + .asString() + .isEqualTo(MetricsModes.Truncate.withLength(16).toString()); + assertThat(deserialized.columnMode("col3")) + .asString() + .isEqualTo(MetricsModes.Counts.get().toString()); + } + + @Test + public void testMetricsConfigJavaSerialization() throws Exception { + Map metricsConfig = + ImmutableMap.of( + TableProperties.DEFAULT_WRITE_METRICS_MODE, + "counts", + TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "col1", + "full", + TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "col2", + "truncate(16)"); + + MetricsConfig config = MetricsConfig.fromProperties(metricsConfig); + MetricsConfig deserialized = TestHelpers.roundTripSerialize(config); + + assertThat(deserialized.columnMode("col1")) + .asString() + .isEqualTo(MetricsModes.Full.get().toString()); + assertThat(deserialized.columnMode("col2")) + .asString() + .isEqualTo(MetricsModes.Truncate.withLength(16).toString()); + assertThat(deserialized.columnMode("col3")) + .asString() + .isEqualTo(MetricsModes.Counts.get().toString()); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java new file mode 100644 index 000000000000..e0b590e5a6e8 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java @@ -0,0 +1,853 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.time.Instant; +import java.time.LocalDate; +import java.time.temporal.ChronoUnit; +import java.util.Map; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionUtil; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.UnboundTerm; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.functions.BucketFunction; +import org.apache.iceberg.spark.functions.DaysFunction; +import org.apache.iceberg.spark.functions.HoursFunction; +import org.apache.iceberg.spark.functions.IcebergVersionFunction; +import org.apache.iceberg.spark.functions.MonthsFunction; +import org.apache.iceberg.spark.functions.TruncateFunction; +import org.apache.iceberg.spark.functions.YearsFunction; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; +import org.apache.spark.sql.connector.expressions.FieldReference; +import org.apache.spark.sql.connector.expressions.LiteralValue; +import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.expressions.UserDefinedScalarFunc; +import org.apache.spark.sql.connector.expressions.filter.And; +import org.apache.spark.sql.connector.expressions.filter.Not; +import org.apache.spark.sql.connector.expressions.filter.Or; +import org.apache.spark.sql.connector.expressions.filter.Predicate; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.jupiter.api.Test; + +public class TestSparkV2Filters { + + private static final Types.StructType STRUCT = + Types.StructType.of( + Types.NestedField.optional(1, "dateCol", Types.DateType.get()), + Types.NestedField.optional(2, "tsCol", Types.TimestampType.withZone()), + Types.NestedField.optional(3, "tsNtzCol", Types.TimestampType.withoutZone()), + Types.NestedField.optional(4, "intCol", Types.IntegerType.get()), + Types.NestedField.optional(5, "strCol", Types.StringType.get())); + + @SuppressWarnings("checkstyle:MethodLength") + @Test + public void testV2Filters() { + Map attrMap = Maps.newHashMap(); + attrMap.put("id", "id"); + attrMap.put("`i.d`", "i.d"); + attrMap.put("`i``d`", "i`d"); + attrMap.put("`d`.b.`dd```", "d.b.dd`"); + attrMap.put("a.`aa```.c", "a.aa`.c"); + + attrMap.forEach( + (quoted, unquoted) -> { + NamedReference namedReference = FieldReference.apply(quoted); + org.apache.spark.sql.connector.expressions.Expression[] attrOnly = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference}; + + LiteralValue value = new LiteralValue(1, DataTypes.IntegerType); + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference, value}; + org.apache.spark.sql.connector.expressions.Expression[] valueAndAttr = + new org.apache.spark.sql.connector.expressions.Expression[] {value, namedReference}; + + Predicate isNull = new Predicate("IS_NULL", attrOnly); + Expression expectedIsNull = Expressions.isNull(unquoted); + Expression actualIsNull = SparkV2Filters.convert(isNull); + assertThat(actualIsNull) + .asString() + .as("IsNull must match") + .isEqualTo(expectedIsNull.toString()); + + Predicate isNotNull = new Predicate("IS_NOT_NULL", attrOnly); + Expression expectedIsNotNull = Expressions.notNull(unquoted); + Expression actualIsNotNull = SparkV2Filters.convert(isNotNull); + assertThat(actualIsNotNull) + .asString() + .as("IsNotNull must match") + .isEqualTo(expectedIsNotNull.toString()); + + Predicate lt1 = new Predicate("<", attrAndValue); + Expression expectedLt1 = Expressions.lessThan(unquoted, 1); + Expression actualLt1 = SparkV2Filters.convert(lt1); + assertThat(actualLt1) + .asString() + .as("LessThan must match") + .isEqualTo(expectedLt1.toString()); + + Predicate lt2 = new Predicate("<", valueAndAttr); + Expression expectedLt2 = Expressions.greaterThan(unquoted, 1); + Expression actualLt2 = SparkV2Filters.convert(lt2); + assertThat(actualLt2) + .asString() + .as("LessThan must match") + .isEqualTo(expectedLt2.toString()); + + Predicate ltEq1 = new Predicate("<=", attrAndValue); + Expression expectedLtEq1 = Expressions.lessThanOrEqual(unquoted, 1); + Expression actualLtEq1 = SparkV2Filters.convert(ltEq1); + assertThat(actualLtEq1) + .asString() + .as("LessThanOrEqual must match") + .isEqualTo(expectedLtEq1.toString()); + + Predicate ltEq2 = new Predicate("<=", valueAndAttr); + Expression expectedLtEq2 = Expressions.greaterThanOrEqual(unquoted, 1); + Expression actualLtEq2 = SparkV2Filters.convert(ltEq2); + assertThat(actualLtEq2) + .asString() + .as("LessThanOrEqual must match") + .isEqualTo(expectedLtEq2.toString()); + + Predicate gt1 = new Predicate(">", attrAndValue); + Expression expectedGt1 = Expressions.greaterThan(unquoted, 1); + Expression actualGt1 = SparkV2Filters.convert(gt1); + assertThat(actualGt1) + .asString() + .as("GreaterThan must match") + .isEqualTo(expectedGt1.toString()); + + Predicate gt2 = new Predicate(">", valueAndAttr); + Expression expectedGt2 = Expressions.lessThan(unquoted, 1); + Expression actualGt2 = SparkV2Filters.convert(gt2); + assertThat(actualGt2) + .asString() + .as("GreaterThan must match") + .isEqualTo(expectedGt2.toString()); + + Predicate gtEq1 = new Predicate(">=", attrAndValue); + Expression expectedGtEq1 = Expressions.greaterThanOrEqual(unquoted, 1); + Expression actualGtEq1 = SparkV2Filters.convert(gtEq1); + assertThat(actualGtEq1) + .asString() + .as("GreaterThanOrEqual must match") + .isEqualTo(expectedGtEq1.toString()); + + Predicate gtEq2 = new Predicate(">=", valueAndAttr); + Expression expectedGtEq2 = Expressions.lessThanOrEqual(unquoted, 1); + Expression actualGtEq2 = SparkV2Filters.convert(gtEq2); + assertThat(actualGtEq2) + .asString() + .as("GreaterThanOrEqual must match") + .isEqualTo(expectedGtEq2.toString()); + + Predicate eq1 = new Predicate("=", attrAndValue); + Expression expectedEq1 = Expressions.equal(unquoted, 1); + Expression actualEq1 = SparkV2Filters.convert(eq1); + assertThat(actualEq1) + .asString() + .as("EqualTo must match") + .isEqualTo(expectedEq1.toString()); + + Predicate eq2 = new Predicate("=", valueAndAttr); + Expression expectedEq2 = Expressions.equal(unquoted, 1); + Expression actualEq2 = SparkV2Filters.convert(eq2); + assertThat(actualEq2) + .asString() + .as("EqualTo must match") + .isEqualTo(expectedEq2.toString()); + + Predicate notEq1 = new Predicate("<>", attrAndValue); + Expression expectedNotEq1 = Expressions.notEqual(unquoted, 1); + Expression actualNotEq1 = SparkV2Filters.convert(notEq1); + assertThat(actualNotEq1) + .asString() + .as("NotEqualTo must match") + .isEqualTo(expectedNotEq1.toString()); + + Predicate notEq2 = new Predicate("<>", valueAndAttr); + Expression expectedNotEq2 = Expressions.notEqual(unquoted, 1); + Expression actualNotEq2 = SparkV2Filters.convert(notEq2); + assertThat(actualNotEq2) + .asString() + .as("NotEqualTo must match") + .isEqualTo(expectedNotEq2.toString()); + + Predicate eqNullSafe1 = new Predicate("<=>", attrAndValue); + Expression expectedEqNullSafe1 = Expressions.equal(unquoted, 1); + Expression actualEqNullSafe1 = SparkV2Filters.convert(eqNullSafe1); + assertThat(actualEqNullSafe1) + .asString() + .as("EqualNullSafe must match") + .isEqualTo(expectedEqNullSafe1.toString()); + + Predicate eqNullSafe2 = new Predicate("<=>", valueAndAttr); + Expression expectedEqNullSafe2 = Expressions.equal(unquoted, 1); + Expression actualEqNullSafe2 = SparkV2Filters.convert(eqNullSafe2); + assertThat(actualEqNullSafe2) + .asString() + .as("EqualNullSafe must match") + .isEqualTo(expectedEqNullSafe2.toString()); + + LiteralValue str = + new LiteralValue(UTF8String.fromString("iceberg"), DataTypes.StringType); + org.apache.spark.sql.connector.expressions.Expression[] attrAndStr = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference, str}; + Predicate startsWith = new Predicate("STARTS_WITH", attrAndStr); + Expression expectedStartsWith = Expressions.startsWith(unquoted, "iceberg"); + Expression actualStartsWith = SparkV2Filters.convert(startsWith); + assertThat(actualStartsWith) + .asString() + .as("StartsWith must match") + .isEqualTo(expectedStartsWith.toString()); + + Predicate in = new Predicate("IN", attrAndValue); + Expression expectedIn = Expressions.in(unquoted, 1); + Expression actualIn = SparkV2Filters.convert(in); + assertThat(actualIn).asString().as("In must match").isEqualTo(expectedIn.toString()); + + Predicate and = new And(lt1, eq1); + Expression expectedAnd = Expressions.and(expectedLt1, expectedEq1); + Expression actualAnd = SparkV2Filters.convert(and); + assertThat(actualAnd).asString().as("And must match").isEqualTo(expectedAnd.toString()); + + org.apache.spark.sql.connector.expressions.Expression[] attrAndAttr = + new org.apache.spark.sql.connector.expressions.Expression[] { + namedReference, namedReference + }; + Predicate invalid = new Predicate("<", attrAndAttr); + Predicate andWithInvalidLeft = new And(invalid, eq1); + Expression convertedAnd = SparkV2Filters.convert(andWithInvalidLeft); + assertThat(convertedAnd).as("And must match").isNull(); + + Predicate or = new Or(lt1, eq1); + Expression expectedOr = Expressions.or(expectedLt1, expectedEq1); + Expression actualOr = SparkV2Filters.convert(or); + assertThat(actualOr).asString().as("Or must match").isEqualTo(expectedOr.toString()); + + Predicate orWithInvalidLeft = new Or(invalid, eq1); + Expression convertedOr = SparkV2Filters.convert(orWithInvalidLeft); + assertThat(convertedOr).as("Or must match").isNull(); + + Predicate not = new Not(lt1); + Expression expectedNot = Expressions.not(expectedLt1); + Expression actualNot = SparkV2Filters.convert(not); + assertThat(actualNot).asString().as("Not must match").isEqualTo(expectedNot.toString()); + }); + } + + @Test + public void testEqualToNull() { + String col = "col"; + NamedReference namedReference = FieldReference.apply(col); + LiteralValue value = new LiteralValue(null, DataTypes.IntegerType); + + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference, value}; + org.apache.spark.sql.connector.expressions.Expression[] valueAndAttr = + new org.apache.spark.sql.connector.expressions.Expression[] {value, namedReference}; + + Predicate eq1 = new Predicate("=", attrAndValue); + assertThatThrownBy(() -> SparkV2Filters.convert(eq1)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("Expression is always false"); + + Predicate eq2 = new Predicate("=", valueAndAttr); + assertThatThrownBy(() -> SparkV2Filters.convert(eq2)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("Expression is always false"); + + Predicate eqNullSafe1 = new Predicate("<=>", attrAndValue); + Expression expectedEqNullSafe = Expressions.isNull(col); + Expression actualEqNullSafe1 = SparkV2Filters.convert(eqNullSafe1); + assertThat(actualEqNullSafe1.toString()).isEqualTo(expectedEqNullSafe.toString()); + + Predicate eqNullSafe2 = new Predicate("<=>", valueAndAttr); + Expression actualEqNullSafe2 = SparkV2Filters.convert(eqNullSafe2); + assertThat(actualEqNullSafe2.toString()).isEqualTo(expectedEqNullSafe.toString()); + } + + @Test + public void testEqualToNaN() { + String col = "col"; + NamedReference namedReference = FieldReference.apply(col); + LiteralValue value = new LiteralValue(Float.NaN, DataTypes.FloatType); + + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference, value}; + org.apache.spark.sql.connector.expressions.Expression[] valueAndAttr = + new org.apache.spark.sql.connector.expressions.Expression[] {value, namedReference}; + + Predicate eqNaN1 = new Predicate("=", attrAndValue); + Expression expectedEqNaN = Expressions.isNaN(col); + Expression actualEqNaN1 = SparkV2Filters.convert(eqNaN1); + assertThat(actualEqNaN1.toString()).isEqualTo(expectedEqNaN.toString()); + + Predicate eqNaN2 = new Predicate("=", valueAndAttr); + Expression actualEqNaN2 = SparkV2Filters.convert(eqNaN2); + assertThat(actualEqNaN2.toString()).isEqualTo(expectedEqNaN.toString()); + } + + @Test + public void testNotEqualToNull() { + String col = "col"; + NamedReference namedReference = FieldReference.apply(col); + LiteralValue value = new LiteralValue(null, DataTypes.IntegerType); + + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference, value}; + org.apache.spark.sql.connector.expressions.Expression[] valueAndAttr = + new org.apache.spark.sql.connector.expressions.Expression[] {value, namedReference}; + + Predicate notEq1 = new Predicate("<>", attrAndValue); + assertThatThrownBy(() -> SparkV2Filters.convert(notEq1)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("Expression is always false"); + + Predicate notEq2 = new Predicate("<>", valueAndAttr); + assertThatThrownBy(() -> SparkV2Filters.convert(notEq2)) + .isInstanceOf(NullPointerException.class) + .hasMessageContaining("Expression is always false"); + } + + @Test + public void testNotEqualToNaN() { + String col = "col"; + NamedReference namedReference = FieldReference.apply(col); + LiteralValue value = new LiteralValue(Float.NaN, DataTypes.FloatType); + + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference, value}; + org.apache.spark.sql.connector.expressions.Expression[] valueAndAttr = + new org.apache.spark.sql.connector.expressions.Expression[] {value, namedReference}; + + Predicate notEqNaN1 = new Predicate("<>", attrAndValue); + Expression expectedNotEqNaN = Expressions.notNaN(col); + Expression actualNotEqNaN1 = SparkV2Filters.convert(notEqNaN1); + assertThat(actualNotEqNaN1.toString()).isEqualTo(expectedNotEqNaN.toString()); + + Predicate notEqNaN2 = new Predicate("<>", valueAndAttr); + Expression actualNotEqNaN2 = SparkV2Filters.convert(notEqNaN2); + assertThat(actualNotEqNaN2.toString()).isEqualTo(expectedNotEqNaN.toString()); + } + + @Test + public void testInValuesContainNull() { + String col = "strCol"; + NamedReference namedReference = FieldReference.apply(col); + LiteralValue nullValue = new LiteralValue(null, DataTypes.StringType); + LiteralValue value1 = new LiteralValue("value1", DataTypes.StringType); + LiteralValue value2 = new LiteralValue("value2", DataTypes.StringType); + + // Values only contains null + Predicate inNull = new Predicate("IN", expressions(namedReference, nullValue)); + Expression expectedInNull = Expressions.in(col); + Expression actualInNull = SparkV2Filters.convert(inNull); + assertEquals(expectedInNull, actualInNull); + + Predicate in = new Predicate("IN", expressions(namedReference, nullValue, value1, value2)); + Expression expectedIn = Expressions.in(col, "value1", "value2"); + Expression actualIn = SparkV2Filters.convert(in); + assertEquals(expectedIn, actualIn); + } + + @Test + public void testNotInNull() { + String col = "strCol"; + NamedReference namedReference = FieldReference.apply(col); + LiteralValue nullValue = new LiteralValue(null, DataTypes.StringType); + LiteralValue value1 = new LiteralValue("value1", DataTypes.StringType); + LiteralValue value2 = new LiteralValue("value2", DataTypes.StringType); + + // Values only contains null + Predicate notInNull = new Not(new Predicate("IN", expressions(namedReference, nullValue))); + Expression expectedNotInNull = + Expressions.and(Expressions.notNull(col), Expressions.notIn(col)); + Expression actualNotInNull = SparkV2Filters.convert(notInNull); + assertEquals(expectedNotInNull, actualNotInNull); + + Predicate notIn = + new Not(new Predicate("IN", expressions(namedReference, nullValue, value1, value2))); + Expression expectedNotIn = + Expressions.and(Expressions.notNull(col), Expressions.notIn(col, "value1", "value2")); + Expression actualNotIn = SparkV2Filters.convert(notIn); + assertEquals(expectedNotIn, actualNotIn); + } + + @Test + public void testTimestampFilterConversion() { + Instant instant = Instant.parse("2018-10-18T00:00:57.907Z"); + long epochMicros = ChronoUnit.MICROS.between(Instant.EPOCH, instant); + + NamedReference namedReference = FieldReference.apply("x"); + LiteralValue ts = new LiteralValue(epochMicros, DataTypes.TimestampType); + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference, ts}; + + Predicate predicate = new Predicate(">", attrAndValue); + Expression tsExpression = SparkV2Filters.convert(predicate); + Expression rawExpression = Expressions.greaterThan("x", epochMicros); + + assertThat(tsExpression) + .asString() + .as("Generated Timestamp expression should be correct") + .isEqualTo(rawExpression.toString()); + } + + @Test + public void testDateFilterConversion() { + LocalDate localDate = LocalDate.parse("2018-10-18"); + long epochDay = localDate.toEpochDay(); + + NamedReference namedReference = FieldReference.apply("x"); + LiteralValue ts = new LiteralValue(epochDay, DataTypes.DateType); + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference, ts}; + + Predicate predicate = new Predicate(">", attrAndValue); + Expression dateExpression = SparkV2Filters.convert(predicate); + Expression rawExpression = Expressions.greaterThan("x", epochDay); + + assertThat(dateExpression) + .asString() + .as("Generated date expression should be correct") + .isEqualTo(rawExpression.toString()); + } + + @Test + public void testNestedInInsideNot() { + NamedReference namedReference1 = FieldReference.apply("col1"); + LiteralValue v1 = new LiteralValue(1, DataTypes.IntegerType); + LiteralValue v2 = new LiteralValue(2, DataTypes.IntegerType); + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue1 = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference1, v1}; + Predicate equal = new Predicate("=", attrAndValue1); + + NamedReference namedReference2 = FieldReference.apply("col2"); + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue2 = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference2, v1, v2}; + Predicate in = new Predicate("IN", attrAndValue2); + + Not filter = new Not(new And(equal, in)); + Expression converted = SparkV2Filters.convert(filter); + assertThat(converted).as("Expression should not be converted").isNull(); + } + + @Test + public void testNotIn() { + NamedReference namedReference = FieldReference.apply("col"); + LiteralValue v1 = new LiteralValue(1, DataTypes.IntegerType); + LiteralValue v2 = new LiteralValue(2, DataTypes.IntegerType); + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue = + new org.apache.spark.sql.connector.expressions.Expression[] {namedReference, v1, v2}; + + Predicate in = new Predicate("IN", attrAndValue); + Not not = new Not(in); + + Expression actual = SparkV2Filters.convert(not); + Expression expected = + Expressions.and(Expressions.notNull("col"), Expressions.notIn("col", 1, 2)); + assertThat(actual).asString().as("Expressions should match").isEqualTo(expected.toString()); + } + + @Test + public void testDateToYears() { + ScalarFunction dateToYearsFunc = new YearsFunction.DateToYearsFunction(); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + dateToYearsFunc.name(), + dateToYearsFunc.canonicalName(), + expressions(FieldReference.apply("dateCol"))); + testUDF(udf, Expressions.year("dateCol"), dateToYears("2023-06-25"), DataTypes.IntegerType); + } + + @Test + public void testTsToYears() { + ScalarFunction tsToYearsFunc = new YearsFunction.TimestampToYearsFunction(); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + tsToYearsFunc.name(), + tsToYearsFunc.canonicalName(), + expressions(FieldReference.apply("tsCol"))); + testUDF( + udf, + Expressions.year("tsCol"), + timestampToYears("2023-12-03T10:15:30+01:00"), + DataTypes.IntegerType); + } + + @Test + public void testTsNtzToYears() { + ScalarFunction tsNtzToYearsFunc = new YearsFunction.TimestampNtzToYearsFunction(); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + tsNtzToYearsFunc.name(), + tsNtzToYearsFunc.canonicalName(), + expressions(FieldReference.apply("tsNtzCol"))); + testUDF( + udf, + Expressions.year("tsNtzCol"), + timestampNtzToYears("2023-06-25T13:15:30"), + DataTypes.IntegerType); + } + + @Test + public void testDateToMonths() { + ScalarFunction dateToMonthsFunc = new MonthsFunction.DateToMonthsFunction(); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + dateToMonthsFunc.name(), + dateToMonthsFunc.canonicalName(), + expressions(FieldReference.apply("dateCol"))); + testUDF(udf, Expressions.month("dateCol"), dateToMonths("2023-06-25"), DataTypes.IntegerType); + } + + @Test + public void testTsToMonths() { + ScalarFunction tsToMonthsFunc = new MonthsFunction.TimestampToMonthsFunction(); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + tsToMonthsFunc.name(), + tsToMonthsFunc.canonicalName(), + expressions(FieldReference.apply("tsCol"))); + testUDF( + udf, + Expressions.month("tsCol"), + timestampToMonths("2023-12-03T10:15:30+01:00"), + DataTypes.IntegerType); + } + + @Test + public void testTsNtzToMonths() { + ScalarFunction tsNtzToMonthsFunc = new MonthsFunction.TimestampNtzToMonthsFunction(); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + tsNtzToMonthsFunc.name(), + tsNtzToMonthsFunc.canonicalName(), + expressions(FieldReference.apply("tsNtzCol"))); + testUDF( + udf, + Expressions.month("tsNtzCol"), + timestampNtzToMonths("2023-12-03T10:15:30"), + DataTypes.IntegerType); + } + + @Test + public void testDateToDays() { + ScalarFunction dateToDayFunc = new DaysFunction.DateToDaysFunction(); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + dateToDayFunc.name(), + dateToDayFunc.canonicalName(), + expressions(FieldReference.apply("dateCol"))); + testUDF(udf, Expressions.day("dateCol"), dateToDays("2023-06-25"), DataTypes.IntegerType); + } + + @Test + public void testTsToDays() { + ScalarFunction tsToDaysFunc = new DaysFunction.TimestampToDaysFunction(); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + tsToDaysFunc.name(), + tsToDaysFunc.canonicalName(), + expressions(FieldReference.apply("tsCol"))); + testUDF( + udf, + Expressions.day("tsCol"), + timestampToDays("2023-12-03T10:15:30+01:00"), + DataTypes.IntegerType); + } + + @Test + public void testTsNtzToDays() { + ScalarFunction tsNtzToDaysFunc = new DaysFunction.TimestampNtzToDaysFunction(); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + tsNtzToDaysFunc.name(), + tsNtzToDaysFunc.canonicalName(), + expressions(FieldReference.apply("tsNtzCol"))); + testUDF( + udf, + Expressions.day("tsNtzCol"), + timestampNtzToDays("2023-12-03T10:15:30"), + DataTypes.IntegerType); + } + + @Test + public void testTsToHours() { + ScalarFunction tsToHourFunc = new HoursFunction.TimestampToHoursFunction(); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + tsToHourFunc.name(), + tsToHourFunc.canonicalName(), + expressions(FieldReference.apply("tsCol"))); + testUDF( + udf, + Expressions.hour("tsCol"), + timestampToHours("2023-12-03T10:15:30+01:00"), + DataTypes.IntegerType); + } + + @Test + public void testTsNtzToHours() { + ScalarFunction tsNtzToHourFunc = new HoursFunction.TimestampNtzToHoursFunction(); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + tsNtzToHourFunc.name(), + tsNtzToHourFunc.canonicalName(), + expressions(FieldReference.apply("tsNtzCol"))); + testUDF( + udf, + Expressions.hour("tsNtzCol"), + timestampNtzToHours("2023-12-03T10:15:30"), + DataTypes.IntegerType); + } + + @Test + public void testBucket() { + ScalarFunction bucketInt = new BucketFunction.BucketInt(DataTypes.IntegerType); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + bucketInt.name(), + bucketInt.canonicalName(), + expressions( + LiteralValue.apply(4, DataTypes.IntegerType), FieldReference.apply("intCol"))); + testUDF(udf, Expressions.bucket("intCol", 4), 2, DataTypes.IntegerType); + } + + @Test + public void testTruncate() { + ScalarFunction truncate = new TruncateFunction.TruncateString(); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + truncate.name(), + truncate.canonicalName(), + expressions( + LiteralValue.apply(6, DataTypes.IntegerType), FieldReference.apply("strCol"))); + testUDF(udf, Expressions.truncate("strCol", 6), "prefix", DataTypes.StringType); + } + + @Test + public void testUnsupportedUDFConvert() { + ScalarFunction icebergVersionFunc = + (ScalarFunction) new IcebergVersionFunction().bind(new StructType()); + UserDefinedScalarFunc udf = + new UserDefinedScalarFunc( + icebergVersionFunc.name(), + icebergVersionFunc.canonicalName(), + new org.apache.spark.sql.connector.expressions.Expression[] {}); + LiteralValue literalValue = new LiteralValue("1.3.0", DataTypes.StringType); + Predicate predicate = new Predicate("=", expressions(udf, literalValue)); + + Expression icebergExpr = SparkV2Filters.convert(predicate); + assertThat(icebergExpr).isNull(); + } + + private void testUDF( + org.apache.spark.sql.connector.expressions.Expression udf, + UnboundTerm expectedTerm, + T value, + DataType dataType) { + org.apache.spark.sql.connector.expressions.Expression[] attrOnly = expressions(udf); + + LiteralValue literalValue = new LiteralValue(value, dataType); + org.apache.spark.sql.connector.expressions.Expression[] attrAndValue = + expressions(udf, literalValue); + org.apache.spark.sql.connector.expressions.Expression[] valueAndAttr = + expressions(literalValue, udf); + + Predicate isNull = new Predicate("IS_NULL", attrOnly); + Expression expectedIsNull = Expressions.isNull(expectedTerm); + Expression actualIsNull = SparkV2Filters.convert(isNull); + assertEquals(expectedIsNull, actualIsNull); + + Predicate isNotNull = new Predicate("IS_NOT_NULL", attrOnly); + Expression expectedIsNotNull = Expressions.notNull(expectedTerm); + Expression actualIsNotNull = SparkV2Filters.convert(isNotNull); + assertEquals(expectedIsNotNull, actualIsNotNull); + + Predicate lt1 = new Predicate("<", attrAndValue); + Expression expectedLt1 = Expressions.lessThan(expectedTerm, value); + Expression actualLt1 = SparkV2Filters.convert(lt1); + assertEquals(expectedLt1, actualLt1); + + Predicate lt2 = new Predicate("<", valueAndAttr); + Expression expectedLt2 = Expressions.greaterThan(expectedTerm, value); + Expression actualLt2 = SparkV2Filters.convert(lt2); + assertEquals(expectedLt2, actualLt2); + + Predicate ltEq1 = new Predicate("<=", attrAndValue); + Expression expectedLtEq1 = Expressions.lessThanOrEqual(expectedTerm, value); + Expression actualLtEq1 = SparkV2Filters.convert(ltEq1); + assertEquals(expectedLtEq1, actualLtEq1); + + Predicate ltEq2 = new Predicate("<=", valueAndAttr); + Expression expectedLtEq2 = Expressions.greaterThanOrEqual(expectedTerm, value); + Expression actualLtEq2 = SparkV2Filters.convert(ltEq2); + assertEquals(expectedLtEq2, actualLtEq2); + + Predicate gt1 = new Predicate(">", attrAndValue); + Expression expectedGt1 = Expressions.greaterThan(expectedTerm, value); + Expression actualGt1 = SparkV2Filters.convert(gt1); + assertEquals(expectedGt1, actualGt1); + + Predicate gt2 = new Predicate(">", valueAndAttr); + Expression expectedGt2 = Expressions.lessThan(expectedTerm, value); + Expression actualGt2 = SparkV2Filters.convert(gt2); + assertEquals(expectedGt2, actualGt2); + + Predicate gtEq1 = new Predicate(">=", attrAndValue); + Expression expectedGtEq1 = Expressions.greaterThanOrEqual(expectedTerm, value); + Expression actualGtEq1 = SparkV2Filters.convert(gtEq1); + assertEquals(expectedGtEq1, actualGtEq1); + + Predicate gtEq2 = new Predicate(">=", valueAndAttr); + Expression expectedGtEq2 = Expressions.lessThanOrEqual(expectedTerm, value); + Expression actualGtEq2 = SparkV2Filters.convert(gtEq2); + assertEquals(expectedGtEq2, actualGtEq2); + + Predicate eq1 = new Predicate("=", attrAndValue); + Expression expectedEq1 = Expressions.equal(expectedTerm, value); + Expression actualEq1 = SparkV2Filters.convert(eq1); + assertEquals(expectedEq1, actualEq1); + + Predicate eq2 = new Predicate("=", valueAndAttr); + Expression expectedEq2 = Expressions.equal(expectedTerm, value); + Expression actualEq2 = SparkV2Filters.convert(eq2); + assertEquals(expectedEq2, actualEq2); + + Predicate notEq1 = new Predicate("<>", attrAndValue); + Expression expectedNotEq1 = Expressions.notEqual(expectedTerm, value); + Expression actualNotEq1 = SparkV2Filters.convert(notEq1); + assertEquals(expectedNotEq1, actualNotEq1); + + Predicate notEq2 = new Predicate("<>", valueAndAttr); + Expression expectedNotEq2 = Expressions.notEqual(expectedTerm, value); + Expression actualNotEq2 = SparkV2Filters.convert(notEq2); + assertEquals(expectedNotEq2, actualNotEq2); + + Predicate eqNullSafe1 = new Predicate("<=>", attrAndValue); + Expression expectedEqNullSafe1 = Expressions.equal(expectedTerm, value); + Expression actualEqNullSafe1 = SparkV2Filters.convert(eqNullSafe1); + assertEquals(expectedEqNullSafe1, actualEqNullSafe1); + + Predicate eqNullSafe2 = new Predicate("<=>", valueAndAttr); + Expression expectedEqNullSafe2 = Expressions.equal(expectedTerm, value); + Expression actualEqNullSafe2 = SparkV2Filters.convert(eqNullSafe2); + assertEquals(expectedEqNullSafe2, actualEqNullSafe2); + + Predicate in = new Predicate("IN", attrAndValue); + Expression expectedIn = Expressions.in(expectedTerm, value); + Expression actualIn = SparkV2Filters.convert(in); + assertEquals(expectedIn, actualIn); + + Predicate notIn = new Not(in); + Expression expectedNotIn = + Expressions.and(Expressions.notNull(expectedTerm), Expressions.notIn(expectedTerm, value)); + Expression actualNotIn = SparkV2Filters.convert(notIn); + assertEquals(expectedNotIn, actualNotIn); + + Predicate and = new And(lt1, eq1); + Expression expectedAnd = Expressions.and(expectedLt1, expectedEq1); + Expression actualAnd = SparkV2Filters.convert(and); + assertEquals(expectedAnd, actualAnd); + + org.apache.spark.sql.connector.expressions.Expression[] attrAndAttr = expressions(udf, udf); + Predicate invalid = new Predicate("<", attrAndAttr); + Predicate andWithInvalidLeft = new And(invalid, eq1); + Expression convertedAnd = SparkV2Filters.convert(andWithInvalidLeft); + assertThat(convertedAnd).isNull(); + + Predicate or = new Or(lt1, eq1); + Expression expectedOr = Expressions.or(expectedLt1, expectedEq1); + Expression actualOr = SparkV2Filters.convert(or); + assertEquals(expectedOr, actualOr); + + Predicate orWithInvalidLeft = new Or(invalid, eq1); + Expression convertedOr = SparkV2Filters.convert(orWithInvalidLeft); + assertThat(convertedOr).isNull(); + + Predicate not = new Not(lt1); + Expression expectedNot = Expressions.not(expectedLt1); + Expression actualNot = SparkV2Filters.convert(not); + assertEquals(expectedNot, actualNot); + } + + private static void assertEquals(Expression expected, Expression actual) { + assertThat(ExpressionUtil.equivalent(expected, actual, STRUCT, true)).isTrue(); + } + + private org.apache.spark.sql.connector.expressions.Expression[] expressions( + org.apache.spark.sql.connector.expressions.Expression... expressions) { + return expressions; + } + + private static int dateToYears(String dateString) { + return DateTimeUtil.daysToYears(DateTimeUtil.isoDateToDays(dateString)); + } + + private static int timestampToYears(String timestampString) { + return DateTimeUtil.microsToYears(DateTimeUtil.isoTimestamptzToMicros(timestampString)); + } + + private static int timestampNtzToYears(String timestampNtzString) { + return DateTimeUtil.microsToYears(DateTimeUtil.isoTimestampToMicros(timestampNtzString)); + } + + private static int dateToMonths(String dateString) { + return DateTimeUtil.daysToMonths(DateTimeUtil.isoDateToDays(dateString)); + } + + private static int timestampToMonths(String timestampString) { + return DateTimeUtil.microsToMonths(DateTimeUtil.isoTimestamptzToMicros(timestampString)); + } + + private static int timestampNtzToMonths(String timestampNtzString) { + return DateTimeUtil.microsToMonths(DateTimeUtil.isoTimestampToMicros(timestampNtzString)); + } + + private static int dateToDays(String dateString) { + return DateTimeUtil.isoDateToDays(dateString); + } + + private static int timestampToDays(String timestampString) { + return DateTimeUtil.microsToDays(DateTimeUtil.isoTimestamptzToMicros(timestampString)); + } + + private static int timestampNtzToDays(String timestampNtzString) { + return DateTimeUtil.microsToDays(DateTimeUtil.isoTimestampToMicros(timestampNtzString)); + } + + private static int timestampToHours(String timestampString) { + return DateTimeUtil.microsToHours(DateTimeUtil.isoTimestamptzToMicros(timestampString)); + } + + private static int timestampNtzToHours(String timestampNtzString) { + return DateTimeUtil.microsToHours(DateTimeUtil.isoTimestampToMicros(timestampNtzString)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkValueConverter.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkValueConverter.java new file mode 100644 index 000000000000..c7a2e6c18fca --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkValueConverter.java @@ -0,0 +1,94 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.junit.jupiter.api.Test; + +public class TestSparkValueConverter { + @Test + public void testSparkNullMapConvert() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 5, + "locations", + Types.MapType.ofOptional( + 6, + 7, + Types.StringType.get(), + Types.StructType.of( + Types.NestedField.required(1, "lat", Types.FloatType.get()), + Types.NestedField.required(2, "long", Types.FloatType.get()))))); + + assertCorrectNullConversion(schema); + } + + @Test + public void testSparkNullListConvert() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 5, "locations", Types.ListType.ofOptional(6, Types.StringType.get()))); + + assertCorrectNullConversion(schema); + } + + @Test + public void testSparkNullStructConvert() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 5, + "location", + Types.StructType.of( + Types.NestedField.required(1, "lat", Types.FloatType.get()), + Types.NestedField.required(2, "long", Types.FloatType.get())))); + + assertCorrectNullConversion(schema); + } + + @Test + public void testSparkNullPrimitiveConvert() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(5, "location", Types.StringType.get())); + assertCorrectNullConversion(schema); + } + + private void assertCorrectNullConversion(Schema schema) { + Row sparkRow = RowFactory.create(1, null); + Record record = GenericRecord.create(schema); + record.set(0, 1); + assertThat(SparkValueConverter.convert(schema, sparkRow)) + .as("Round-trip conversion should produce original value") + .isEqualTo(record); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java new file mode 100644 index 000000000000..61aacfa4589d --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java @@ -0,0 +1,643 @@ +/* + * 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 static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DELETE_AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.DELETE_AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DELETE_DISTRIBUTION_MODE; +import static org.apache.iceberg.TableProperties.DELETE_ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.DELETE_ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.DELETE_PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.DELETE_PARQUET_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.MERGE_DISTRIBUTION_MODE; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.UPDATE_DISTRIBUTION_MODE; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE; +import static org.apache.iceberg.spark.SparkSQLProperties.COMPRESSION_CODEC; +import static org.apache.iceberg.spark.SparkSQLProperties.COMPRESSION_LEVEL; +import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.DELETE; +import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.MERGE; +import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.UPDATE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.time.Duration; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.UpdateProperties; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.internal.SQLConf; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkWriteConf extends TestBaseWithCatalog { + + @BeforeEach + public void before() { + super.before(); + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date, days(ts))", + tableName); + } + + @AfterEach + public void after() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testOptionCaseInsensitive() { + Table table = validationCatalog.loadTable(tableIdent); + Map options = ImmutableMap.of("option", "value"); + SparkConfParser parser = new SparkConfParser(spark, table, options); + String parsedValue = parser.stringConf().option("oPtIoN").parseOptional(); + assertThat(parsedValue).isEqualTo("value"); + } + + @TestTemplate + public void testCamelCaseSparkSessionConf() { + Table table = validationCatalog.loadTable(tableIdent); + String confName = "spark.sql.iceberg.some-int-conf"; + String sparkConfName = "spark.sql.iceberg.someIntConf"; + + withSQLConf( + ImmutableMap.of(sparkConfName, "1"), + () -> { + SparkConfParser parser = new SparkConfParser(spark, table, ImmutableMap.of()); + Integer value = parser.intConf().sessionConf(confName).parseOptional(); + assertThat(value).isEqualTo(1); + }); + } + + @TestTemplate + public void testCamelCaseSparkOption() { + Table table = validationCatalog.loadTable(tableIdent); + String option = "some-int-option"; + String sparkOption = "someIntOption"; + Map options = ImmutableMap.of(sparkOption, "1"); + SparkConfParser parser = new SparkConfParser(spark, table, options); + Integer value = parser.intConf().option(option).parseOptional(); + assertThat(value).isEqualTo(1); + } + + @TestTemplate + public void testDurationConf() { + Table table = validationCatalog.loadTable(tableIdent); + String confName = "spark.sql.iceberg.some-duration-conf"; + + withSQLConf( + ImmutableMap.of(confName, "10s"), + () -> { + SparkConfParser parser = new SparkConfParser(spark, table, ImmutableMap.of()); + Duration duration = parser.durationConf().sessionConf(confName).parseOptional(); + assertThat(duration).hasSeconds(10); + }); + + withSQLConf( + ImmutableMap.of(confName, "2m"), + () -> { + SparkConfParser parser = new SparkConfParser(spark, table, ImmutableMap.of()); + Duration duration = parser.durationConf().sessionConf(confName).parseOptional(); + assertThat(duration).hasMinutes(2); + }); + } + + @TestTemplate + public void testDeleteGranularityDefault() { + Table table = validationCatalog.loadTable(tableIdent); + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + + DeleteGranularity value = writeConf.deleteGranularity(); + assertThat(value).isEqualTo(DeleteGranularity.FILE); + } + + @TestTemplate + public void testDeleteGranularityTableProperty() { + Table table = validationCatalog.loadTable(tableIdent); + + table + .updateProperties() + .set(TableProperties.DELETE_GRANULARITY, DeleteGranularity.PARTITION.toString()) + .commit(); + + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + + DeleteGranularity value = writeConf.deleteGranularity(); + assertThat(value).isEqualTo(DeleteGranularity.PARTITION); + } + + @TestTemplate + public void testDeleteGranularityWriteOption() { + Table table = validationCatalog.loadTable(tableIdent); + + table + .updateProperties() + .set(TableProperties.DELETE_GRANULARITY, DeleteGranularity.PARTITION.toString()) + .commit(); + + Map options = + ImmutableMap.of(SparkWriteOptions.DELETE_GRANULARITY, DeleteGranularity.FILE.toString()); + + SparkWriteConf writeConf = new SparkWriteConf(spark, table, options); + + DeleteGranularity value = writeConf.deleteGranularity(); + assertThat(value).isEqualTo(DeleteGranularity.FILE); + } + + @TestTemplate + public void testDeleteGranularityInvalidValue() { + Table table = validationCatalog.loadTable(tableIdent); + + table.updateProperties().set(TableProperties.DELETE_GRANULARITY, "invalid").commit(); + + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + + assertThatThrownBy(writeConf::deleteGranularity) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Unknown delete granularity"); + } + + @TestTemplate + public void testAdvisoryPartitionSize() { + Table table = validationCatalog.loadTable(tableIdent); + + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + + long value1 = writeConf.writeRequirements().advisoryPartitionSize(); + assertThat(value1).isGreaterThan(64L * 1024 * 1024).isLessThan(2L * 1024 * 1024 * 1024); + + spark.conf().set(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES().key(), "2GB"); + long value2 = writeConf.writeRequirements().advisoryPartitionSize(); + assertThat(value2).isEqualTo(2L * 1024 * 1024 * 1024); + + spark.conf().set(SQLConf.ADVISORY_PARTITION_SIZE_IN_BYTES().key(), "10MB"); + long value3 = writeConf.writeRequirements().advisoryPartitionSize(); + assertThat(value3).isGreaterThan(10L * 1024 * 1024); + } + + @TestTemplate + public void testSparkWriteConfDistributionDefault() { + Table table = validationCatalog.loadTable(tableIdent); + + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + + checkMode(DistributionMode.HASH, writeConf); + } + + @TestTemplate + public void testSparkWriteConfDistributionModeWithWriteOption() { + Table table = validationCatalog.loadTable(tableIdent); + + Map writeOptions = + ImmutableMap.of(SparkWriteOptions.DISTRIBUTION_MODE, DistributionMode.NONE.modeName()); + + SparkWriteConf writeConf = new SparkWriteConf(spark, table, writeOptions); + checkMode(DistributionMode.NONE, writeConf); + } + + @TestTemplate + public void testSparkWriteConfDistributionModeWithSessionConfig() { + withSQLConf( + ImmutableMap.of(SparkSQLProperties.DISTRIBUTION_MODE, DistributionMode.NONE.modeName()), + () -> { + Table table = validationCatalog.loadTable(tableIdent); + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + checkMode(DistributionMode.NONE, writeConf); + }); + } + + @TestTemplate + public void testSparkWriteConfDistributionModeWithTableProperties() { + Table table = validationCatalog.loadTable(tableIdent); + + table + .updateProperties() + .set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE) + .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE) + .set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE) + .set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_NONE) + .commit(); + + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + checkMode(DistributionMode.NONE, writeConf); + } + + @TestTemplate + public void testSparkWriteConfDistributionModeWithTblPropAndSessionConfig() { + withSQLConf( + ImmutableMap.of(SparkSQLProperties.DISTRIBUTION_MODE, DistributionMode.NONE.modeName()), + () -> { + Table table = validationCatalog.loadTable(tableIdent); + + table + .updateProperties() + .set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE) + .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE) + .set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE) + .set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_RANGE) + .commit(); + + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + // session config overwrite the table properties + checkMode(DistributionMode.NONE, writeConf); + }); + } + + @TestTemplate + public void testSparkWriteConfDistributionModeWithWriteOptionAndSessionConfig() { + withSQLConf( + ImmutableMap.of(SparkSQLProperties.DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()), + () -> { + Table table = validationCatalog.loadTable(tableIdent); + + Map writeOptions = + ImmutableMap.of( + SparkWriteOptions.DISTRIBUTION_MODE, DistributionMode.NONE.modeName()); + + SparkWriteConf writeConf = new SparkWriteConf(spark, table, writeOptions); + // write options overwrite the session config + checkMode(DistributionMode.NONE, writeConf); + }); + } + + @TestTemplate + public void testSparkWriteConfDistributionModeWithEverything() { + withSQLConf( + ImmutableMap.of(SparkSQLProperties.DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()), + () -> { + Table table = validationCatalog.loadTable(tableIdent); + + Map writeOptions = + ImmutableMap.of( + SparkWriteOptions.DISTRIBUTION_MODE, DistributionMode.NONE.modeName()); + + table + .updateProperties() + .set(WRITE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH) + .set(DELETE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH) + .set(UPDATE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH) + .set(MERGE_DISTRIBUTION_MODE, WRITE_DISTRIBUTION_MODE_HASH) + .commit(); + + SparkWriteConf writeConf = new SparkWriteConf(spark, table, writeOptions); + // write options take the highest priority + checkMode(DistributionMode.NONE, writeConf); + }); + } + + @TestTemplate + public void testSparkConfOverride() { + List>> propertiesSuites = + Lists.newArrayList( + Lists.newArrayList( + ImmutableMap.of(COMPRESSION_CODEC, "zstd", COMPRESSION_LEVEL, "3"), + ImmutableMap.of( + DEFAULT_FILE_FORMAT, + "parquet", + DELETE_DEFAULT_FILE_FORMAT, + "parquet", + TableProperties.PARQUET_COMPRESSION, + "gzip", + TableProperties.DELETE_PARQUET_COMPRESSION, + "snappy"), + ImmutableMap.of( + DELETE_PARQUET_COMPRESSION, + "zstd", + PARQUET_COMPRESSION, + "zstd", + PARQUET_COMPRESSION_LEVEL, + "3", + DELETE_PARQUET_COMPRESSION_LEVEL, + "3")), + Lists.newArrayList( + ImmutableMap.of( + COMPRESSION_CODEC, + "zstd", + SparkSQLProperties.COMPRESSION_STRATEGY, + "compression"), + ImmutableMap.of( + DEFAULT_FILE_FORMAT, + "orc", + DELETE_DEFAULT_FILE_FORMAT, + "orc", + ORC_COMPRESSION, + "zlib", + DELETE_ORC_COMPRESSION, + "snappy"), + ImmutableMap.of( + DELETE_ORC_COMPRESSION, + "zstd", + ORC_COMPRESSION, + "zstd", + DELETE_ORC_COMPRESSION_STRATEGY, + "compression", + ORC_COMPRESSION_STRATEGY, + "compression")), + Lists.newArrayList( + ImmutableMap.of(COMPRESSION_CODEC, "zstd", COMPRESSION_LEVEL, "9"), + ImmutableMap.of( + DEFAULT_FILE_FORMAT, + "avro", + DELETE_DEFAULT_FILE_FORMAT, + "avro", + AVRO_COMPRESSION, + "gzip", + DELETE_AVRO_COMPRESSION, + "snappy"), + ImmutableMap.of( + DELETE_AVRO_COMPRESSION, + "zstd", + AVRO_COMPRESSION, + "zstd", + AVRO_COMPRESSION_LEVEL, + "9", + DELETE_AVRO_COMPRESSION_LEVEL, + "9"))); + for (List> propertiesSuite : propertiesSuites) { + testWriteProperties(propertiesSuite); + } + } + + @TestTemplate + public void testExtraSnapshotMetadataReflectsSessionConfig() { + withSQLConf( + ImmutableMap.of("spark.sql.iceberg.snapshot-property.test-key", "session-value"), + () -> { + Table table = validationCatalog.loadTable(tableIdent); + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + + Map metadata = writeConf.extraSnapshotMetadata(); + + assertThat(metadata).containsEntry("test-key", "session-value"); + }); + } + + @TestTemplate + public void testExtraSnapshotMetadataWriteOptionsOverrideSessionConfig() { + withSQLConf( + ImmutableMap.of("spark.sql.iceberg.snapshot-property.test-key", "session-value"), + () -> { + Table table = validationCatalog.loadTable(tableIdent); + Map writeOptions = + ImmutableMap.of("snapshot-property.test-key", "write-option-value"); + SparkWriteConf writeConf = new SparkWriteConf(spark, table, writeOptions); + + Map metadata = writeConf.extraSnapshotMetadata(); + + // Assert that writeOptions take precedence over session config + assertThat(metadata).containsEntry("test-key", "write-option-value"); + }); + } + + @TestTemplate + public void testExtraSnapshotMetadataPersistedOnWrite() { + String propertyKey = "test-key"; + String propertyValue = "session-value"; + + withSQLConf( + ImmutableMap.of("spark.sql.iceberg.snapshot-property." + propertyKey, propertyValue), + () -> { + spark.sql( + String.format( + "INSERT INTO %s VALUES (1, 'a', DATE '2021-01-01', TIMESTAMP '2021-01-01 00:00:00')", + tableName)); + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.currentSnapshot().summary()).containsEntry(propertyKey, propertyValue); + }); + } + + @TestTemplate + public void testDataPropsDefaultsAsDeleteProps() { + List>> propertiesSuites = + Lists.newArrayList( + Lists.newArrayList( + ImmutableMap.of(), + ImmutableMap.of( + DEFAULT_FILE_FORMAT, + "parquet", + DELETE_DEFAULT_FILE_FORMAT, + "parquet", + PARQUET_COMPRESSION, + "zstd", + PARQUET_COMPRESSION_LEVEL, + "5"), + ImmutableMap.of( + DELETE_PARQUET_COMPRESSION, + "zstd", + PARQUET_COMPRESSION, + "zstd", + PARQUET_COMPRESSION_LEVEL, + "5", + DELETE_PARQUET_COMPRESSION_LEVEL, + "5")), + Lists.newArrayList( + ImmutableMap.of(), + ImmutableMap.of( + DEFAULT_FILE_FORMAT, + "orc", + DELETE_DEFAULT_FILE_FORMAT, + "orc", + ORC_COMPRESSION, + "snappy", + ORC_COMPRESSION_STRATEGY, + "speed"), + ImmutableMap.of( + DELETE_ORC_COMPRESSION, + "snappy", + ORC_COMPRESSION, + "snappy", + ORC_COMPRESSION_STRATEGY, + "speed", + DELETE_ORC_COMPRESSION_STRATEGY, + "speed")), + Lists.newArrayList( + ImmutableMap.of(), + ImmutableMap.of( + DEFAULT_FILE_FORMAT, + "avro", + DELETE_DEFAULT_FILE_FORMAT, + "avro", + AVRO_COMPRESSION, + "snappy", + AVRO_COMPRESSION_LEVEL, + "9"), + ImmutableMap.of( + DELETE_AVRO_COMPRESSION, + "snappy", + AVRO_COMPRESSION, + "snappy", + AVRO_COMPRESSION_LEVEL, + "9", + DELETE_AVRO_COMPRESSION_LEVEL, + "9"))); + for (List> propertiesSuite : propertiesSuites) { + testWriteProperties(propertiesSuite); + } + } + + @TestTemplate + public void testDeleteFileWriteConf() { + List>> propertiesSuites = + Lists.newArrayList( + Lists.newArrayList( + ImmutableMap.of(), + ImmutableMap.of( + DEFAULT_FILE_FORMAT, + "parquet", + DELETE_DEFAULT_FILE_FORMAT, + "parquet", + TableProperties.PARQUET_COMPRESSION, + "zstd", + PARQUET_COMPRESSION_LEVEL, + "5", + DELETE_PARQUET_COMPRESSION_LEVEL, + "6"), + ImmutableMap.of( + DELETE_PARQUET_COMPRESSION, + "zstd", + PARQUET_COMPRESSION, + "zstd", + PARQUET_COMPRESSION_LEVEL, + "5", + DELETE_PARQUET_COMPRESSION_LEVEL, + "6")), + Lists.newArrayList( + ImmutableMap.of(), + ImmutableMap.of( + DEFAULT_FILE_FORMAT, + "orc", + DELETE_DEFAULT_FILE_FORMAT, + "orc", + ORC_COMPRESSION, + "snappy", + ORC_COMPRESSION_STRATEGY, + "speed", + DELETE_ORC_COMPRESSION, + "zstd", + DELETE_ORC_COMPRESSION_STRATEGY, + "compression"), + ImmutableMap.of( + DELETE_ORC_COMPRESSION, + "zstd", + ORC_COMPRESSION, + "snappy", + ORC_COMPRESSION_STRATEGY, + "speed", + DELETE_ORC_COMPRESSION_STRATEGY, + "compression")), + Lists.newArrayList( + ImmutableMap.of(), + ImmutableMap.of( + DEFAULT_FILE_FORMAT, + "avro", + DELETE_DEFAULT_FILE_FORMAT, + "avro", + AVRO_COMPRESSION, + "snappy", + AVRO_COMPRESSION_LEVEL, + "9", + DELETE_AVRO_COMPRESSION, + "zstd", + DELETE_AVRO_COMPRESSION_LEVEL, + "16"), + ImmutableMap.of( + DELETE_AVRO_COMPRESSION, + "zstd", + AVRO_COMPRESSION, + "snappy", + AVRO_COMPRESSION_LEVEL, + "9", + DELETE_AVRO_COMPRESSION_LEVEL, + "16"))); + for (List> propertiesSuite : propertiesSuites) { + testWriteProperties(propertiesSuite); + } + } + + @TestTemplate + public void testDVWriteConf() { + Table table = validationCatalog.loadTable(tableIdent); + table.updateProperties().set(TableProperties.FORMAT_VERSION, "3").commit(); + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + assertThat(writeConf.deleteFileFormat()).isEqualTo(FileFormat.PUFFIN); + } + + private void testWriteProperties(List> propertiesSuite) { + withSQLConf( + propertiesSuite.get(0), + () -> { + Table table = validationCatalog.loadTable(tableIdent); + Map tableProperties = propertiesSuite.get(1); + UpdateProperties updateProperties = table.updateProperties(); + for (Map.Entry entry : tableProperties.entrySet()) { + updateProperties.set(entry.getKey(), entry.getValue()); + } + + updateProperties.commit(); + + SparkWriteConf writeConf = new SparkWriteConf(spark, table, ImmutableMap.of()); + Map writeProperties = writeConf.writeProperties(); + Map expectedProperties = propertiesSuite.get(2); + assertThat(writeConf.writeProperties()).hasSameSizeAs(expectedProperties); + for (Map.Entry entry : writeProperties.entrySet()) { + assertThat(expectedProperties).containsEntry(entry.getKey(), entry.getValue()); + } + + table.refresh(); + updateProperties = table.updateProperties(); + for (Map.Entry entry : tableProperties.entrySet()) { + updateProperties.remove(entry.getKey()); + } + + updateProperties.commit(); + }); + } + + private void checkMode(DistributionMode expectedMode, SparkWriteConf writeConf) { + assertThat(writeConf.distributionMode()).isEqualTo(expectedMode); + assertThat(writeConf.copyOnWriteDistributionMode(DELETE)).isEqualTo(expectedMode); + assertThat(writeConf.positionDeltaDistributionMode(DELETE)).isEqualTo(expectedMode); + assertThat(writeConf.copyOnWriteDistributionMode(UPDATE)).isEqualTo(expectedMode); + assertThat(writeConf.positionDeltaDistributionMode(UPDATE)).isEqualTo(expectedMode); + assertThat(writeConf.copyOnWriteDistributionMode(MERGE)).isEqualTo(expectedMode); + assertThat(writeConf.positionDeltaDistributionMode(MERGE)).isEqualTo(expectedMode); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java new file mode 100644 index 000000000000..303411eb7ddd --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java @@ -0,0 +1,309 @@ +/* + * 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.actions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionStatisticsFile; +import org.apache.iceberg.PartitionStats; +import org.apache.iceberg.PartitionStatsHandler; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.types.Types; +import org.assertj.core.groups.Tuple; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestComputePartitionStatsAction extends CatalogTestBase { + + private static final int DEFAULT_SPEC_ID = 0; + private static final long DEFAULT_POS_DEL_RECORD_COUNT = 0L; + private static final int DEFAULT_POS_DEL_FILE_COUNT = 0; + private static final long DEFAULT_EQ_DEL_RECORD_COUNT = 0L; + private static final int DEFAULT_EQ_DEL_FILE_COUNT = 0; + private static final Long DEFAULT_TOTAL_RECORD_COUNT = null; + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void emptyTable() { + createPartitionedTable(); + Table table = validationCatalog.loadTable(tableIdent); + ComputePartitionStatsSparkAction.Result result = + SparkActions.get().computePartitionStats(table).execute(); + assertThat(result.statisticsFile()).isNull(); + } + + @TestTemplate + public void emptyBranch() { + createPartitionedTable(); + Table table = validationCatalog.loadTable(tableIdent); + table.manageSnapshots().createBranch("b1").commit(); + ComputePartitionStatsSparkAction.Result result = + SparkActions.get() + .computePartitionStats(table) + .snapshot(table.refs().get("b1").snapshotId()) + .execute(); + assertThat(result.statisticsFile()).isNull(); + } + + @TestTemplate + public void invalidSnapshot() { + createPartitionedTable(); + Table table = validationCatalog.loadTable(tableIdent); + assertThatThrownBy( + () -> SparkActions.get().computePartitionStats(table).snapshot(42L).execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Snapshot not found: 42"); + } + + @TestTemplate + public void partitionStatsComputeOnLatestSnapshot() throws IOException { + createPartitionedTable(); + // foo, A -> 4 records, + // foo, B -> 2 records, + // bar, A -> 2 records, + // bar, B -> 1 record + sql( + "INSERT into %s values (0, 'foo', 'A'), (1, 'foo', 'A'), (2, 'foo', 'B'), (3, 'foo', 'B')", + tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snapshot1 = table.currentSnapshot(); + sql("INSERT into %s values(4, 'bar', 'A'), (5, 'bar', 'A'), (6, 'bar', 'B')", tableName); + table.refresh(); + Snapshot snapshot2 = table.currentSnapshot(); + sql("INSERT into %s values(7, 'foo', 'A'), (8, 'foo', 'A')", tableName); + // snapshot3 is unused for partition stats as the same partition is modified by snapshot4 + + // delete one record of foo, A + sql("DELETE FROM %s WHERE c1=1", tableName); + table.refresh(); + Snapshot snapshot4 = table.currentSnapshot(); + + assertThat(table.partitionStatisticsFiles()).isEmpty(); + + PartitionStatisticsFile statisticsFile = + SparkActions.get().computePartitionStats(table).execute().statisticsFile(); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.snapshotId()).isEqualTo(snapshot4.snapshotId()); + // check table metadata registration + assertThat(table.partitionStatisticsFiles()).containsExactly(statisticsFile); + + Types.StructType partitionType = Partitioning.partitionType(table); + Schema dataSchema = PartitionStatsHandler.schema(partitionType, 2); + validatePartitionStats( + statisticsFile, + dataSchema, + Tuple.tuple( + partitionRecord(partitionType, "foo", "A"), + DEFAULT_SPEC_ID, + 4L, // dataRecordCount (total 4 records for this partition) + 2, // dataFileCount + totalDataFileSizeInBytes("foo", "A"), + 1L, // positionDeleteRecordCount (from delete operation) + 1, // positionDeleteFileCount (from delete operation) + DEFAULT_EQ_DEL_RECORD_COUNT, + DEFAULT_EQ_DEL_FILE_COUNT, + DEFAULT_TOTAL_RECORD_COUNT, + snapshot4.timestampMillis(), // lastUpdatedAt (last modified by snapshot4) + snapshot4.snapshotId() // lastUpdatedSnapshotId + ), + Tuple.tuple( + partitionRecord(partitionType, "foo", "B"), + DEFAULT_SPEC_ID, + 2L, // dataRecordCount + 1, // dataFileCount + totalDataFileSizeInBytes("foo", "B"), + DEFAULT_POS_DEL_RECORD_COUNT, + DEFAULT_POS_DEL_FILE_COUNT, + DEFAULT_EQ_DEL_RECORD_COUNT, + DEFAULT_EQ_DEL_FILE_COUNT, + DEFAULT_TOTAL_RECORD_COUNT, + snapshot1.timestampMillis(), // lastUpdatedAt (added by snapshot1) + snapshot1.snapshotId() // lastUpdatedSnapshotId + ), + Tuple.tuple( + partitionRecord(partitionType, "bar", "A"), + DEFAULT_SPEC_ID, + 2L, // dataRecordCount + 1, // dataFileCount + totalDataFileSizeInBytes("bar", "A"), + DEFAULT_POS_DEL_RECORD_COUNT, + DEFAULT_POS_DEL_FILE_COUNT, + DEFAULT_EQ_DEL_RECORD_COUNT, + DEFAULT_EQ_DEL_FILE_COUNT, + DEFAULT_TOTAL_RECORD_COUNT, + snapshot2.timestampMillis(), // lastUpdatedAt (added by snapshot2) + snapshot2.snapshotId() // lastUpdatedSnapshotId + ), + Tuple.tuple( + partitionRecord(partitionType, "bar", "B"), + DEFAULT_SPEC_ID, + 1L, // dataRecordCount + 1, // dataFileCount + totalDataFileSizeInBytes("bar", "B"), + DEFAULT_POS_DEL_RECORD_COUNT, + DEFAULT_POS_DEL_FILE_COUNT, + DEFAULT_EQ_DEL_RECORD_COUNT, + DEFAULT_EQ_DEL_FILE_COUNT, + DEFAULT_TOTAL_RECORD_COUNT, + snapshot2.timestampMillis(), // lastUpdatedAt + snapshot2.snapshotId() // lastUpdatedSnapshotId + )); + } + + @TestTemplate + public void partitionStatsComputeOnSnapshot() throws IOException { + createPartitionedTableV1(); + // foo, A -> 2 records, + // foo, B -> 1 record, + // bar, A -> 2 records, + sql("INSERT into %s values (0, 'foo', 'A'), (1, 'foo', 'A'), (2, 'foo', 'B')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snapshot1 = table.currentSnapshot(); + sql("INSERT into %s values(3, 'bar', 'A'), (4, 'bar', 'A')", tableName); + table.refresh(); + + assertThat(table.partitionStatisticsFiles()).isEmpty(); + + PartitionStatisticsFile statisticsFile = + SparkActions.get() + .computePartitionStats(table) + .snapshot(snapshot1.snapshotId()) + .execute() + .statisticsFile(); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + // should be mapped to snapshot1 instead of latest snapshot + assertThat(statisticsFile.snapshotId()).isEqualTo(snapshot1.snapshotId()); + // check table metadata registration + assertThat(table.partitionStatisticsFiles()).containsExactly(statisticsFile); + + Types.StructType partitionType = Partitioning.partitionType(table); + Schema dataSchema = PartitionStatsHandler.schema(partitionType, 2); + // should contain stats for only partitions of snapshot1 (no entry for partition bar, A) + validatePartitionStats( + statisticsFile, + dataSchema, + Tuple.tuple( + partitionRecord(partitionType, "foo", "A"), + DEFAULT_SPEC_ID, + 2L, // dataRecordCount + 1, // dataFileCount + totalDataFileSizeInBytes("foo", "A"), + DEFAULT_POS_DEL_RECORD_COUNT, + DEFAULT_POS_DEL_FILE_COUNT, + DEFAULT_EQ_DEL_RECORD_COUNT, + DEFAULT_EQ_DEL_FILE_COUNT, + DEFAULT_TOTAL_RECORD_COUNT, + snapshot1.timestampMillis(), // lastUpdatedAt + snapshot1.snapshotId()), // lastUpdatedSnapshotId + Tuple.tuple( + partitionRecord(partitionType, "foo", "B"), + DEFAULT_SPEC_ID, + 1L, // dataRecordCount + 1, // dataFileCount + totalDataFileSizeInBytes("foo", "B"), + DEFAULT_POS_DEL_RECORD_COUNT, + DEFAULT_POS_DEL_FILE_COUNT, + DEFAULT_EQ_DEL_RECORD_COUNT, + DEFAULT_EQ_DEL_FILE_COUNT, + DEFAULT_TOTAL_RECORD_COUNT, + snapshot1.timestampMillis(), // lastUpdatedAt + snapshot1.snapshotId() // lastUpdatedSnapshotId + )); + + // try again on same snapshot + PartitionStatisticsFile newStatsFile = + SparkActions.get() + .computePartitionStats(table) + .snapshot(snapshot1.snapshotId()) + .execute() + .statisticsFile(); + assertThat(newStatsFile).isEqualTo(statisticsFile); + } + + private long totalDataFileSizeInBytes(String col1, String col2) { + return (long) + sql( + "SELECT sum(file_size_in_bytes) FROM %s.data_files WHERE partition.c2 = '%s' AND partition.c3 = '%s'", + tableName, col1, col2) + .get(0)[0]; + } + + private void createPartitionedTable() { + sql( + "CREATE TABLE %s (c1 int, c2 string, c3 string) USING iceberg PARTITIONED BY (c2, c3) TBLPROPERTIES('write.delete.mode'='merge-on-read')", + tableName); + } + + private void createPartitionedTableV1() { + sql( + "CREATE TABLE %s (c1 int, c2 string, c3 string) USING iceberg PARTITIONED BY (c2, c3) TBLPROPERTIES('format-version'='1')", + tableName); + } + + private void validatePartitionStats( + PartitionStatisticsFile result, Schema recordSchema, Tuple... expectedValues) + throws IOException { + // read the partition entries from the stats file + List partitionStats; + try (CloseableIterable recordIterator = + PartitionStatsHandler.readPartitionStatsFile( + recordSchema, Files.localInput(result.path()))) { + partitionStats = Lists.newArrayList(recordIterator); + } + + assertThat(partitionStats) + .extracting( + PartitionStats::partition, + PartitionStats::specId, + PartitionStats::dataRecordCount, + PartitionStats::dataFileCount, + PartitionStats::totalDataFileSizeInBytes, + PartitionStats::positionDeleteRecordCount, + PartitionStats::positionDeleteFileCount, + PartitionStats::equalityDeleteRecordCount, + PartitionStats::equalityDeleteFileCount, + PartitionStats::totalRecords, + PartitionStats::lastUpdatedAt, + PartitionStats::lastUpdatedSnapshotId) + .containsExactlyInAnyOrder(expectedValues); + } + + private StructLike partitionRecord(Types.StructType partitionType, String val1, String val2) { + GenericRecord record = GenericRecord.create(partitionType); + record.set(0, val1); + record.set(1, val2); + return record; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java new file mode 100644 index 000000000000..2252ccda62fe --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java @@ -0,0 +1,443 @@ +/* + * 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.actions; + +import static org.apache.iceberg.spark.actions.NDVSketchUtil.APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatNoException; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +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.Lists; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.iceberg.types.Types; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestComputeTableStatsAction extends CatalogTestBase { + private static final Types.StructType LEAF_STRUCT_TYPE = + Types.StructType.of( + optional(1, "leafLongCol", Types.LongType.get()), + optional(2, "leafDoubleCol", Types.DoubleType.get())); + + private static final Types.StructType NESTED_STRUCT_TYPE = + Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE)); + + private static final Schema NESTED_SCHEMA = + new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE)); + + private static final Schema SCHEMA_WITH_NESTED_COLUMN = + new Schema( + required(4, "nestedStructCol", NESTED_STRUCT_TYPE), + required(5, "stringCol", Types.StringType.get())); + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testLoadingTableDirectly() { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + sql("INSERT into %s values(1, 'abcd')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).execute(); + StatisticsFile statisticsFile = results.statisticsFile(); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(2); + } + + @TestTemplate + public void testComputeTableStatsAction() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + // To create multiple splits on the mapper + table + .updateProperties() + .set("read.split.target-size", "100") + .set("write.parquet.row-group-size-bytes", "100") + .commit(); + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark.createDataset(records, Encoders.bean(SimpleRecord.class)).writeTo(tableName).append(); + SparkActions actions = SparkActions.get(); + table.refresh(); + ComputeTableStats.Result results = + actions.computeTableStats(table).columns("id", "data").execute(); + assertThat(results).isNotNull(); + + List statisticsFiles = table.statisticsFiles(); + assertThat(statisticsFiles) + .singleElement() + .satisfies( + statisticsFile -> { + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()) + .hasSize(2) + .element(0) + .satisfies( + blobMetadata -> { + assertThat(blobMetadata.properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "4"); + }); + }); + } + + @TestTemplate + public void testComputeTableStatsActionWithoutExplicitColumns() + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).execute(); + assertThat(results).isNotNull(); + + assertThat(table.statisticsFiles()) + .singleElement() + .satisfies( + statisticsFile -> { + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()) + .hasSize(2) + .satisfiesExactly( + blobMetadata -> { + assertThat(blobMetadata.properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "4"); + }, + blobMetadata -> { + assertThat(blobMetadata.properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "4"); + }); + }); + } + + @TestTemplate + public void testComputeTableStatsForInvalidColumns() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + assertThatThrownBy(() -> actions.computeTableStats(table).columns("id1").execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Can't find column id1 in table"); + } + + @TestTemplate + public void testComputeTableStatsWithNoSnapshots() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result result = actions.computeTableStats(table).columns("id").execute(); + assertThat(result.statisticsFile()).isNull(); + } + + @TestTemplate + public void testComputeTableStatsWithNullValues() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + List records = + Lists.newArrayList( + new SimpleRecord(1, null), + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); + assertThat(results).isNotNull(); + + assertThat(table.statisticsFiles()) + .singleElement() + .satisfies( + statisticsFile -> { + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()) + .singleElement() + .satisfies( + blobMetadata -> { + assertThat(blobMetadata.properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "4"); + }); + }); + } + + @TestTemplate + public void testComputeTableStatsWithSnapshotHavingDifferentSchemas() + throws NoSuchTableException, ParseException { + SparkActions actions = SparkActions.get(); + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + long snapshotId1 = Spark3Util.loadIcebergTable(spark, tableName).currentSnapshot().snapshotId(); + // Snapshot id not specified + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + assertThatNoException() + .isThrownBy(() -> actions.computeTableStats(table).columns("data").execute()); + + sql("ALTER TABLE %s DROP COLUMN %s", tableName, "data"); + // Append data to create snapshot + sql("INSERT into %s values(1)", tableName); + table.refresh(); + long snapshotId2 = Spark3Util.loadIcebergTable(spark, tableName).currentSnapshot().snapshotId(); + + // Snapshot id specified + assertThatNoException() + .isThrownBy( + () -> actions.computeTableStats(table).snapshot(snapshotId1).columns("data").execute()); + + assertThatThrownBy( + () -> actions.computeTableStats(table).snapshot(snapshotId2).columns("data").execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Can't find column data in table"); + } + + @TestTemplate + public void testComputeTableStatsWhenSnapshotIdNotSpecified() + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); + + assertThat(results).isNotNull(); + + assertThat(table.statisticsFiles()) + .singleElement() + .satisfies( + statisticsFile -> { + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()) + .singleElement() + .satisfies( + blobMetadata -> { + assertThat(blobMetadata.properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "1"); + }); + }); + } + + @TestTemplate + public void testComputeTableStatsWithNestedSchema() + throws NoSuchTableException, ParseException, IOException { + List records = Lists.newArrayList(createNestedRecord()); + Table table = + validationCatalog.createTable( + tableIdent, + SCHEMA_WITH_NESTED_COLUMN, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); + table.newAppend().appendFile(dataFile).commit(); + + Table tbl = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + actions.computeTableStats(tbl).execute(); + + tbl.refresh(); + assertThat(tbl.statisticsFiles()) + .singleElement() + .satisfies( + statisticsFile -> { + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(1); + }); + } + + @TestTemplate + public void testComputeTableStatsWithNoComputableColumns() throws IOException { + List records = Lists.newArrayList(createNestedRecord()); + Table table = + validationCatalog.createTable( + tableIdent, NESTED_SCHEMA, PartitionSpec.unpartitioned(), ImmutableMap.of()); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); + table.newAppend().appendFile(dataFile).commit(); + + table.refresh(); + SparkActions actions = SparkActions.get(); + assertThatThrownBy(() -> actions.computeTableStats(table).execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("No columns found to compute stats"); + } + + @TestTemplate + public void testComputeTableStatsOnByteColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("byte_col", "TINYINT"); + } + + @TestTemplate + public void testComputeTableStatsOnShortColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("short_col", "SMALLINT"); + } + + @TestTemplate + public void testComputeTableStatsOnIntColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("int_col", "INT"); + } + + @TestTemplate + public void testComputeTableStatsOnLongColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("long_col", "BIGINT"); + } + + @TestTemplate + public void testComputeTableStatsOnTimestampColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("timestamp_col", "TIMESTAMP"); + } + + @TestTemplate + public void testComputeTableStatsOnTimestampNtzColumn() + throws NoSuchTableException, ParseException { + testComputeTableStats("timestamp_col", "TIMESTAMP_NTZ"); + } + + @TestTemplate + public void testComputeTableStatsOnDateColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("date_col", "DATE"); + } + + @TestTemplate + public void testComputeTableStatsOnDecimalColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("decimal_col", "DECIMAL(20, 2)"); + } + + @TestTemplate + public void testComputeTableStatsOnBinaryColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("binary_col", "BINARY"); + } + + public void testComputeTableStats(String columnName, String type) + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, %s %s) USING iceberg", tableName, columnName, type); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + Dataset dataDF = randomDataDF(table.schema()); + append(tableName, dataDF); + + SparkActions actions = SparkActions.get(); + table.refresh(); + ComputeTableStats.Result results = + actions.computeTableStats(table).columns(columnName).execute(); + assertThat(results).isNotNull(); + + assertThat(table.statisticsFiles()) + .singleElement() + .satisfies( + statisticsFile -> { + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()) + .singleElement() + .satisfies( + blobMetadata -> { + assertThat(blobMetadata.properties()) + .containsKey(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY); + }); + }); + } + + private GenericRecord createNestedRecord() { + GenericRecord record = GenericRecord.create(SCHEMA_WITH_NESTED_COLUMN); + GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE); + GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE); + leaf.set(0, 0L); + leaf.set(1, 0.0); + nested.set(0, leaf); + record.set(0, nested); + record.set(1, "data"); + return record; + } + + private Dataset randomDataDF(Schema schema) { + Iterable rows = RandomData.generateSpark(schema, 10, 0); + JavaRDD rowRDD = sparkContext.parallelize(Lists.newArrayList(rows)); + StructType rowSparkType = SparkSchemaUtil.convert(schema); + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + + return ((org.apache.spark.sql.classic.SparkSession) spark) + .internalCreateDataFrame(JavaRDD.toRDD(rowRDD), rowSparkType, false); + } + + private void append(String table, Dataset df) throws NoSuchTableException { + // fanout writes are enabled as write-time clustering is not supported without Spark extensions + df.coalesce(1).writeTo(table).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java new file mode 100644 index 000000000000..eb89b0a23274 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -0,0 +1,1056 @@ +/* + * 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.actions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.File; +import java.io.FilenameFilter; +import java.io.IOException; +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.commons.io.FileUtils; +import org.apache.commons.io.filefilter.TrueFileFilter; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.MigrateTable; +import org.apache.iceberg.actions.SnapshotTable; +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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.iceberg.spark.source.SparkTable; +import org.apache.iceberg.types.Types; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.MessageTypeParser; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.catalog.CatalogTable; +import org.apache.spark.sql.catalyst.catalog.CatalogTablePartition; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import scala.Option; +import scala.Some; +import scala.collection.JavaConverters; +import scala.collection.Seq; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestCreateActions extends CatalogTestBase { + private static final String CREATE_PARTITIONED_PARQUET = + "CREATE TABLE %s (id INT, data STRING) " + "using parquet PARTITIONED BY (id) LOCATION '%s'"; + private static final String CREATE_PARQUET = + "CREATE TABLE %s (id INT, data STRING) " + "using parquet LOCATION '%s'"; + private static final String CREATE_HIVE_EXTERNAL_PARQUET = + "CREATE EXTERNAL TABLE %s (data STRING) " + + "PARTITIONED BY (id INT) STORED AS parquet LOCATION '%s'"; + private static final String CREATE_HIVE_PARQUET = + "CREATE TABLE %s (data STRING) " + "PARTITIONED BY (id INT) STORED AS parquet"; + + private static final String NAMESPACE = "default"; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, type = {3}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] { + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "parquet-enabled", "true", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + ), + "hive" + }, + new Object[] { + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", "hadoop", + "default-namespace", "default", + "parquet-enabled", "true", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + ), + "hadoop" + }, + new Object[] { + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default"), + "hive" + }, + new Object[] { + "testhadoop", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", "hadoop", + "default-namespace", "default"), + "hadoop" + } + }; + } + + private final String baseTableName = "baseTable"; + @TempDir private File tableDir; + private String tableLocation; + + @Parameter(index = 3) + private String type; + + private TableCatalog catalog; + + @BeforeEach + @Override + public void before() { + super.before(); + this.tableLocation = tableDir.toURI().toString(); + this.catalog = (TableCatalog) spark.sessionState().catalogManager().catalog(catalogName); + + spark.conf().set("hive.exec.dynamic.partition", "true"); + spark.conf().set("hive.exec.dynamic.partition.mode", "nonstrict"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", false); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", false); + spark.sql(String.format("DROP TABLE IF EXISTS %s", baseTableName)); + + List expected = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + df.select("id", "data") + .orderBy("data") + .write() + .mode("append") + .option("path", tableLocation) + .saveAsTable(baseTableName); + } + + @AfterEach + public void after() throws IOException { + // Drop the hive table. + spark.sql(String.format("DROP TABLE IF EXISTS %s", baseTableName)); + spark.sessionState().catalogManager().reset(); + spark.conf().unset("spark.sql.catalog.spark_catalog.type"); + spark.conf().unset("spark.sql.catalog.spark_catalog.default-namespace"); + spark.conf().unset("spark.sql.catalog.spark_catalog.parquet-enabled"); + spark.conf().unset("spark.sql.catalog.spark_catalog.cache-enabled"); + } + + @TestTemplate + public void testMigratePartitioned() throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); + String source = sourceName("test_migrate_partitioned_table"); + String dest = source; + createSourceTable(CREATE_PARTITIONED_PARQUET, source); + assertMigratedFileCount(SparkActions.get().migrateTable(source), source, dest); + } + + @TestTemplate + public void testPartitionedTableWithUnRecoveredPartitions() throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); + String source = sourceName("test_unrecovered_partitions"); + String dest = source; + File location = Files.createTempDirectory(temp, "junit").toFile(); + sql(CREATE_PARTITIONED_PARQUET, source, location); + + // Data generation and partition addition + spark + .range(5) + .selectExpr("id", "cast(id as STRING) as data") + .write() + .partitionBy("id") + .mode(SaveMode.Overwrite) + .parquet(location.toURI().toString()); + sql("ALTER TABLE %s ADD PARTITION(id=0)", source); + + assertMigratedFileCount(SparkActions.get().migrateTable(source), source, dest); + } + + @TestTemplate + public void testPartitionedTableWithCustomPartitions() throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); + String source = sourceName("test_custom_parts"); + String dest = source; + File tblLocation = Files.createTempDirectory(temp, "junit").toFile(); + File partitionDataLoc = Files.createTempDirectory(temp, "junit").toFile(); + + // Data generation and partition addition + spark.sql(String.format(CREATE_PARTITIONED_PARQUET, source, tblLocation)); + spark + .range(10) + .selectExpr("cast(id as STRING) as data") + .write() + .mode(SaveMode.Overwrite) + .parquet(partitionDataLoc.toURI().toString()); + sql( + "ALTER TABLE %s ADD PARTITION(id=0) LOCATION '%s'", + source, partitionDataLoc.toURI().toString()); + assertMigratedFileCount(SparkActions.get().migrateTable(source), source, dest); + } + + @TestTemplate + public void testAddColumnOnMigratedTableAtEnd() throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); + String source = sourceName("test_add_column_migrated_table"); + String dest = source; + createSourceTable(CREATE_PARQUET, source); + List expected1 = sql("select *, null from %s order by id", source); + List expected2 = sql("select *, null, null from %s order by id", source); + + // migrate table + SparkActions.get().migrateTable(source).execute(); + SparkTable sparkTable = loadTable(dest); + Table table = sparkTable.table(); + + // test column addition on migrated table + Schema beforeSchema = table.schema(); + String newCol1 = "newCol1"; + sparkTable.table().updateSchema().addColumn(newCol1, Types.IntegerType.get()).commit(); + Schema afterSchema = table.schema(); + assertThat(beforeSchema.findField(newCol1)).isNull(); + assertThat(afterSchema.findField(newCol1)).isNotNull(); + + // reads should succeed without any exceptions + List results1 = sql("select * from %s order by id", dest); + assertThat(results1).isNotEmpty(); + assertEquals("Output must match", results1, expected1); + + String newCol2 = "newCol2"; + sql("ALTER TABLE %s ADD COLUMN %s INT", dest, newCol2); + StructType schema = spark.table(dest).schema(); + assertThat(schema.fieldNames()).contains(newCol2); + + // reads should succeed without any exceptions + List results2 = sql("select * from %s order by id", dest); + assertThat(results2).isNotEmpty(); + assertEquals("Output must match", results2, expected2); + } + + @TestTemplate + public void testAddColumnOnMigratedTableAtMiddle() throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); + String source = sourceName("test_add_column_migrated_table_middle"); + String dest = source; + createSourceTable(CREATE_PARQUET, source); + + // migrate table + SparkActions.get().migrateTable(source).execute(); + SparkTable sparkTable = loadTable(dest); + Table table = sparkTable.table(); + List expected = sql("select id, null, data from %s order by id", source); + + // test column addition on migrated table + Schema beforeSchema = table.schema(); + String newCol1 = "newCol"; + sparkTable + .table() + .updateSchema() + .addColumn("newCol", Types.IntegerType.get()) + .moveAfter(newCol1, "id") + .commit(); + Schema afterSchema = table.schema(); + assertThat(beforeSchema.findField(newCol1)).isNull(); + assertThat(afterSchema.findField(newCol1)).isNotNull(); + + // reads should succeed + List results = sql("select * from %s order by id", dest); + assertThat(results).isNotEmpty(); + assertEquals("Output must match", results, expected); + } + + @TestTemplate + public void removeColumnsAtEnd() throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); + String source = sourceName("test_remove_column_migrated_table"); + String dest = source; + + String colName1 = "newCol1"; + String colName2 = "newCol2"; + spark + .range(10) + .selectExpr("cast(id as INT)", "CAST(id as INT) " + colName1, "CAST(id as INT) " + colName2) + .write() + .mode(SaveMode.Overwrite) + .saveAsTable(dest); + List expected1 = sql("select id, %s from %s order by id", colName1, source); + List expected2 = sql("select id from %s order by id", source); + + // migrate table + SparkActions.get().migrateTable(source).execute(); + SparkTable sparkTable = loadTable(dest); + Table table = sparkTable.table(); + + // test column removal on migrated table + Schema beforeSchema = table.schema(); + sparkTable.table().updateSchema().deleteColumn(colName1).commit(); + Schema afterSchema = table.schema(); + assertThat(beforeSchema.findField(colName1)).isNotNull(); + assertThat(afterSchema.findField(colName1)).isNull(); + + // reads should succeed without any exceptions + List results1 = sql("select * from %s order by id", dest); + assertThat(results1).isNotEmpty(); + assertEquals("Output must match", expected1, results1); + + sql("ALTER TABLE %s DROP COLUMN %s", dest, colName2); + StructType schema = spark.table(dest).schema(); + assertThat(schema.fieldNames()).doesNotContain(colName2); + + // reads should succeed without any exceptions + List results2 = sql("select * from %s order by id", dest); + assertThat(results2).isNotEmpty(); + assertEquals("Output must match", expected2, results2); + } + + @TestTemplate + public void removeColumnFromMiddle() throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); + String source = sourceName("test_remove_column_migrated_table_from_middle"); + String dest = source; + String dropColumnName = "col1"; + + spark + .range(10) + .selectExpr( + "cast(id as INT)", "CAST(id as INT) as " + dropColumnName, "CAST(id as INT) as col2") + .write() + .mode(SaveMode.Overwrite) + .saveAsTable(dest); + List expected = sql("select id, col2 from %s order by id", source); + + // migrate table + SparkActions.get().migrateTable(source).execute(); + + // drop column + sql("ALTER TABLE %s DROP COLUMN %s", dest, "col1"); + StructType schema = spark.table(dest).schema(); + assertThat(schema.fieldNames()).doesNotContain(dropColumnName); + + // reads should return same output as that of non-iceberg table + List results = sql("select * from %s order by id", dest); + assertThat(results).isNotEmpty(); + assertEquals("Output must match", expected, results); + } + + @TestTemplate + public void testMigrateUnpartitioned() throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); + String source = sourceName("test_migrate_unpartitioned_table"); + String dest = source; + createSourceTable(CREATE_PARQUET, source); + assertMigratedFileCount(SparkActions.get().migrateTable(source), source, dest); + } + + @TestTemplate + public void testSnapshotPartitioned() throws Exception { + assumeThat(type) + .as("Cannot snapshot with arbitrary location in a hadoop based catalog") + .isNotEqualTo("hadoop"); + File location = Files.createTempDirectory(temp, "junit").toFile(); + String source = sourceName("test_snapshot_partitioned_table"); + String dest = destName("iceberg_snapshot_partitioned"); + createSourceTable(CREATE_PARTITIONED_PARQUET, source); + assertSnapshotFileCount( + SparkActions.get().snapshotTable(source).as(dest).tableLocation(location.toString()), + source, + dest); + assertIsolatedSnapshot(source, dest); + } + + @TestTemplate + public void testSnapshotUnpartitioned() throws Exception { + assumeThat(type) + .as("Cannot snapshot with arbitrary location in a hadoop based catalog") + .isNotEqualTo("hadoop"); + File location = Files.createTempDirectory(temp, "junit").toFile(); + String source = sourceName("test_snapshot_unpartitioned_table"); + String dest = destName("iceberg_snapshot_unpartitioned"); + createSourceTable(CREATE_PARQUET, source); + assertSnapshotFileCount( + SparkActions.get().snapshotTable(source).as(dest).tableLocation(location.toString()), + source, + dest); + assertIsolatedSnapshot(source, dest); + } + + @TestTemplate + public void testSnapshotHiveTable() throws Exception { + assumeThat(type) + .as("Cannot snapshot with arbitrary location in a hadoop based catalog") + .isNotEqualTo("hadoop"); + File location = Files.createTempDirectory(temp, "junit").toFile(); + String source = sourceName("snapshot_hive_table"); + String dest = destName("iceberg_snapshot_hive_table"); + createSourceTable(CREATE_HIVE_EXTERNAL_PARQUET, source); + assertSnapshotFileCount( + SparkActions.get().snapshotTable(source).as(dest).tableLocation(location.toString()), + source, + dest); + assertIsolatedSnapshot(source, dest); + } + + @TestTemplate + public void testMigrateHiveTable() throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + String source = sourceName("migrate_hive_table"); + String dest = source; + createSourceTable(CREATE_HIVE_EXTERNAL_PARQUET, source); + assertMigratedFileCount(SparkActions.get().migrateTable(source), source, dest); + } + + @TestTemplate + public void testSnapshotManagedHiveTable() throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + File location = Files.createTempDirectory(temp, "junit").toFile(); + String source = sourceName("snapshot_managed_hive_table"); + String dest = destName("iceberg_snapshot_managed_hive_table"); + createSourceTable(CREATE_HIVE_PARQUET, source); + assertSnapshotFileCount( + SparkActions.get().snapshotTable(source).as(dest).tableLocation(location.toString()), + source, + dest); + assertIsolatedSnapshot(source, dest); + } + + @TestTemplate + public void testMigrateManagedHiveTable() throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + File location = Files.createTempDirectory(temp, "junit").toFile(); + String source = sourceName("migrate_managed_hive_table"); + String dest = destName("iceberg_migrate_managed_hive_table"); + createSourceTable(CREATE_HIVE_PARQUET, source); + assertSnapshotFileCount( + SparkActions.get().snapshotTable(source).as(dest).tableLocation(location.toString()), + source, + dest); + } + + @TestTemplate + public void testProperties() throws Exception { + String source = sourceName("test_properties_table"); + String dest = destName("iceberg_properties"); + Map props = Maps.newHashMap(); + props.put("city", "New Orleans"); + props.put("note", "Jazz"); + createSourceTable(CREATE_PARQUET, source); + for (Map.Entry keyValue : props.entrySet()) { + spark.sql( + String.format( + "ALTER TABLE %s SET TBLPROPERTIES (\"%s\" = \"%s\")", + source, keyValue.getKey(), keyValue.getValue())); + } + assertSnapshotFileCount( + SparkActions.get().snapshotTable(source).as(dest).tableProperty("dogs", "sundance"), + source, + dest); + SparkTable table = loadTable(dest); + + Map expectedProps = Maps.newHashMap(); + expectedProps.putAll(props); + expectedProps.put("dogs", "sundance"); + + assertThat(table.properties()).containsAllEntriesOf(expectedProps); + } + + @TestTemplate + public void testSparkTableReservedProperties() throws Exception { + String destTableName = "iceberg_reserved_properties"; + String source = sourceName("test_reserved_properties_table"); + String dest = destName(destTableName); + createSourceTable(CREATE_PARQUET, source); + SnapshotTableSparkAction action = SparkActions.get().snapshotTable(source).as(dest); + action.tableProperty(TableProperties.FORMAT_VERSION, "1"); + assertSnapshotFileCount(action, source, dest); + SparkTable table = loadTable(dest); + // set sort orders + table.table().replaceSortOrder().asc("id").desc("data").commit(); + + String[] keys = {"provider", "format", "current-snapshot-id", "location", "sort-order"}; + + assertThat(table.properties()) + .as("Created table missing reserved properties") + .containsKeys(keys); + + assertThat(table.properties()) + .containsEntry("provider", "iceberg") + .containsEntry("format", "iceberg/parquet") + .hasEntrySatisfying("current-snapshot-id", id -> assertThat(id).isNotEqualTo("none")) + .hasEntrySatisfying("location", loc -> assertThat(loc).endsWith(destTableName)); + + assertThat(table.properties()).containsEntry("format-version", "1"); + table.table().updateProperties().set("format-version", "2").commit(); + assertThat(table.properties()).containsEntry("format-version", "2"); + + assertThat(table.properties()) + .containsEntry("sort-order", "id ASC NULLS FIRST, data DESC NULLS LAST") + .doesNotContainKey("identifier-fields"); + + table + .table() + .updateSchema() + .allowIncompatibleChanges() + .requireColumn("id") + .setIdentifierFields("id") + .commit(); + assertThat(table.properties()).containsEntry("identifier-fields", "[id]"); + } + + @TestTemplate + public void testSnapshotDefaultLocation() throws Exception { + String source = sourceName("test_snapshot_default"); + String dest = destName("iceberg_snapshot_default"); + createSourceTable(CREATE_PARTITIONED_PARQUET, source); + assertSnapshotFileCount(SparkActions.get().snapshotTable(source).as(dest), source, dest); + assertIsolatedSnapshot(source, dest); + } + + @TestTemplate + public void schemaEvolutionTestWithSparkAPI() throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); + + File location = Files.createTempDirectory(temp, "junit").toFile(); + String tblName = sourceName("schema_evolution_test"); + + // Data generation and partition addition + spark + .range(0, 5) + .selectExpr("CAST(id as INT) as col0", "CAST(id AS FLOAT) col2", "CAST(id AS LONG) col3") + .write() + .mode(SaveMode.Append) + .parquet(location.toURI().toString()); + Dataset rowDataset = + spark + .range(6, 10) + .selectExpr( + "CAST(id as INT) as col0", + "CAST(id AS STRING) col1", + "CAST(id AS FLOAT) col2", + "CAST(id AS LONG) col3"); + rowDataset.write().mode(SaveMode.Append).parquet(location.toURI().toString()); + spark + .read() + .schema(rowDataset.schema()) + .parquet(location.toURI().toString()) + .write() + .saveAsTable(tblName); + List expectedBeforeAddColumn = sql("SELECT * FROM %s ORDER BY col0", tblName); + List expectedAfterAddColumn = + sql("SELECT col0, null, col1, col2, col3 FROM %s ORDER BY col0", tblName); + + // Migrate table + SparkActions.get().migrateTable(tblName).execute(); + + // check if iceberg and non-iceberg output + List afterMigarteBeforeAddResults = sql("SELECT * FROM %s ORDER BY col0", tblName); + assertEquals("Output must match", expectedBeforeAddColumn, afterMigarteBeforeAddResults); + + // Update schema and check output correctness + SparkTable sparkTable = loadTable(tblName); + sparkTable + .table() + .updateSchema() + .addColumn("newCol", Types.IntegerType.get()) + .moveAfter("newCol", "col0") + .commit(); + List afterMigarteAfterAddResults = sql("SELECT * FROM %s ORDER BY col0", tblName); + assertEquals("Output must match", expectedAfterAddColumn, afterMigarteAfterAddResults); + } + + @TestTemplate + public void schemaEvolutionTestWithSparkSQL() throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + assumeThat(catalog.name()) + .as("Can only migrate from Spark Session Catalog") + .isEqualTo("spark_catalog"); + String tblName = sourceName("schema_evolution_test_sql"); + + // Data generation and partition addition + spark + .range(0, 5) + .selectExpr("CAST(id as INT) col0", "CAST(id AS FLOAT) col1", "CAST(id AS STRING) col2") + .write() + .mode(SaveMode.Append) + .saveAsTable(tblName); + sql("ALTER TABLE %s ADD COLUMN col3 INT", tblName); + spark + .range(6, 10) + .selectExpr( + "CAST(id AS INT) col0", + "CAST(id AS FLOAT) col1", + "CAST(id AS STRING) col2", + "CAST(id AS INT) col3") + .registerTempTable("tempdata"); + sql("INSERT INTO TABLE %s SELECT * FROM tempdata", tblName); + List expectedBeforeAddColumn = sql("SELECT * FROM %s ORDER BY col0", tblName); + List expectedAfterAddColumn = + sql("SELECT col0, null, col1, col2, col3 FROM %s ORDER BY col0", tblName); + + // Migrate table + SparkActions.get().migrateTable(tblName).execute(); + + // check if iceberg and non-iceberg output + List afterMigarteBeforeAddResults = sql("SELECT * FROM %s ORDER BY col0", tblName); + assertEquals("Output must match", expectedBeforeAddColumn, afterMigarteBeforeAddResults); + + // Update schema and check output correctness + SparkTable sparkTable = loadTable(tblName); + sparkTable + .table() + .updateSchema() + .addColumn("newCol", Types.IntegerType.get()) + .moveAfter("newCol", "col0") + .commit(); + List afterMigarteAfterAddResults = sql("SELECT * FROM %s ORDER BY col0", tblName); + assertEquals("Output must match", expectedAfterAddColumn, afterMigarteAfterAddResults); + } + + @TestTemplate + public void testHiveStyleThreeLevelList() throws Exception { + threeLevelList(true); + } + + @TestTemplate + public void testThreeLevelList() throws Exception { + threeLevelList(false); + } + + @TestTemplate + public void testHiveStyleThreeLevelListWithNestedStruct() throws Exception { + threeLevelListWithNestedStruct(true); + } + + @TestTemplate + public void testThreeLevelListWithNestedStruct() throws Exception { + threeLevelListWithNestedStruct(false); + } + + @TestTemplate + public void testHiveStyleThreeLevelLists() throws Exception { + threeLevelLists(true); + } + + @TestTemplate + public void testThreeLevelLists() throws Exception { + threeLevelLists(false); + } + + @TestTemplate + public void testHiveStyleStructOfThreeLevelLists() throws Exception { + structOfThreeLevelLists(true); + } + + @TestTemplate + public void testStructOfThreeLevelLists() throws Exception { + structOfThreeLevelLists(false); + } + + @TestTemplate + public void testTwoLevelList() throws IOException { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + + spark.conf().set("spark.sql.parquet.writeLegacyFormat", true); + + String tableName = sourceName("testTwoLevelList"); + File location = Files.createTempDirectory(temp, "junit").toFile(); + + StructType sparkSchema = + new StructType( + new StructField[] { + new StructField( + "col1", + new ArrayType( + new StructType( + new StructField[] { + new StructField("col2", DataTypes.IntegerType, false, Metadata.empty()) + }), + false), + true, + Metadata.empty()) + }); + + // even though this list looks like three level list, it is actually a 2-level list where the + // items are + // structs with 1 field. + String expectedParquetSchema = + "message spark_schema {\n" + + " optional group col1 (LIST) {\n" + + " repeated group array {\n" + + " required int32 col2;\n" + + " }\n" + + " }\n" + + "}\n"; + + // generate parquet file with required schema + List testData = Collections.singletonList("{\"col1\": [{\"col2\": 1}]}"); + spark + .read() + .schema(sparkSchema) + .json(JavaSparkContext.fromSparkContext(spark.sparkContext()).parallelize(testData)) + .coalesce(1) + .write() + .format("parquet") + .mode(SaveMode.Append) + .save(location.getPath()); + + File parquetFile = + Arrays.stream( + Preconditions.checkNotNull( + location.listFiles( + new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return name.endsWith("parquet"); + } + }))) + .findAny() + .get(); + + // verify generated parquet file has expected schema + ParquetFileReader pqReader = + ParquetFileReader.open( + HadoopInputFile.fromPath( + new Path(parquetFile.getPath()), spark.sessionState().newHadoopConf())); + MessageType schema = pqReader.getFooter().getFileMetaData().getSchema(); + assertThat(schema).isEqualTo(MessageTypeParser.parseMessageType(expectedParquetSchema)); + + // create sql table on top of it + sql( + "CREATE EXTERNAL TABLE %s (col1 ARRAY>)" + + " STORED AS parquet" + + " LOCATION '%s'", + tableName, location); + List expected = sql("select array(struct(1))"); + + // migrate table + SparkActions.get().migrateTable(tableName).execute(); + + // check migrated table is returning expected result + List results = sql("SELECT * FROM %s", tableName); + assertThat(results).isNotEmpty(); + assertEquals("Output must match", expected, results); + } + + private void threeLevelList(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); + + String tableName = sourceName(String.format("threeLevelList_%s", useLegacyMode)); + File location = Files.createTempDirectory(temp, "junit").toFile(); + sql( + "CREATE TABLE %s (col1 ARRAY>)" + " STORED AS parquet" + " LOCATION '%s'", + tableName, location); + + int testValue = 12345; + sql("INSERT INTO %s VALUES (ARRAY(STRUCT(%s)))", tableName, testValue); + List expected = sql(String.format("SELECT * FROM %s", tableName)); + + // migrate table + SparkActions.get().migrateTable(tableName).execute(); + + // check migrated table is returning expected result + List results = sql("SELECT * FROM %s", tableName); + assertThat(results).isNotEmpty(); + assertEquals("Output must match", expected, results); + } + + private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); + + String tableName = + sourceName(String.format("threeLevelListWithNestedStruct_%s", useLegacyMode)); + File location = Files.createTempDirectory(temp, "junit").toFile(); + sql( + "CREATE TABLE %s (col1 ARRAY>>)" + + " STORED AS parquet" + + " LOCATION '%s'", + tableName, location); + + int testValue = 12345; + sql("INSERT INTO %s VALUES (ARRAY(STRUCT(STRUCT(%s))))", tableName, testValue); + List expected = sql(String.format("SELECT * FROM %s", tableName)); + + // migrate table + SparkActions.get().migrateTable(tableName).execute(); + + // check migrated table is returning expected result + List results = sql("SELECT * FROM %s", tableName); + assertThat(results).isNotEmpty(); + assertEquals("Output must match", expected, results); + } + + private void threeLevelLists(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); + + String tableName = sourceName(String.format("threeLevelLists_%s", useLegacyMode)); + File location = Files.createTempDirectory(temp, "junit").toFile(); + sql( + "CREATE TABLE %s (col1 ARRAY>, col3 ARRAY>)" + + " STORED AS parquet" + + " LOCATION '%s'", + tableName, location); + + int testValue1 = 12345; + int testValue2 = 987654; + sql( + "INSERT INTO %s VALUES (ARRAY(STRUCT(%s)), ARRAY(STRUCT(%s)))", + tableName, testValue1, testValue2); + List expected = sql(String.format("SELECT * FROM %s", tableName)); + + // migrate table + SparkActions.get().migrateTable(tableName).execute(); + + // check migrated table is returning expected result + List results = sql("SELECT * FROM %s", tableName); + assertThat(results).isNotEmpty(); + assertEquals("Output must match", expected, results); + } + + private void structOfThreeLevelLists(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); + + String tableName = sourceName(String.format("structOfThreeLevelLists_%s", useLegacyMode)); + File location = Files.createTempDirectory(temp, "junit").toFile(); + sql( + "CREATE TABLE %s (col1 STRUCT>>)" + + " STORED AS parquet" + + " LOCATION '%s'", + tableName, location); + + int testValue1 = 12345; + sql("INSERT INTO %s VALUES (STRUCT(STRUCT(ARRAY(STRUCT(%s)))))", tableName, testValue1); + List expected = sql(String.format("SELECT * FROM %s", tableName)); + + // migrate table + SparkActions.get().migrateTable(tableName).execute(); + + // check migrated table is returning expected result + List results = sql("SELECT * FROM %s", tableName); + assertThat(results).isNotEmpty(); + assertEquals("Output must match", expected, results); + } + + private SparkTable loadTable(String name) throws NoSuchTableException, ParseException { + return (SparkTable) + catalog.loadTable(Spark3Util.catalogAndIdentifier(spark, name).identifier()); + } + + private CatalogTable loadSessionTable(String name) + throws NoSuchTableException, NoSuchDatabaseException, ParseException { + Identifier identifier = Spark3Util.catalogAndIdentifier(spark, name).identifier(); + Some namespace = Some.apply(identifier.namespace()[0]); + return spark + .sessionState() + .catalog() + .getTableMetadata(new TableIdentifier(identifier.name(), namespace)); + } + + private void createSourceTable(String createStatement, String tableName) + throws IOException, NoSuchTableException, NoSuchDatabaseException, ParseException { + File location = Files.createTempDirectory(temp, "junit").toFile(); + spark.sql(String.format(createStatement, tableName, location)); + CatalogTable table = loadSessionTable(tableName); + String format = table.provider().get(); + spark + .table(baseTableName) + .selectExpr(table.schema().names()) + .write() + .mode(SaveMode.Append) + .format(format) + .insertInto(tableName); + } + + // Counts the number of files in the source table, makes sure the same files exist in the + // destination table + private void assertMigratedFileCount(MigrateTable migrateAction, String source, String dest) + throws NoSuchTableException, NoSuchDatabaseException, ParseException { + long expectedFiles = expectedFilesCount(source); + MigrateTable.Result migratedFiles = migrateAction.execute(); + validateTables(source, dest); + assertThat(migratedFiles.migratedDataFilesCount()) + .as("Expected number of migrated files") + .isEqualTo(expectedFiles); + } + + // Counts the number of files in the source table, makes sure the same files exist in the + // destination table + private void assertSnapshotFileCount(SnapshotTable snapshotTable, String source, String dest) + throws NoSuchTableException, NoSuchDatabaseException, ParseException { + long expectedFiles = expectedFilesCount(source); + SnapshotTable.Result snapshotTableResult = snapshotTable.execute(); + validateTables(source, dest); + assertThat(snapshotTableResult.importedDataFilesCount()) + .as("Expected number of imported snapshot files") + .isEqualTo(expectedFiles); + } + + private void validateTables(String source, String dest) + throws NoSuchTableException, ParseException { + List expected = spark.table(source).collectAsList(); + SparkTable destTable = loadTable(dest); + assertThat(destTable.properties()).containsEntry(TableCatalog.PROP_PROVIDER, "iceberg"); + List actual = spark.table(dest).collectAsList(); + assertThat(actual) + .as( + String.format( + "Rows in migrated table did not match\nExpected :%s rows \nFound :%s", + expected, actual)) + .containsAll(expected); + assertThat(expected) + .as( + String.format( + "Rows in migrated table did not match\nExpected :%s rows \nFound :%s", + expected, actual)) + .containsAll(actual); + } + + private long expectedFilesCount(String source) + throws NoSuchDatabaseException, NoSuchTableException, ParseException { + CatalogTable sourceTable = loadSessionTable(source); + List uris; + if (sourceTable.partitionColumnNames().isEmpty()) { + uris = Lists.newArrayList(); + uris.add(sourceTable.location()); + } else { + Seq catalogTablePartitionSeq = + spark + .sessionState() + .catalog() + .listPartitions(sourceTable.identifier(), Option.apply(null)); + uris = + JavaConverters.seqAsJavaList(catalogTablePartitionSeq).stream() + .map(CatalogTablePartition::location) + .collect(Collectors.toList()); + } + return uris.stream() + .flatMap( + uri -> + FileUtils.listFiles( + Paths.get(uri).toFile(), TrueFileFilter.INSTANCE, TrueFileFilter.INSTANCE) + .stream()) + .filter(file -> !file.toString().endsWith("crc") && !file.toString().contains("_SUCCESS")) + .count(); + } + + // Insert records into the destination, makes sure those records exist and source table is + // unchanged + private void assertIsolatedSnapshot(String source, String dest) { + List expected = spark.sql(String.format("SELECT * FROM %s", source)).collectAsList(); + + List extraData = Lists.newArrayList(new SimpleRecord(4, "d")); + Dataset df = spark.createDataFrame(extraData, SimpleRecord.class); + df.write().format("iceberg").mode("append").saveAsTable(dest); + + List result = spark.sql(String.format("SELECT * FROM %s", source)).collectAsList(); + assertThat(result) + .as("No additional rows should be added to the original table") + .hasSameSizeAs(expected); + + List snapshot = + spark + .sql(String.format("SELECT * FROM %s WHERE id = 4 AND data = 'd'", dest)) + .collectAsList(); + assertThat(snapshot).as("Added row not found in snapshot").hasSize(1); + } + + private String sourceName(String source) { + return NAMESPACE + "." + catalog.name() + "_" + type + "_" + source; + } + + private String destName(String dest) { + if (catalog.name().equals("spark_catalog")) { + return NAMESPACE + "." + catalog.name() + "_" + type + "_" + dest; + } else { + return catalog.name() + "." + NAMESPACE + "." + catalog.name() + "_" + type + "_" + dest; + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java new file mode 100644 index 000000000000..1bd26f7aa1de --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestDeleteReachableFilesAction.java @@ -0,0 +1,403 @@ +/* + * 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.actions; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.StreamSupport; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileGenerationUtil; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.actions.ActionsProvider; +import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.actions.DeleteReachableFiles; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.hadoop.HadoopTables; +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.Lists; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestDeleteReachableFilesAction extends TestBase { + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + private static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + private static final int SHUFFLE_PARTITIONS = 2; + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + + static final DataFile FILE_A = + DataFiles.builder(SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartition(TestHelpers.Row.of(0)) + .withRecordCount(1) + .build(); + static final DataFile FILE_B = + DataFiles.builder(SPEC) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartition(TestHelpers.Row.of(1)) + .withRecordCount(1) + .build(); + static final DataFile FILE_C = + DataFiles.builder(SPEC) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withPartition(TestHelpers.Row.of(2)) + .withRecordCount(1) + .build(); + static final DataFile FILE_D = + DataFiles.builder(SPEC) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(10) + .withPartition(TestHelpers.Row.of(3)) + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-a-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartition(TestHelpers.Row.of(0)) + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-a-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartition(TestHelpers.Row.of(0)) + .withRecordCount(1) + .build(); + + @TempDir private File tableDir; + @Parameter private int formatVersion; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return org.apache.iceberg.TestHelpers.V2_AND_ABOVE; + } + + private Table table; + + @BeforeEach + public void setupTableLocation() throws Exception { + String tableLocation = tableDir.toURI().toString(); + this.table = + TABLES.create( + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), + tableLocation); + spark.conf().set("spark.sql.shuffle.partitions", SHUFFLE_PARTITIONS); + } + + private void checkRemoveFilesResults( + long expectedDatafiles, + long expectedPosDeleteFiles, + long expectedEqDeleteFiles, + long expectedManifestsDeleted, + long expectedManifestListsDeleted, + long expectedOtherFilesDeleted, + DeleteReachableFiles.Result results) { + assertThat(results.deletedManifestsCount()) + .as("Incorrect number of manifest files deleted") + .isEqualTo(expectedManifestsDeleted); + + assertThat(results.deletedDataFilesCount()) + .as("Incorrect number of datafiles deleted") + .isEqualTo(expectedDatafiles); + + assertThat(results.deletedPositionDeleteFilesCount()) + .as("Incorrect number of position delete files deleted") + .isEqualTo(expectedPosDeleteFiles); + + assertThat(results.deletedEqualityDeleteFilesCount()) + .as("Incorrect number of equality delete files deleted") + .isEqualTo(expectedEqDeleteFiles); + + assertThat(results.deletedManifestListsCount()) + .as("Incorrect number of manifest lists deleted") + .isEqualTo(expectedManifestListsDeleted); + + assertThat(results.deletedOtherFilesCount()) + .as("Incorrect number of other lists deleted") + .isEqualTo(expectedOtherFilesDeleted); + } + + @TestTemplate + public void dataFilesCleanupWithParallelTasks() { + table.newFastAppend().appendFile(FILE_A).commit(); + + table.newFastAppend().appendFile(FILE_B).commit(); + + table.newRewrite().rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D)).commit(); + + table.newRewrite().rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C)).commit(); + + Set deletedFiles = ConcurrentHashMap.newKeySet(); + Set deleteThreads = ConcurrentHashMap.newKeySet(); + AtomicInteger deleteThreadsIndex = new AtomicInteger(0); + + DeleteReachableFiles.Result result = + sparkActions() + .deleteReachableFiles(metadataLocation(table)) + .io(table.io()) + .executeDeleteWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("remove-files-" + deleteThreadsIndex.getAndIncrement()); + thread.setDaemon( + true); // daemon threads will be terminated abruptly when the JVM exits + return thread; + })) + .deleteWith( + s -> { + deleteThreads.add(Thread.currentThread().getName()); + deletedFiles.add(s); + }) + .execute(); + + // Verifies that the delete methods ran in the threads created by the provided ExecutorService + // ThreadFactory + assertThat(deleteThreads) + .containsExactlyInAnyOrder( + "remove-files-0", "remove-files-1", "remove-files-2", "remove-files-3"); + + Lists.newArrayList(FILE_A, FILE_B, FILE_C, FILE_D) + .forEach( + file -> + assertThat(deletedFiles) + .as("FILE_A should be deleted") + .contains(FILE_A.location())); + checkRemoveFilesResults(4L, 0, 0, 6L, 4L, 6, result); + } + + @TestTemplate + public void testWithExpiringDanglingStageCommit() { + table.location(); + // `A` commit + table.newAppend().appendFile(FILE_A).commit(); + + // `B` staged commit + table.newAppend().appendFile(FILE_B).stageOnly().commit(); + + // `C` commit + table.newAppend().appendFile(FILE_C).commit(); + + DeleteReachableFiles.Result result = + sparkActions().deleteReachableFiles(metadataLocation(table)).io(table.io()).execute(); + + checkRemoveFilesResults(3L, 0, 0, 3L, 3L, 5, result); + } + + @TestTemplate + public void testRemoveFileActionOnEmptyTable() { + DeleteReachableFiles.Result result = + sparkActions().deleteReachableFiles(metadataLocation(table)).io(table.io()).execute(); + + checkRemoveFilesResults(0, 0, 0, 0, 0, 2, result); + } + + @TestTemplate + public void testRemoveFilesActionWithReducedVersionsTable() { + table.updateProperties().set(TableProperties.METADATA_PREVIOUS_VERSIONS_MAX, "2").commit(); + table.newAppend().appendFile(FILE_A).commit(); + + table.newAppend().appendFile(FILE_B).commit(); + + table.newAppend().appendFile(FILE_B).commit(); + + table.newAppend().appendFile(FILE_C).commit(); + + table.newAppend().appendFile(FILE_D).commit(); + + DeleteReachableFiles baseRemoveFilesSparkAction = + sparkActions().deleteReachableFiles(metadataLocation(table)).io(table.io()); + DeleteReachableFiles.Result result = baseRemoveFilesSparkAction.execute(); + + checkRemoveFilesResults(4, 0, 0, 5, 5, 8, result); + } + + @TestTemplate + public void testRemoveFilesAction() { + table.newAppend().appendFile(FILE_A).commit(); + + table.newAppend().appendFile(FILE_B).commit(); + + DeleteReachableFiles baseRemoveFilesSparkAction = + sparkActions().deleteReachableFiles(metadataLocation(table)).io(table.io()); + checkRemoveFilesResults(2, 0, 0, 2, 2, 4, baseRemoveFilesSparkAction.execute()); + } + + @TestTemplate + public void testPositionDeleteFiles() { + table.newAppend().appendFile(FILE_A).commit(); + + table.newAppend().appendFile(FILE_B).commit(); + + table.newRowDelta().addDeletes(fileADeletes()).commit(); + + DeleteReachableFiles baseRemoveFilesSparkAction = + sparkActions().deleteReachableFiles(metadataLocation(table)).io(table.io()); + checkRemoveFilesResults(2, 1, 0, 3, 3, 5, baseRemoveFilesSparkAction.execute()); + } + + @TestTemplate + public void testEqualityDeleteFiles() { + table.newAppend().appendFile(FILE_A).commit(); + + table.newAppend().appendFile(FILE_B).commit(); + + table.newRowDelta().addDeletes(FILE_A_EQ_DELETES).commit(); + + DeleteReachableFiles baseRemoveFilesSparkAction = + sparkActions().deleteReachableFiles(metadataLocation(table)).io(table.io()); + checkRemoveFilesResults(2, 0, 1, 3, 3, 5, baseRemoveFilesSparkAction.execute()); + } + + @TestTemplate + public void testRemoveFilesActionWithDefaultIO() { + table.newAppend().appendFile(FILE_A).commit(); + + table.newAppend().appendFile(FILE_B).commit(); + + // IO not set explicitly on removeReachableFiles action + // IO defaults to HadoopFileIO + DeleteReachableFiles baseRemoveFilesSparkAction = + sparkActions().deleteReachableFiles(metadataLocation(table)); + checkRemoveFilesResults(2, 0, 0, 2, 2, 4, baseRemoveFilesSparkAction.execute()); + } + + @TestTemplate + public void testUseLocalIterator() { + table.newFastAppend().appendFile(FILE_A).commit(); + + table.newOverwrite().deleteFile(FILE_A).addFile(FILE_B).commit(); + + table.newFastAppend().appendFile(FILE_C).commit(); + + int jobsBefore = spark.sparkContext().dagScheduler().nextJobId().get(); + + withSQLConf( + ImmutableMap.of("spark.sql.adaptive.enabled", "false"), + () -> { + DeleteReachableFiles.Result results = + sparkActions() + .deleteReachableFiles(metadataLocation(table)) + .io(table.io()) + .option("stream-results", "true") + .execute(); + + int jobsAfter = spark.sparkContext().dagScheduler().nextJobId().get(); + int totalJobsRun = jobsAfter - jobsBefore; + + checkRemoveFilesResults(3L, 0, 0, 4L, 3L, 5, results); + + assertThat(totalJobsRun) + .as("Expected total jobs to be equal to total number of shuffle partitions") + .isEqualTo(SHUFFLE_PARTITIONS); + }); + } + + @TestTemplate + public void testIgnoreMetadataFilesNotFound() { + table.updateProperties().set(TableProperties.METADATA_PREVIOUS_VERSIONS_MAX, "1").commit(); + + table.newAppend().appendFile(FILE_A).commit(); + // There are three metadata json files at this point + DeleteOrphanFiles.Result result = + sparkActions().deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); + + assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); + assertThat(StreamSupport.stream(result.orphanFileLocations().spliterator(), false)) + .as("Should remove v1 file") + .anyMatch(file -> file.contains("v1.metadata.json")); + + DeleteReachableFiles baseRemoveFilesSparkAction = + sparkActions().deleteReachableFiles(metadataLocation(table)).io(table.io()); + DeleteReachableFiles.Result res = baseRemoveFilesSparkAction.execute(); + + checkRemoveFilesResults(1, 0, 0, 1, 1, 4, res); + } + + @TestTemplate + public void testEmptyIOThrowsException() { + DeleteReachableFiles baseRemoveFilesSparkAction = + sparkActions().deleteReachableFiles(metadataLocation(table)).io(null); + + assertThatThrownBy(baseRemoveFilesSparkAction::execute) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("File IO cannot be null"); + } + + @TestTemplate + public void testRemoveFilesActionWhenGarbageCollectionDisabled() { + table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); + + assertThatThrownBy(() -> sparkActions().deleteReachableFiles(metadataLocation(table)).execute()) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Cannot delete files: GC is disabled (deleting files may corrupt other tables)"); + } + + private String metadataLocation(Table tbl) { + return ((HasTableOperations) tbl).operations().current().metadataFileLocation(); + } + + private ActionsProvider sparkActions() { + return SparkActions.get(); + } + + private DeleteFile fileADeletes() { + return formatVersion >= 3 ? FileGenerationUtil.generateDV(table, FILE_A) : FILE_A_POS_DELETES; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java new file mode 100644 index 000000000000..7e07c66e0650 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestExpireSnapshotsAction.java @@ -0,0 +1,1407 @@ +/* + * 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.actions; + +import static org.apache.iceberg.types.Types.NestedField.optional; +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.nio.file.Path; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileGenerationUtil; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.ReachableFileUtil; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.ExpireSnapshots; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.hadoop.HadoopTables; +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.Sets; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestExpireSnapshotsAction extends TestBase { + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + private static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + private static final int SHUFFLE_PARTITIONS = 2; + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + + static final DataFile FILE_A = + DataFiles.builder(SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=0") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_B = + DataFiles.builder(SPEC) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=1") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_C = + DataFiles.builder(SPEC) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=2") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_D = + DataFiles.builder(SPEC) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=3") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-a-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=0") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-a-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=0") // easy way to set partition data for now + .withRecordCount(1) + .build(); + + @TempDir private Path temp; + @Parameter private int formatVersion; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return org.apache.iceberg.TestHelpers.V2_AND_ABOVE; + } + + @TempDir private File tableDir; + private String tableLocation; + private Table table; + + @BeforeEach + public void setupTableLocation() throws Exception { + this.tableLocation = tableDir.toURI().toString(); + this.table = + TABLES.create( + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), + tableLocation); + spark.conf().set("spark.sql.shuffle.partitions", SHUFFLE_PARTITIONS); + } + + private Long rightAfterSnapshot() { + return rightAfterSnapshot(table.currentSnapshot().snapshotId()); + } + + private Long rightAfterSnapshot(long snapshotId) { + Long end = System.currentTimeMillis(); + while (end <= table.snapshot(snapshotId).timestampMillis()) { + end = System.currentTimeMillis(); + } + return end; + } + + private DeleteFile fileADeletes() { + return formatVersion >= 3 ? FileGenerationUtil.generateDV(table, FILE_A) : FILE_A_POS_DELETES; + } + + private void checkExpirationResults( + long expectedDatafiles, + long expectedPosDeleteFiles, + long expectedEqDeleteFiles, + long expectedManifestsDeleted, + long expectedManifestListsDeleted, + ExpireSnapshots.Result results) { + + assertThat(results.deletedManifestsCount()) + .as("Incorrect number of manifest files deleted") + .isEqualTo(expectedManifestsDeleted); + + assertThat(results.deletedDataFilesCount()) + .as("Incorrect number of datafiles deleted") + .isEqualTo(expectedDatafiles); + + assertThat(results.deletedPositionDeleteFilesCount()) + .as("Incorrect number of pos deletefiles deleted") + .isEqualTo(expectedPosDeleteFiles); + + assertThat(results.deletedEqualityDeleteFilesCount()) + .as("Incorrect number of eq deletefiles deleted") + .isEqualTo(expectedEqDeleteFiles); + + assertThat(results.deletedManifestListsCount()) + .as("Incorrect number of manifest lists deleted") + .isEqualTo(expectedManifestListsDeleted); + } + + @TestTemplate + public void testFilesCleaned() throws Exception { + table.newFastAppend().appendFile(FILE_A).commit(); + + table.newOverwrite().deleteFile(FILE_A).addFile(FILE_B).commit(); + + table.newFastAppend().appendFile(FILE_C).commit(); + + long end = rightAfterSnapshot(); + + ExpireSnapshots.Result results = + SparkActions.get().expireSnapshots(table).expireOlderThan(end).execute(); + + assertThat(table.snapshots()).as("Table does not have 1 snapshot after expiration").hasSize(1); + + checkExpirationResults(1L, 0L, 0L, 1L, 2L, results); + } + + @TestTemplate + public void dataFilesCleanupWithParallelTasks() throws IOException { + + table.newFastAppend().appendFile(FILE_A).commit(); + + table.newFastAppend().appendFile(FILE_B).commit(); + + table.newRewrite().rewriteFiles(ImmutableSet.of(FILE_B), ImmutableSet.of(FILE_D)).commit(); + + table.newRewrite().rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_C)).commit(); + + long t4 = rightAfterSnapshot(); + + Set deletedFiles = ConcurrentHashMap.newKeySet(); + Set deleteThreads = ConcurrentHashMap.newKeySet(); + AtomicInteger deleteThreadsIndex = new AtomicInteger(0); + + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .executeDeleteWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("remove-snapshot-" + deleteThreadsIndex.getAndIncrement()); + thread.setDaemon( + true); // daemon threads will be terminated abruptly when the JVM exits + return thread; + })) + .expireOlderThan(t4) + .deleteWith( + s -> { + deleteThreads.add(Thread.currentThread().getName()); + deletedFiles.add(s); + }) + .execute(); + + // Verifies that the delete methods ran in the threads created by the provided ExecutorService + // ThreadFactory + assertThat(deleteThreads) + .containsExactlyInAnyOrder( + "remove-snapshot-0", "remove-snapshot-1", "remove-snapshot-2", "remove-snapshot-3"); + + assertThat(deletedFiles).as("FILE_A should be deleted").contains(FILE_A.location()); + assertThat(deletedFiles).as("FILE_B should be deleted").contains(FILE_B.location()); + + checkExpirationResults(2L, 0L, 0L, 3L, 3L, result); + } + + @TestTemplate + public void testNoFilesDeletedWhenNoSnapshotsExpired() throws Exception { + table.newFastAppend().appendFile(FILE_A).commit(); + + ExpireSnapshots.Result results = SparkActions.get().expireSnapshots(table).execute(); + checkExpirationResults(0L, 0L, 0L, 0L, 0L, results); + } + + @TestTemplate + public void testCleanupRepeatedOverwrites() throws Exception { + table.newFastAppend().appendFile(FILE_A).commit(); + + for (int i = 0; i < 10; i++) { + table.newOverwrite().deleteFile(FILE_A).addFile(FILE_B).commit(); + + table.newOverwrite().deleteFile(FILE_B).addFile(FILE_A).commit(); + } + + long end = rightAfterSnapshot(); + ExpireSnapshots.Result results = + SparkActions.get().expireSnapshots(table).expireOlderThan(end).execute(); + checkExpirationResults(1L, 0L, 0L, 39L, 20L, results); + } + + @TestTemplate + public void testRetainLastWithExpireOlderThan() { + table + .newAppend() + .appendFile(FILE_A) // data_bucket=0 + .commit(); + long firstSnapshotId = table.currentSnapshot().snapshotId(); + long t1 = System.currentTimeMillis(); + while (t1 <= table.currentSnapshot().timestampMillis()) { + t1 = System.currentTimeMillis(); + } + + table + .newAppend() + .appendFile(FILE_B) // data_bucket=1 + .commit(); + + table + .newAppend() + .appendFile(FILE_C) // data_bucket=2 + .commit(); + + long t3 = rightAfterSnapshot(); + + // Retain last 2 snapshots + SparkActions.get().expireSnapshots(table).expireOlderThan(t3).retainLast(2).execute(); + + assertThat(table.snapshots()).as("Should have two snapshots.").hasSize(2); + assertThat(table.snapshot(firstSnapshotId)).as("First snapshot should not present.").isNull(); + } + + @TestTemplate + public void testExpireTwoSnapshotsById() throws Exception { + table + .newAppend() + .appendFile(FILE_A) // data_bucket=0 + .commit(); + long firstSnapshotId = table.currentSnapshot().snapshotId(); + + table + .newAppend() + .appendFile(FILE_B) // data_bucket=1 + .commit(); + + long secondSnapshotID = table.currentSnapshot().snapshotId(); + + table + .newAppend() + .appendFile(FILE_C) // data_bucket=2 + .commit(); + + // Retain last 2 snapshots + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireSnapshotId(firstSnapshotId) + .expireSnapshotId(secondSnapshotID) + .execute(); + + assertThat(table.snapshots()).as("Should have one snapshot.").hasSize(1); + assertThat(table.snapshot(firstSnapshotId)).as("First snapshot should not present.").isNull(); + assertThat(table.snapshot(secondSnapshotID)).as("Second snapshot should not present.").isNull(); + + checkExpirationResults(0L, 0L, 0L, 0L, 2L, result); + } + + @TestTemplate + public void testRetainLastWithExpireById() { + table + .newAppend() + .appendFile(FILE_A) // data_bucket=0 + .commit(); + long firstSnapshotId = table.currentSnapshot().snapshotId(); + + table + .newAppend() + .appendFile(FILE_B) // data_bucket=1 + .commit(); + + table + .newAppend() + .appendFile(FILE_C) // data_bucket=2 + .commit(); + + // Retain last 3 snapshots, but explicitly remove the first snapshot + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireSnapshotId(firstSnapshotId) + .retainLast(3) + .execute(); + + assertThat(table.snapshots()).as("Should have 2 snapshots.").hasSize(2); + assertThat(table.snapshot(firstSnapshotId)).as("First snapshot should not present.").isNull(); + checkExpirationResults(0L, 0L, 0L, 0L, 1L, result); + } + + @TestTemplate + public void testRetainLastWithTooFewSnapshots() { + table + .newAppend() + .appendFile(FILE_A) // data_bucket=0 + .appendFile(FILE_B) // data_bucket=1 + .commit(); + long firstSnapshotId = table.currentSnapshot().snapshotId(); + + table + .newAppend() + .appendFile(FILE_C) // data_bucket=2 + .commit(); + + long t2 = rightAfterSnapshot(); + + // Retain last 3 snapshots + ExpireSnapshots.Result result = + SparkActions.get().expireSnapshots(table).expireOlderThan(t2).retainLast(3).execute(); + + assertThat(table.snapshots()).as("Should have two snapshots.").hasSize(2); + assertThat(table.snapshot(firstSnapshotId)) + .as("First snapshot should still be present.") + .isNotNull(); + checkExpirationResults(0L, 0L, 0L, 0L, 0L, result); + } + + @TestTemplate + public void testRetainLastKeepsExpiringSnapshot() { + table + .newAppend() + .appendFile(FILE_A) // data_bucket=0 + .commit(); + + table + .newAppend() + .appendFile(FILE_B) // data_bucket=1 + .commit(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + table + .newAppend() + .appendFile(FILE_C) // data_bucket=2 + .commit(); + + table + .newAppend() + .appendFile(FILE_D) // data_bucket=3 + .commit(); + + // Retain last 2 snapshots and expire older than t3 + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(secondSnapshot.timestampMillis()) + .retainLast(2) + .execute(); + + assertThat(table.snapshots()).as("Should have three snapshots.").hasSize(3); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("First snapshot should be present.") + .isNotNull(); + checkExpirationResults(0L, 0L, 0L, 0L, 1L, result); + } + + @TestTemplate + public void testExpireSnapshotsWithDisabledGarbageCollection() { + table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); + + table.newAppend().appendFile(FILE_A).commit(); + + assertThatThrownBy(() -> SparkActions.get().expireSnapshots(table)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Cannot expire snapshots: GC is disabled (deleting files may corrupt other tables)"); + } + + @TestTemplate + public void testExpireOlderThanMultipleCalls() { + table + .newAppend() + .appendFile(FILE_A) // data_bucket=0 + .commit(); + + table + .newAppend() + .appendFile(FILE_B) // data_bucket=1 + .commit(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + table + .newAppend() + .appendFile(FILE_C) // data_bucket=2 + .commit(); + + Snapshot thirdSnapshot = table.currentSnapshot(); + + // Retain last 2 snapshots and expire older than t3 + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(secondSnapshot.timestampMillis()) + .expireOlderThan(thirdSnapshot.timestampMillis()) + .execute(); + + assertThat(table.snapshots()).as("Should have one snapshot.").hasSize(1); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Second snapshot should not present.") + .isNull(); + checkExpirationResults(0L, 0L, 0L, 0L, 2L, result); + } + + @TestTemplate + public void testRetainLastMultipleCalls() { + table + .newAppend() + .appendFile(FILE_A) // data_bucket=0 + .commit(); + + table + .newAppend() + .appendFile(FILE_B) // data_bucket=1 + .commit(); + + Snapshot secondSnapshot = table.currentSnapshot(); + + table + .newAppend() + .appendFile(FILE_C) // data_bucket=2 + .commit(); + + long t3 = rightAfterSnapshot(); + + // Retain last 2 snapshots and expire older than t3 + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(t3) + .retainLast(2) + .retainLast(1) + .execute(); + + assertThat(table.snapshots()).as("Should have one snapshot.").hasSize(1); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Second snapshot should not present.") + .isNull(); + checkExpirationResults(0L, 0L, 0L, 0L, 2L, result); + } + + @TestTemplate + public void testRetainZeroSnapshots() { + assertThatThrownBy(() -> SparkActions.get().expireSnapshots(table).retainLast(0).execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Number of snapshots to retain must be at least 1, cannot be: 0"); + } + + @TestTemplate + public void testScanExpiredManifestInValidSnapshotAppend() { + table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + + table.newOverwrite().addFile(FILE_C).deleteFile(FILE_A).commit(); + + table.newAppend().appendFile(FILE_D).commit(); + + long t3 = rightAfterSnapshot(); + + Set deletedFiles = Sets.newHashSet(); + + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(t3) + .deleteWith(deletedFiles::add) + .execute(); + + assertThat(deletedFiles).as("FILE_A should be deleted").contains(FILE_A.location()); + checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); + } + + @TestTemplate + public void testScanExpiredManifestInValidSnapshotFastAppend() { + table + .updateProperties() + .set(TableProperties.MANIFEST_MERGE_ENABLED, "true") + .set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1") + .commit(); + + table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + + table.newOverwrite().addFile(FILE_C).deleteFile(FILE_A).commit(); + + table.newFastAppend().appendFile(FILE_D).commit(); + + long t3 = rightAfterSnapshot(); + + Set deletedFiles = Sets.newHashSet(); + + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(t3) + .deleteWith(deletedFiles::add) + .execute(); + + assertThat(deletedFiles).as("FILE_A should be deleted").contains(FILE_A.location()); + checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); + } + + /** + * Test on table below, and expiring the staged commit `B` using `expireOlderThan` API. Table: A - + * C ` B (staged) + */ + @TestTemplate + public void testWithExpiringDanglingStageCommit() { + // `A` commit + table.newAppend().appendFile(FILE_A).commit(); + + // `B` staged commit + table.newAppend().appendFile(FILE_B).stageOnly().commit(); + + TableMetadata base = ((BaseTable) table).operations().current(); + Snapshot snapshotA = base.snapshots().get(0); + Snapshot snapshotB = base.snapshots().get(1); + + // `C` commit + table.newAppend().appendFile(FILE_C).commit(); + + Set deletedFiles = Sets.newHashSet(); + + // Expire all commits including dangling staged snapshot. + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .deleteWith(deletedFiles::add) + .expireOlderThan(snapshotB.timestampMillis() + 1) + .execute(); + + checkExpirationResults(1L, 0L, 0L, 1L, 2L, result); + + Set expectedDeletes = Sets.newHashSet(); + expectedDeletes.add(snapshotA.manifestListLocation()); + + // Files should be deleted of dangling staged snapshot + snapshotB + .addedDataFiles(table.io()) + .forEach( + i -> { + expectedDeletes.add(i.location()); + }); + + // ManifestList should be deleted too + expectedDeletes.add(snapshotB.manifestListLocation()); + snapshotB + .dataManifests(table.io()) + .forEach( + file -> { + // Only the manifest of B should be deleted. + if (file.snapshotId() == snapshotB.snapshotId()) { + expectedDeletes.add(file.path()); + } + }); + assertThat(expectedDeletes) + .as("Files deleted count should be expected") + .hasSameSizeAs(deletedFiles); + // Take the diff + expectedDeletes.removeAll(deletedFiles); + assertThat(expectedDeletes).as("Exactly same files should be deleted").isEmpty(); + } + + /** + * Expire cherry-pick the commit as shown below, when `B` is in table's current state Table: A - B + * - C <--current snapshot `- D (source=B) + */ + @TestTemplate + public void testWithCherryPickTableSnapshot() { + // `A` commit + table.newAppend().appendFile(FILE_A).commit(); + Snapshot snapshotA = table.currentSnapshot(); + + // `B` commit + Set deletedAFiles = Sets.newHashSet(); + table.newOverwrite().addFile(FILE_B).deleteFile(FILE_A).deleteWith(deletedAFiles::add).commit(); + assertThat(deletedAFiles).as("No files should be physically deleted").isEmpty(); + + // pick the snapshot 'B` + Snapshot snapshotB = table.currentSnapshot(); + + // `C` commit to let cherry-pick take effect, and avoid fast-forward of `B` with cherry-pick + table.newAppend().appendFile(FILE_C).commit(); + Snapshot snapshotC = table.currentSnapshot(); + + // Move the table back to `A` + table.manageSnapshots().setCurrentSnapshot(snapshotA.snapshotId()).commit(); + + // Generate A -> `D (B)` + table.manageSnapshots().cherrypick(snapshotB.snapshotId()).commit(); + Snapshot snapshotD = table.currentSnapshot(); + + // Move the table back to `C` + table.manageSnapshots().setCurrentSnapshot(snapshotC.snapshotId()).commit(); + List deletedFiles = Lists.newArrayList(); + + // Expire `C` + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .deleteWith(deletedFiles::add) + .expireOlderThan(snapshotC.timestampMillis() + 1) + .execute(); + + // Make sure no dataFiles are deleted for the B, C, D snapshot + Lists.newArrayList(snapshotB, snapshotC, snapshotD) + .forEach( + i -> { + i.addedDataFiles(table.io()) + .forEach( + item -> { + assertThat(deletedFiles).doesNotContain(item.location()); + }); + }); + + checkExpirationResults(1L, 0L, 0L, 2L, 2L, result); + } + + /** + * Test on table below, and expiring `B` which is not in current table state. 1) Expire `B` 2) All + * commit Table: A - C - D (B) ` B (staged) + */ + @TestTemplate + public void testWithExpiringStagedThenCherrypick() { + // `A` commit + table.newAppend().appendFile(FILE_A).commit(); + + // `B` commit + table.newAppend().appendFile(FILE_B).stageOnly().commit(); + + // pick the snapshot that's staged but not committed + TableMetadata base = ((BaseTable) table).operations().current(); + Snapshot snapshotB = base.snapshots().get(1); + + // `C` commit to let cherry-pick take effect, and avoid fast-forward of `B` with cherry-pick + table.newAppend().appendFile(FILE_C).commit(); + + // `D (B)` cherry-pick commit + table.manageSnapshots().cherrypick(snapshotB.snapshotId()).commit(); + + base = ((BaseTable) table).operations().current(); + Snapshot snapshotD = base.snapshots().get(3); + + List deletedFiles = Lists.newArrayList(); + + // Expire `B` commit. + ExpireSnapshots.Result firstResult = + SparkActions.get() + .expireSnapshots(table) + .deleteWith(deletedFiles::add) + .expireSnapshotId(snapshotB.snapshotId()) + .execute(); + + // Make sure no dataFiles are deleted for the staged snapshot + Lists.newArrayList(snapshotB) + .forEach( + i -> { + i.addedDataFiles(table.io()) + .forEach( + item -> { + assertThat(deletedFiles).doesNotContain(item.location()); + }); + }); + checkExpirationResults(0L, 0L, 0L, 1L, 1L, firstResult); + + // Expire all snapshots including cherry-pick + ExpireSnapshots.Result secondResult = + SparkActions.get() + .expireSnapshots(table) + .deleteWith(deletedFiles::add) + .expireOlderThan(table.currentSnapshot().timestampMillis() + 1) + .execute(); + + // Make sure no dataFiles are deleted for the staged and cherry-pick + Lists.newArrayList(snapshotB, snapshotD) + .forEach( + i -> { + i.addedDataFiles(table.io()) + .forEach( + item -> { + assertThat(deletedFiles).doesNotContain(item.location()); + }); + }); + checkExpirationResults(0L, 0L, 0L, 0L, 2L, secondResult); + } + + @TestTemplate + public void testExpireOlderThan() { + table.newAppend().appendFile(FILE_A).commit(); + + Snapshot firstSnapshot = table.currentSnapshot(); + + rightAfterSnapshot(); + + table.newAppend().appendFile(FILE_B).commit(); + + long snapshotId = table.currentSnapshot().snapshotId(); + + long tAfterCommits = rightAfterSnapshot(); + + Set deletedFiles = Sets.newHashSet(); + + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(tAfterCommits) + .deleteWith(deletedFiles::add) + .execute(); + + assertThat(table.currentSnapshot().snapshotId()) + .as("Expire should not change current snapshot.") + .isEqualTo(snapshotId); + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Expire should remove the oldest snapshot.") + .isNull(); + assertThat(deletedFiles) + .as("Should remove only the expired manifest list location.") + .containsExactly(firstSnapshot.manifestListLocation()); + + checkExpirationResults(0, 0, 0, 0, 1, result); + } + + @TestTemplate + public void testExpireOlderThanWithDelete() { + table.newAppend().appendFile(FILE_A).commit(); + + Snapshot firstSnapshot = table.currentSnapshot(); + assertThat(firstSnapshot.allManifests(table.io())).as("Should create one manifest").hasSize(1); + + rightAfterSnapshot(); + + table.newDelete().deleteFile(FILE_A).commit(); + + Snapshot secondSnapshot = table.currentSnapshot(); + assertThat(secondSnapshot.allManifests(table.io())) + .as("Should create replace manifest with a rewritten manifest") + .hasSize(1); + + table.newAppend().appendFile(FILE_B).commit(); + + rightAfterSnapshot(); + + long snapshotId = table.currentSnapshot().snapshotId(); + + long tAfterCommits = rightAfterSnapshot(); + + Set deletedFiles = Sets.newHashSet(); + + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(tAfterCommits) + .deleteWith(deletedFiles::add) + .execute(); + + assertThat(table.currentSnapshot().snapshotId()) + .as("Expire should not change current snapshot.") + .isEqualTo(snapshotId); + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Expire should remove the oldest snapshot.") + .isNull(); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Expire should remove the second oldest snapshot.") + .isNull(); + assertThat(deletedFiles) + .as("Should remove expired manifest lists and deleted data file.") + .containsExactlyInAnyOrder( + firstSnapshot.manifestListLocation(), // snapshot expired + firstSnapshot + .allManifests(table.io()) + .get(0) + .path(), // manifest was rewritten for delete + secondSnapshot.manifestListLocation(), // snapshot expired + secondSnapshot + .allManifests(table.io()) + .get(0) + .path(), // manifest contained only deletes, was dropped + FILE_A.location()); + + checkExpirationResults(1, 0, 0, 2, 2, result); + } + + @TestTemplate + public void testExpireOlderThanWithDeleteInMergedManifests() { + // merge every commit + table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "0").commit(); + + table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + + Snapshot firstSnapshot = table.currentSnapshot(); + assertThat(firstSnapshot.allManifests(table.io())).as("Should create one manifest").hasSize(1); + + rightAfterSnapshot(); + + table + .newDelete() + .deleteFile(FILE_A) // FILE_B is still in the dataset + .commit(); + + Snapshot secondSnapshot = table.currentSnapshot(); + assertThat(secondSnapshot.allManifests(table.io())) + .as("Should replace manifest with a rewritten manifest") + .hasSize(1); + table + .newFastAppend() // do not merge to keep the last snapshot's manifest valid + .appendFile(FILE_C) + .commit(); + + rightAfterSnapshot(); + + long snapshotId = table.currentSnapshot().snapshotId(); + + long tAfterCommits = rightAfterSnapshot(); + + Set deletedFiles = Sets.newHashSet(); + + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(tAfterCommits) + .deleteWith(deletedFiles::add) + .execute(); + + assertThat(table.currentSnapshot().snapshotId()) + .as("Expire should not change current snapshot.") + .isEqualTo(snapshotId); + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Expire should remove the oldest snapshot.") + .isNull(); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Expire should remove the second oldest snapshot.") + .isNull(); + + assertThat(deletedFiles) + .as("Should remove expired manifest lists and deleted data file.") + .containsExactlyInAnyOrder( + firstSnapshot.manifestListLocation(), // snapshot expired + firstSnapshot + .allManifests(table.io()) + .get(0) + .path(), // manifest was rewritten for delete + secondSnapshot.manifestListLocation(), // snapshot expired + FILE_A.location()); + checkExpirationResults(1, 0, 0, 1, 2, result); + } + + @TestTemplate + public void testExpireOlderThanWithRollback() { + // merge every commit + table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "0").commit(); + + table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + + Snapshot firstSnapshot = table.currentSnapshot(); + assertThat(firstSnapshot.allManifests(table.io())).as("Should create one manifest").hasSize(1); + + rightAfterSnapshot(); + + table.newDelete().deleteFile(FILE_B).commit(); + + Snapshot secondSnapshot = table.currentSnapshot(); + Set secondSnapshotManifests = + Sets.newHashSet(secondSnapshot.allManifests(table.io())); + secondSnapshotManifests.removeAll(firstSnapshot.allManifests(table.io())); + assertThat(secondSnapshotManifests).as("Should add one new manifest for append").hasSize(1); + + table.manageSnapshots().rollbackTo(firstSnapshot.snapshotId()).commit(); + + long tAfterCommits = rightAfterSnapshot(secondSnapshot.snapshotId()); + + long snapshotId = table.currentSnapshot().snapshotId(); + + Set deletedFiles = Sets.newHashSet(); + + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(tAfterCommits) + .deleteWith(deletedFiles::add) + .execute(); + + assertThat(table.currentSnapshot().snapshotId()) + .as("Expire should not change current snapshot.") + .isEqualTo(snapshotId); + + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Expire should keep the oldest snapshot, current.") + .isNotNull(); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Expire should remove the orphaned snapshot.") + .isNull(); + + assertThat(deletedFiles) + .as("Should remove expired manifest lists and reverted appended data file") + .containsExactlyInAnyOrder( + secondSnapshot.manifestListLocation(), // snapshot expired + Iterables.getOnlyElement(secondSnapshotManifests).path()); + + checkExpirationResults(0, 0, 0, 1, 1, result); + } + + @TestTemplate + public void testExpireOlderThanWithRollbackAndMergedManifests() { + table.newAppend().appendFile(FILE_A).commit(); + + Snapshot firstSnapshot = table.currentSnapshot(); + assertThat(firstSnapshot.allManifests(table.io())).as("Should create one manifest").hasSize(1); + rightAfterSnapshot(); + + table.newAppend().appendFile(FILE_B).commit(); + + Snapshot secondSnapshot = table.currentSnapshot(); + Set secondSnapshotManifests = + Sets.newHashSet(secondSnapshot.allManifests(table.io())); + secondSnapshotManifests.removeAll(firstSnapshot.allManifests(table.io())); + assertThat(secondSnapshotManifests).as("Should add one new manifest for append").hasSize(1); + + table.manageSnapshots().rollbackTo(firstSnapshot.snapshotId()).commit(); + + long tAfterCommits = rightAfterSnapshot(secondSnapshot.snapshotId()); + + long snapshotId = table.currentSnapshot().snapshotId(); + + Set deletedFiles = Sets.newHashSet(); + + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(tAfterCommits) + .deleteWith(deletedFiles::add) + .execute(); + + assertThat(table.currentSnapshot().snapshotId()) + .as("Expire should not change current snapshot.") + .isEqualTo(snapshotId); + + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Expire should keep the oldest snapshot, current.") + .isNotNull(); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Expire should remove the orphaned snapshot.") + .isNull(); + + assertThat(deletedFiles) + .as("Should remove expired manifest lists and reverted appended data file") + .containsExactlyInAnyOrder( + secondSnapshot.manifestListLocation(), // snapshot expired + Iterables.getOnlyElement(secondSnapshotManifests) + .path(), // manifest is no longer referenced + FILE_B.location()); + + checkExpirationResults(1, 0, 0, 1, 1, result); + } + + @TestTemplate + public void testExpireOlderThanWithDeleteFile() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + table.updateProperties().set(TableProperties.MANIFEST_MERGE_ENABLED, "false").commit(); + + // Add Data File + table.newAppend().appendFile(FILE_A).commit(); + Snapshot firstSnapshot = table.currentSnapshot(); + + // Add POS Delete + DeleteFile fileADeletes = fileADeletes(); + table.newRowDelta().addDeletes(fileADeletes).commit(); + Snapshot secondSnapshot = table.currentSnapshot(); + + // Add EQ Delete + table.newRowDelta().addDeletes(FILE_A_EQ_DELETES).commit(); + Snapshot thirdSnapshot = table.currentSnapshot(); + + // Move files to DELETED + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + Snapshot fourthSnapshot = table.currentSnapshot(); + + long afterAllDeleted = rightAfterSnapshot(); + + table.newAppend().appendFile(FILE_B).commit(); + + Set deletedFiles = Sets.newHashSet(); + + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(afterAllDeleted) + .deleteWith(deletedFiles::add) + .execute(); + + Set expectedDeletes = + Sets.newHashSet( + firstSnapshot.manifestListLocation(), + secondSnapshot.manifestListLocation(), + thirdSnapshot.manifestListLocation(), + fourthSnapshot.manifestListLocation(), + FILE_A.location(), + fileADeletes.location(), + FILE_A_EQ_DELETES.location()); + + expectedDeletes.addAll( + thirdSnapshot.allManifests(table.io()).stream() + .map(ManifestFile::path) + .map(CharSequence::toString) + .collect(Collectors.toSet())); + // Delete operation (fourth snapshot) generates new manifest files + expectedDeletes.addAll( + fourthSnapshot.allManifests(table.io()).stream() + .map(ManifestFile::path) + .map(CharSequence::toString) + .collect(Collectors.toSet())); + + assertThat(deletedFiles) + .as("Should remove expired manifest lists and deleted data file") + .isEqualTo(expectedDeletes); + + checkExpirationResults(1, 1, 1, 6, 4, result); + } + + @TestTemplate + public void testExpireOnEmptyTable() { + Set deletedFiles = Sets.newHashSet(); + + // table has no data, testing ExpireSnapshots should not fail with no snapshot + ExpireSnapshots.Result result = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(System.currentTimeMillis()) + .deleteWith(deletedFiles::add) + .execute(); + + checkExpirationResults(0, 0, 0, 0, 0, result); + } + + @TestTemplate + public void testExpireAction() { + table.newAppend().appendFile(FILE_A).commit(); + + Snapshot firstSnapshot = table.currentSnapshot(); + + rightAfterSnapshot(); + + table.newAppend().appendFile(FILE_B).commit(); + + long snapshotId = table.currentSnapshot().snapshotId(); + + long tAfterCommits = rightAfterSnapshot(); + + Set deletedFiles = Sets.newHashSet(); + + ExpireSnapshotsSparkAction action = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(tAfterCommits) + .deleteWith(deletedFiles::add); + Dataset pendingDeletes = action.expireFiles(); + + List pending = pendingDeletes.collectAsList(); + + assertThat(table.currentSnapshot().snapshotId()) + .as("Should not change current snapshot.") + .isEqualTo(snapshotId); + + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Should remove the oldest snapshot") + .isNull(); + assertThat(pending).as("Pending deletes should contain one row").hasSize(1); + + assertThat(pending.get(0).getPath()) + .as("Pending delete should be the expired manifest list location") + .isEqualTo(firstSnapshot.manifestListLocation()); + + assertThat(pending.get(0).getType()) + .as("Pending delete should be a manifest list") + .isEqualTo("Manifest List"); + + assertThat(deletedFiles).as("Should not delete any files").isEmpty(); + + assertThat(action.expireFiles().count()) + .as("Multiple calls to expire should return the same count of deleted files") + .isEqualTo(pendingDeletes.count()); + } + + @TestTemplate + public void testUseLocalIterator() { + table.newFastAppend().appendFile(FILE_A).commit(); + + table.newOverwrite().deleteFile(FILE_A).addFile(FILE_B).commit(); + + table.newFastAppend().appendFile(FILE_C).commit(); + + long end = rightAfterSnapshot(); + + int jobsBeforeStreamResults = spark.sparkContext().dagScheduler().nextJobId().get(); + + withSQLConf( + ImmutableMap.of("spark.sql.adaptive.enabled", "false"), + () -> { + ExpireSnapshots.Result results = + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(end) + .option("stream-results", "true") + .execute(); + + int jobsAfterStreamResults = spark.sparkContext().dagScheduler().nextJobId().get(); + int jobsRunDuringStreamResults = jobsAfterStreamResults - jobsBeforeStreamResults; + + checkExpirationResults(1L, 0L, 0L, 1L, 2L, results); + + assertThat(jobsRunDuringStreamResults) + .as( + "Expected total number of jobs with stream-results should match the expected number") + .isEqualTo(4L); + }); + } + + @TestTemplate + public void testExpireAfterExecute() { + table + .newAppend() + .appendFile(FILE_A) // data_bucket=0 + .commit(); + + rightAfterSnapshot(); + + table + .newAppend() + .appendFile(FILE_B) // data_bucket=1 + .commit(); + + table + .newAppend() + .appendFile(FILE_C) // data_bucket=2 + .commit(); + + long t3 = rightAfterSnapshot(); + + ExpireSnapshotsSparkAction action = SparkActions.get().expireSnapshots(table); + + action.expireOlderThan(t3).retainLast(2); + + ExpireSnapshots.Result result = action.execute(); + checkExpirationResults(0L, 0L, 0L, 0L, 1L, result); + + List typedExpiredFiles = action.expireFiles().collectAsList(); + assertThat(typedExpiredFiles).as("Expired results must match").hasSize(1); + + List untypedExpiredFiles = action.expireFiles().collectAsList(); + assertThat(untypedExpiredFiles).as("Expired results must match").hasSize(1); + } + + @TestTemplate + public void testExpireFileDeletionMostExpired() { + textExpireAllCheckFilesDeleted(5, 2); + } + + @TestTemplate + public void testExpireFileDeletionMostRetained() { + textExpireAllCheckFilesDeleted(2, 5); + } + + public void textExpireAllCheckFilesDeleted(int dataFilesExpired, int dataFilesRetained) { + // Add data files to be expired + Set dataFiles = Sets.newHashSet(); + for (int i = 0; i < dataFilesExpired; i++) { + DataFile df = + DataFiles.builder(SPEC) + .withPath(String.format("/path/to/data-expired-%d.parquet", i)) + .withFileSizeInBytes(10) + .withPartitionPath("c1=1") + .withRecordCount(1) + .build(); + dataFiles.add(df.location()); + table.newFastAppend().appendFile(df).commit(); + } + + // Delete them all, these will be deleted on expire snapshot + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + // Clears "DELETED" manifests + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + + Set manifestsBefore = TestHelpers.reachableManifestPaths(table); + + // Add data files to be retained, which are not deleted. + for (int i = 0; i < dataFilesRetained; i++) { + DataFile df = + DataFiles.builder(SPEC) + .withPath(String.format("/path/to/data-retained-%d.parquet", i)) + .withFileSizeInBytes(10) + .withPartitionPath("c1=1") + .withRecordCount(1) + .build(); + table.newFastAppend().appendFile(df).commit(); + } + + long end = rightAfterSnapshot(); + + Set expectedDeletes = Sets.newHashSet(); + expectedDeletes.addAll(ReachableFileUtil.manifestListLocations(table)); + // all snapshot manifest lists except current will be deleted + expectedDeletes.remove(table.currentSnapshot().manifestListLocation()); + expectedDeletes.addAll( + manifestsBefore); // new manifests are reachable from current snapshot and not deleted + expectedDeletes.addAll( + dataFiles); // new data files are reachable from current snapshot and not deleted + + Set deletedFiles = Sets.newHashSet(); + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(end) + .deleteWith(deletedFiles::add) + .execute(); + + assertThat(deletedFiles) + .as("All reachable files before expiration should be deleted") + .isEqualTo(expectedDeletes); + } + + @TestTemplate + public void testExpireSomeCheckFilesDeleted() { + + table.newAppend().appendFile(FILE_A).commit(); + + table.newAppend().appendFile(FILE_B).commit(); + + table.newAppend().appendFile(FILE_C).commit(); + + table.newDelete().deleteFile(FILE_A).commit(); + + long after = rightAfterSnapshot(); + waitUntilAfter(after); + + table.newAppend().appendFile(FILE_D).commit(); + + table.newDelete().deleteFile(FILE_B).commit(); + + Set deletedFiles = Sets.newHashSet(); + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(after) + .deleteWith(deletedFiles::add) + .execute(); + + // C, D should be retained (live) + // B should be retained (previous snapshot points to it) + // A should be deleted + assertThat(deletedFiles) + .contains(FILE_A.location()) + .doesNotContain(FILE_B.location(), FILE_C.location(), FILE_D.location()); + } + + @TestTemplate + public void testNoExpiredMetadataCleanupByDefault() { + table.newAppend().appendFile(FILE_A).commit(); + table.newDelete().deleteFile(FILE_A).commit(); + + long after = rightAfterSnapshot(); + + table.updateSchema().addColumn("extra_col", Types.IntegerType.get()).commit(); + table.newAppend().appendFile(FILE_B).commit(); + + Set schemaIds = table.schemas().keySet(); + + Set deletedFiles = Sets.newHashSet(); + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(after) + .deleteWith(deletedFiles::add) + .execute(); + + assertThat(table.schemas().keySet()).containsExactlyInAnyOrderElementsOf(schemaIds); + assertThat(deletedFiles).contains(FILE_A.location()).doesNotContain(FILE_B.location()); + } + + @TestTemplate + public void testCleanExpiredMetadata() { + table.newAppend().appendFile(FILE_A).commit(); + table.newDelete().deleteFile(FILE_A).commit(); + + long after = rightAfterSnapshot(); + + table.updateSchema().addColumn("extra_col", Types.IntegerType.get()).commit(); + table.updateSpec().addField("extra_col").commit(); + + DataFile fileInNewSpec = + DataFiles.builder(table.spec()) + .withPath("/path/to/data-in-new-spec.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=1/extra_col=11") + .withRecordCount(1) + .build(); + + table.newAppend().appendFile(fileInNewSpec).commit(); + + Set deletedFiles = Sets.newHashSet(); + SparkActions.get() + .expireSnapshots(table) + .expireOlderThan(after) + .deleteWith(deletedFiles::add) + .cleanExpiredMetadata(true) + .execute(); + + assertThat(table.specs().keySet()) + .as("Should have only the latest spec") + .containsExactly(table.spec().specId()); + assertThat(table.schemas().keySet()) + .as("Should have only the latest schema") + .containsExactly(table.schema().schemaId()); + assertThat(deletedFiles) + .as("Should remove the file from first snapshot") + .contains(FILE_A.location()) + .doesNotContain(fileInNewSpec.location()); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java new file mode 100644 index 000000000000..94afa50cf4b8 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java @@ -0,0 +1,68 @@ +/* + * 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.actions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.spark.CatalogTestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMigrateTableAction extends CatalogTestBase { + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s_BACKUP_", tableName); + } + + @TestTemplate + public void testMigrateWithParallelTasks() throws IOException { + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + AtomicInteger migrationThreadsIndex = new AtomicInteger(0); + SparkActions.get() + .migrateTable(tableName) + .executeWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("table-migration-" + migrationThreadsIndex.getAndIncrement()); + thread.setDaemon(true); + return thread; + })) + .execute(); + assertThat(migrationThreadsIndex.get()).isEqualTo(2); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java new file mode 100644 index 000000000000..76084c2b9402 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveDanglingDeleteAction.java @@ -0,0 +1,540 @@ +/* + * 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.actions; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.File; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileGenerationUtil; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.actions.RemoveDanglingDeleteFiles; +import org.apache.iceberg.hadoop.HadoopTables; +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.Lists; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Encoders; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import scala.Tuple2; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRemoveDanglingDeleteAction extends TestBase { + + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + private static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.StringType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + + static final DataFile FILE_A = + DataFiles.builder(SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_A2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_B = + DataFiles.builder(SPEC) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_B2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_C = + DataFiles.builder(SPEC) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=c") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_C2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=c") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_D = + DataFiles.builder(SPEC) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=d") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DataFile FILE_D2 = + DataFiles.builder(SPEC) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=d") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-a-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A2_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-a2-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-a-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_A2_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-a2-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=a") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-b-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B2_POS_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofPositionDeletes() + .withPath("/path/to/data-b2-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-b-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + static final DeleteFile FILE_B2_EQ_DELETES = + FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes() + .withPath("/path/to/data-b2-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=b") // easy way to set partition data for now + .withRecordCount(1) + .build(); + + static final DataFile FILE_UNPARTITIONED = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-unpartitioned.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + static final DeleteFile FILE_UNPARTITIONED_POS_DELETE = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes() + .withPath("/path/to/data-unpartitioned-pos-deletes.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + static final DeleteFile FILE_UNPARTITIONED_EQ_DELETE = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofEqualityDeletes() + .withPath("/path/to/data-unpartitioned-eq-deletes.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + @TempDir private File tableDir; + @Parameter private int formatVersion; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return TestHelpers.V2_AND_ABOVE; + } + + private String tableLocation = null; + private Table table; + + @BeforeEach + public void before() throws Exception { + this.tableLocation = tableDir.toURI().toString(); + } + + @AfterEach + public void after() { + TABLES.dropTable(tableLocation); + } + + private void setupPartitionedTable() { + this.table = + TABLES.create( + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), + tableLocation); + } + + private void setupUnpartitionedTable() { + this.table = + TABLES.create( + SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), + tableLocation); + } + + private DeleteFile fileADeletes() { + return formatVersion >= 3 ? FileGenerationUtil.generateDV(table, FILE_A) : FILE_A_POS_DELETES; + } + + private DeleteFile fileA2Deletes() { + return formatVersion >= 3 ? FileGenerationUtil.generateDV(table, FILE_A2) : FILE_A2_POS_DELETES; + } + + private DeleteFile fileBDeletes() { + return formatVersion >= 3 ? FileGenerationUtil.generateDV(table, FILE_B) : FILE_B_POS_DELETES; + } + + private DeleteFile fileB2Deletes() { + return formatVersion >= 3 ? FileGenerationUtil.generateDV(table, FILE_B2) : FILE_B2_POS_DELETES; + } + + private DeleteFile fileUnpartitionedDeletes() { + return formatVersion >= 3 + ? FileGenerationUtil.generateDV(table, FILE_UNPARTITIONED) + : FILE_UNPARTITIONED_POS_DELETE; + } + + @TestTemplate + public void testPartitionedDeletesWithLesserSeqNo() { + setupPartitionedTable(); + + // Add Data Files + table.newAppend().appendFile(FILE_B).appendFile(FILE_C).appendFile(FILE_D).commit(); + + // Add Delete Files + DeleteFile fileADeletes = fileADeletes(); + DeleteFile fileA2Deletes = fileA2Deletes(); + DeleteFile fileBDeletes = fileBDeletes(); + DeleteFile fileB2Deletes = fileB2Deletes(); + table + .newRowDelta() + .addDeletes(fileADeletes) + .addDeletes(fileA2Deletes) + .addDeletes(fileBDeletes) + .addDeletes(fileB2Deletes) + .addDeletes(FILE_A_EQ_DELETES) + .addDeletes(FILE_A2_EQ_DELETES) + .addDeletes(FILE_B_EQ_DELETES) + .addDeletes(FILE_B2_EQ_DELETES) + .commit(); + + // Add More Data Files + table + .newAppend() + .appendFile(FILE_A2) + .appendFile(FILE_B2) + .appendFile(FILE_C2) + .appendFile(FILE_D2) + .commit(); + + List> actual = allEntries(); + List> expected = + ImmutableList.of( + Tuple2.apply(1L, FILE_B.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, fileADeletes.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, fileA2Deletes.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, fileBDeletes.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, fileB2Deletes.location()), + Tuple2.apply(3L, FILE_A2.location()), + Tuple2.apply(3L, FILE_B2.location()), + Tuple2.apply(3L, FILE_C2.location()), + Tuple2.apply(3L, FILE_D2.location())); + assertThat(actual).containsExactlyInAnyOrderElementsOf(expected); + + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + + // All Delete files of the FILE A partition should be removed + // because there are no data files in partition with a lesser sequence number + + Set removedDeleteFiles = + StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) + .map(DeleteFile::location) + .collect(Collectors.toSet()); + assertThat(removedDeleteFiles) + .as("Expected 4 delete files removed") + .hasSize(4) + .containsExactlyInAnyOrder( + fileADeletes.location(), + fileA2Deletes.location(), + FILE_A_EQ_DELETES.location(), + FILE_A2_EQ_DELETES.location()); + + List> actualAfter = liveEntries(); + List> expectedAfter = + ImmutableList.of( + Tuple2.apply(1L, FILE_B.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, fileBDeletes.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, fileB2Deletes.location()), + Tuple2.apply(3L, FILE_A2.location()), + Tuple2.apply(3L, FILE_B2.location()), + Tuple2.apply(3L, FILE_C2.location()), + Tuple2.apply(3L, FILE_D2.location())); + assertThat(actualAfter).containsExactlyInAnyOrderElementsOf(expectedAfter); + } + + @TestTemplate + public void testPartitionedDeletesWithEqSeqNo() { + setupPartitionedTable(); + + // Add Data Files + table.newAppend().appendFile(FILE_A).appendFile(FILE_C).appendFile(FILE_D).commit(); + + // Add Data Files with EQ and POS deletes + DeleteFile fileADeletes = fileADeletes(); + DeleteFile fileA2Deletes = fileA2Deletes(); + DeleteFile fileBDeletes = fileBDeletes(); + DeleteFile fileB2Deletes = fileB2Deletes(); + table + .newRowDelta() + .addRows(FILE_A2) + .addRows(FILE_B2) + .addRows(FILE_C2) + .addRows(FILE_D2) + .addDeletes(fileADeletes) + .addDeletes(fileA2Deletes) + .addDeletes(FILE_A_EQ_DELETES) + .addDeletes(FILE_A2_EQ_DELETES) + .addDeletes(fileBDeletes) + .addDeletes(fileB2Deletes) + .addDeletes(FILE_B_EQ_DELETES) + .addDeletes(FILE_B2_EQ_DELETES) + .commit(); + + List> actual = allEntries(); + List> expected = + ImmutableList.of( + Tuple2.apply(1L, FILE_A.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, fileADeletes.location()), + Tuple2.apply(2L, FILE_A2.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, fileA2Deletes.location()), + Tuple2.apply(2L, FILE_B_EQ_DELETES.location()), + Tuple2.apply(2L, fileBDeletes.location()), + Tuple2.apply(2L, FILE_B2.location()), + Tuple2.apply(2L, FILE_B2_EQ_DELETES.location()), + Tuple2.apply(2L, fileB2Deletes.location()), + Tuple2.apply(2L, FILE_C2.location()), + Tuple2.apply(2L, FILE_D2.location())); + assertThat(actual).containsExactlyInAnyOrderElementsOf(expected); + + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + + // Eq Delete files of the FILE B partition should be removed + // because there are no data files in partition with a lesser sequence number + Set removedDeleteFiles = + StreamSupport.stream(result.removedDeleteFiles().spliterator(), false) + .map(DeleteFile::location) + .collect(Collectors.toSet()); + assertThat(removedDeleteFiles) + .as("Expected two delete files removed") + .hasSize(2) + .containsExactlyInAnyOrder(FILE_B_EQ_DELETES.location(), FILE_B2_EQ_DELETES.location()); + + List> actualAfter = liveEntries(); + List> expectedAfter = + ImmutableList.of( + Tuple2.apply(1L, FILE_A.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A_EQ_DELETES.location()), + Tuple2.apply(2L, fileADeletes.location()), + Tuple2.apply(2L, FILE_A2.location()), + Tuple2.apply(2L, FILE_A2_EQ_DELETES.location()), + Tuple2.apply(2L, fileA2Deletes.location()), + Tuple2.apply(2L, fileBDeletes.location()), + Tuple2.apply(2L, FILE_B2.location()), + Tuple2.apply(2L, fileB2Deletes.location()), + Tuple2.apply(2L, FILE_C2.location()), + Tuple2.apply(2L, FILE_D2.location())); + assertThat(actualAfter).containsExactlyInAnyOrderElementsOf(expectedAfter); + } + + @TestTemplate + public void testUnpartitionedTable() { + setupUnpartitionedTable(); + + table + .newRowDelta() + .addDeletes(fileUnpartitionedDeletes()) + .addDeletes(FILE_UNPARTITIONED_EQ_DELETE) + .commit(); + table.newAppend().appendFile(FILE_UNPARTITIONED).commit(); + + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + assertThat(result.removedDeleteFiles()).as("No-op for unpartitioned tables").isEmpty(); + } + + @TestTemplate + public void testPartitionedDeletesWithDanglingDvs() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + setupPartitionedTable(); + + table.newAppend().appendFile(FILE_A).appendFile(FILE_C).appendFile(FILE_D).commit(); + + DeleteFile fileADeletes = fileADeletes(); + DeleteFile fileBDeletes = fileBDeletes(); + DeleteFile fileB2Deletes = fileB2Deletes(); + table + .newRowDelta() + .addRows(FILE_A) + .addRows(FILE_C) + .addDeletes(fileADeletes) + // since FILE B doesn't exist, these delete files will be dangling + .addDeletes(fileBDeletes) + .addDeletes(fileB2Deletes) + .commit(); + + List> actual = allEntries(); + List> expected = + ImmutableList.of( + Tuple2.apply(1L, FILE_A.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A.location()), + Tuple2.apply(2L, FILE_C.location()), + Tuple2.apply(2L, fileB2Deletes.location()), + Tuple2.apply(2L, fileBDeletes.location()), + Tuple2.apply(2L, fileADeletes.location())); + assertThat(actual).containsExactlyInAnyOrderElementsOf(expected); + + RemoveDanglingDeleteFiles.Result result = + SparkActions.get().removeDanglingDeleteFiles(table).execute(); + + // DVs of FILE B should be removed because they don't point to valid data files + Set removedDeleteFiles = + Lists.newArrayList(result.removedDeleteFiles()).stream() + .map(DeleteFile::location) + .collect(Collectors.toSet()); + assertThat(removedDeleteFiles) + .as("Expected two delete files to be removed") + .hasSize(2) + .containsExactlyInAnyOrder(fileBDeletes.location(), fileB2Deletes.location()); + + List> actualAfter = liveEntries(); + List> expectedAfter = + ImmutableList.of( + Tuple2.apply(1L, FILE_A.location()), + Tuple2.apply(1L, FILE_C.location()), + Tuple2.apply(1L, FILE_D.location()), + Tuple2.apply(2L, FILE_A.location()), + Tuple2.apply(2L, FILE_C.location()), + Tuple2.apply(2L, fileADeletes.location())); + assertThat(actualAfter).containsExactlyInAnyOrderElementsOf(expectedAfter); + } + + private List> liveEntries() { + return spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .filter("status < 2") // live files + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + } + + private List> allEntries() { + return spark + .read() + .format("iceberg") + .load(tableLocation + "#entries") + .select("sequence_number", "data_file.file_path") + .sort("sequence_number", "data_file.file_path") + .as(Encoders.tuple(Encoders.LONG(), Encoders.STRING())) + .collectAsList(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java new file mode 100644 index 000000000000..40505b856737 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -0,0 +1,1308 @@ +/* + * 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.actions; + +import static org.apache.iceberg.spark.actions.DeleteOrphanFilesSparkAction.MAX_ORPHAN_FILE_SAMPLE_SIZE; +import static org.apache.iceberg.spark.actions.DeleteOrphanFilesSparkAction.STREAM_RESULTS; +import static org.apache.iceberg.types.Types.NestedField.optional; +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 static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyMap; +import static org.mockito.ArgumentMatchers.anyString; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Files; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.hadoop.HiddenPathFilter; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; +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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkSQLProperties; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.spark.actions.DeleteOrphanFilesSparkAction.StringToFileURI; +import org.apache.iceberg.spark.source.FilePathLastModifiedRecord; +import org.apache.iceberg.spark.source.ThreeColumnRecord; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.FileSystemWalker; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.MockedStatic; +import org.mockito.Mockito; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TestRemoveOrphanFilesAction extends TestBase { + + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + protected static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + protected static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).truncate("c2", 2).identity("c3").build(); + + @TempDir private File tableDir = null; + protected String tableLocation = null; + protected Map properties; + @Parameter private int formatVersion; + + @Parameter(index = 1) + private boolean usePrefixListing; + + @Parameters(name = "formatVersion = {0}, usePrefixListing = {1}") + protected static List parameters() { + return TestHelpers.ALL_VERSIONS.stream() + .filter(version -> version > 1) + .flatMap(version -> Stream.of(new Object[] {version, true}, new Object[] {version, false})) + .collect(Collectors.toList()); + } + + @BeforeEach + public void setupTableLocation() throws Exception { + this.tableLocation = tableDir.toURI().toString(); + properties = ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + } + + @TestTemplate + public void testDryRun() throws IOException { + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), properties, tableLocation); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + List validFiles = + spark + .read() + .format("iceberg") + .load(tableLocation + "#files") + .select("file_path") + .as(Encoders.STRING()) + .collectAsList(); + assertThat(validFiles).as("Should be 2 valid files").hasSize(2); + + df.write().mode("append").parquet(tableLocation + "/data"); + + Path dataPath = new Path(tableLocation + "/data"); + FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); + List allFiles = + Arrays.stream(fs.listStatus(dataPath, HiddenPathFilter.get())) + .filter(FileStatus::isFile) + .map(file -> file.getPath().toString()) + .collect(Collectors.toList()); + assertThat(allFiles).as("Should be 3 valid files").hasSize(3); + + List invalidFiles = Lists.newArrayList(allFiles); + invalidFiles.removeAll(validFiles); + assertThat(invalidFiles).as("Should be 1 invalid file").hasSize(1); + + waitUntilAfter(System.currentTimeMillis()); + + SparkActions actions = SparkActions.get(); + + DeleteOrphanFiles.Result result1 = + actions + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .deleteWith(s -> {}) + .execute(); + assertThat(result1.orphanFileLocations()) + .as("Default olderThan interval should be safe") + .isEmpty(); + + DeleteOrphanFiles.Result result2 = + actions + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .deleteWith(s -> {}) + .execute(); + assertThat(result2.orphanFileLocations()) + .as("Action should find 1 file") + .isEqualTo(invalidFiles); + assertThat(fs.exists(new Path(invalidFiles.get(0)))) + .as("Invalid file should be present") + .isTrue(); + + DeleteOrphanFiles.Result result3 = + actions + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .option(STREAM_RESULTS, "true") + .deleteWith(s -> {}) + .execute(); + assertThat(result3.orphanFileLocations()) + .as("Streaming dry run should find 1 file") + .isEqualTo(invalidFiles); + assertThat(fs.exists(new Path(invalidFiles.get(0)))) + .as("Invalid file should be present after streaming dry run") + .isTrue(); + + DeleteOrphanFiles.Result result4 = + actions + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .execute(); + assertThat(result4.orphanFileLocations()) + .as("Action should delete 1 file") + .isEqualTo(invalidFiles); + assertThat(fs.exists(new Path(invalidFiles.get(0)))) + .as("Invalid file should not be present") + .isFalse(); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(records); + expectedRecords.addAll(records); + + Dataset resultDF = spark.read().format("iceberg").load(tableLocation); + List actualRecords = + resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + assertThat(actualRecords).isEqualTo(expectedRecords); + } + + @TestTemplate + public void testAllValidFilesAreKept() throws IOException { + Table table = TABLES.create(SCHEMA, SPEC, properties, tableLocation); + + List records1 = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + Dataset df1 = spark.createDataFrame(records1, ThreeColumnRecord.class).coalesce(1); + + // original append + df1.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + List records2 = + Lists.newArrayList(new ThreeColumnRecord(2, "AAAAAAAAAA", "AAAA")); + Dataset df2 = spark.createDataFrame(records2, ThreeColumnRecord.class).coalesce(1); + + // dynamic partition overwrite + df2.select("c1", "c2", "c3").write().format("iceberg").mode("overwrite").save(tableLocation); + + // second append + df2.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + List snapshots = Lists.newArrayList(table.snapshots()); + + List snapshotFiles1 = snapshotFiles(snapshots.get(0).snapshotId()); + assertThat(snapshotFiles1).hasSize(1); + + List snapshotFiles2 = snapshotFiles(snapshots.get(1).snapshotId()); + assertThat(snapshotFiles2).hasSize(1); + + List snapshotFiles3 = snapshotFiles(snapshots.get(2).snapshotId()); + assertThat(snapshotFiles3).hasSize(2); + + df2.coalesce(1).write().mode("append").parquet(tableLocation + "/data"); + df2.coalesce(1).write().mode("append").parquet(tableLocation + "/data/c2_trunc=AA"); + df2.coalesce(1).write().mode("append").parquet(tableLocation + "/data/c2_trunc=AA/c3=AAAA"); + df2.coalesce(1).write().mode("append").parquet(tableLocation + "/data/invalid/invalid"); + + waitUntilAfter(System.currentTimeMillis()); + + SparkActions actions = SparkActions.get(); + + DeleteOrphanFiles.Result result = + actions + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .execute(); + + assertThat(result.orphanFileLocations()).as("Should delete 4 files").hasSize(4); + + Path dataPath = new Path(tableLocation + "/data"); + FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); + + for (String fileLocation : snapshotFiles1) { + assertThat(fs.exists(new Path(fileLocation))).as("All snapshot files must remain").isTrue(); + } + + for (String fileLocation : snapshotFiles2) { + assertThat(fs.exists(new Path(fileLocation))).as("All snapshot files must remain").isTrue(); + } + + for (String fileLocation : snapshotFiles3) { + assertThat(fs.exists(new Path(fileLocation))).as("All snapshot files must remain").isTrue(); + } + } + + @TestTemplate + public void orphanedFileRemovedWithParallelTasks() { + assumeThat(usePrefixListing) + .as("Should not test both prefix listing and Hadoop file listing (redundant)") + .isEqualTo(false); + Table table = TABLES.create(SCHEMA, SPEC, properties, tableLocation); + + List records1 = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + Dataset df1 = spark.createDataFrame(records1, ThreeColumnRecord.class).coalesce(1); + + // original append + df1.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + List records2 = + Lists.newArrayList(new ThreeColumnRecord(2, "AAAAAAAAAA", "AAAA")); + Dataset df2 = spark.createDataFrame(records2, ThreeColumnRecord.class).coalesce(1); + + // dynamic partition overwrite + df2.select("c1", "c2", "c3").write().format("iceberg").mode("overwrite").save(tableLocation); + + // second append + df2.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + df2.coalesce(1).write().mode("append").parquet(tableLocation + "/data"); + df2.coalesce(1).write().mode("append").parquet(tableLocation + "/data/c2_trunc=AA"); + df2.coalesce(1).write().mode("append").parquet(tableLocation + "/data/c2_trunc=AA/c3=AAAA"); + df2.coalesce(1).write().mode("append").parquet(tableLocation + "/data/invalid/invalid"); + + waitUntilAfter(System.currentTimeMillis()); + + Set deletedFiles = ConcurrentHashMap.newKeySet(); + Set deleteThreads = ConcurrentHashMap.newKeySet(); + AtomicInteger deleteThreadsIndex = new AtomicInteger(0); + + ExecutorService executorService = + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("remove-orphan-" + deleteThreadsIndex.getAndIncrement()); + thread.setDaemon(true); + return thread; + }); + + DeleteOrphanFiles.Result result = + SparkActions.get() + .deleteOrphanFiles(table) + .executeDeleteWith(executorService) + .olderThan(System.currentTimeMillis() + 5000) // Ensure all orphan files are selected + .deleteWith( + file -> { + deleteThreads.add(Thread.currentThread().getName()); + deletedFiles.add(file); + }) + .execute(); + + // Verifies that the delete methods ran in the threads created by the provided ExecutorService + // ThreadFactory + assertThat(deleteThreads) + .containsExactlyInAnyOrder( + "remove-orphan-0", "remove-orphan-1", "remove-orphan-2", "remove-orphan-3"); + assertThat(deletedFiles).hasSize(4); + } + + @TestTemplate + public void testWapFilesAreKept() { + assumeThat(usePrefixListing) + .as("Should not test both prefix listing and Hadoop file listing (redundant)") + .isEqualTo(false); + assumeThat(formatVersion).as("currently fails with DVs").isEqualTo(2); + Map props = Maps.newHashMap(); + props.put(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED, "true"); + props.putAll(properties); + Table table = TABLES.create(SCHEMA, SPEC, props, tableLocation); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); + + // normal write + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + withSQLConf( + Map.of(SparkSQLProperties.WAP_ID, "1"), + () -> { + // wap write + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + Dataset resultDF = spark.read().format("iceberg").load(tableLocation); + List actualRecords = + resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + + // TODO: currently fails because DVs delete stuff from WAP branch + assertThat(actualRecords) + .as("Should not return data from the staged snapshot") + .isEqualTo(records); + }); + + waitUntilAfter(System.currentTimeMillis()); + + SparkActions actions = SparkActions.get(); + + DeleteOrphanFiles.Result result = + actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); + + assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + } + + @TestTemplate + public void testMetadataFolderIsIntact() { + // write data directly to the table location + Map props = Maps.newHashMap(); + props.put(TableProperties.WRITE_DATA_LOCATION, tableLocation); + props.putAll(properties); + Table table = TABLES.create(SCHEMA, SPEC, props, tableLocation); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + df.write().mode("append").parquet(tableLocation + "/c2_trunc=AA/c3=AAAA"); + + waitUntilAfter(System.currentTimeMillis()); + + SparkActions actions = SparkActions.get(); + + DeleteOrphanFiles.Result result = + actions + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .execute(); + + assertThat(result.orphanFileLocations()).as("Should delete 1 file").hasSize(1); + + Dataset resultDF = spark.read().format("iceberg").load(tableLocation); + List actualRecords = + resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + assertThat(actualRecords).as("Rows must match").isEqualTo(records); + } + + @TestTemplate + public void testOlderThanTimestamp() { + Table table = TABLES.create(SCHEMA, SPEC, properties, tableLocation); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + df.write().mode("append").parquet(tableLocation + "/data/c2_trunc=AA/c3=AAAA"); + df.write().mode("append").parquet(tableLocation + "/data/c2_trunc=AA/c3=AAAA"); + + waitUntilAfter(System.currentTimeMillis()); + + long timestamp = System.currentTimeMillis(); + + waitUntilAfter(System.currentTimeMillis() + 1000L); + + df.write().mode("append").parquet(tableLocation + "/data/c2_trunc=AA/c3=AAAA"); + + SparkActions actions = SparkActions.get(); + + DeleteOrphanFiles.Result result = + actions + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(timestamp) + .execute(); + + assertThat(result.orphanFileLocations()).as("Should delete only 2 files").hasSize(2); + } + + @TestTemplate + public void testRemoveUnreachableMetadataVersionFiles() { + Map props = Maps.newHashMap(); + props.put(TableProperties.WRITE_DATA_LOCATION, tableLocation); + props.put(TableProperties.METADATA_PREVIOUS_VERSIONS_MAX, "1"); + props.putAll(properties); + Table table = TABLES.create(SCHEMA, SPEC, props, tableLocation); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); + + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + waitUntilAfter(System.currentTimeMillis()); + + SparkActions actions = SparkActions.get(); + + DeleteOrphanFiles.Result result = + actions + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .execute(); + + assertThat(result.orphanFileLocations()) + .containsExactly(tableLocation + "metadata/v1.metadata.json"); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(records); + expectedRecords.addAll(records); + + Dataset resultDF = spark.read().format("iceberg").load(tableLocation); + List actualRecords = + resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); + } + + @TestTemplate + public void testManyTopLevelPartitions() { + Table table = TABLES.create(SCHEMA, SPEC, properties, tableLocation); + + List records = Lists.newArrayList(); + for (int i = 0; i < 100; i++) { + records.add(new ThreeColumnRecord(i, String.valueOf(i), String.valueOf(i))); + } + + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); + + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + waitUntilAfter(System.currentTimeMillis()); + + SparkActions actions = SparkActions.get(); + + DeleteOrphanFiles.Result result = + actions + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .execute(); + + assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + + Dataset resultDF = spark.read().format("iceberg").load(tableLocation); + assertThat(resultDF.count()).as("Rows count must match").isEqualTo(records.size()); + } + + @TestTemplate + public void testManyLeafPartitions() { + Table table = TABLES.create(SCHEMA, SPEC, properties, tableLocation); + + List records = Lists.newArrayList(); + for (int i = 0; i < 100; i++) { + records.add(new ThreeColumnRecord(i, String.valueOf(i % 3), String.valueOf(i))); + } + + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); + + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + waitUntilAfter(System.currentTimeMillis()); + + SparkActions actions = SparkActions.get(); + + DeleteOrphanFiles.Result result = + actions + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .execute(); + + assertThat(result.orphanFileLocations()).as("Should not delete any files").isEmpty(); + + Dataset resultDF = spark.read().format("iceberg").load(tableLocation); + assertThat(resultDF.count()).as("Row count must match").isEqualTo(records.size()); + } + + @TestTemplate + public void testHiddenPartitionPaths() { + Schema schema = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "_c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + PartitionSpec spec = PartitionSpec.builderFor(schema).truncate("_c2", 2).identity("c3").build(); + Table table = TABLES.create(schema, spec, properties, tableLocation); + + StructType structType = + new StructType() + .add("c1", DataTypes.IntegerType) + .add("_c2", DataTypes.StringType) + .add("c3", DataTypes.StringType); + List records = Lists.newArrayList(RowFactory.create(1, "AAAAAAAAAA", "AAAA")); + Dataset df = spark.createDataFrame(records, structType).coalesce(1); + + df.select("c1", "_c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + df.write().mode("append").parquet(tableLocation + "/data/_c2_trunc=AA/c3=AAAA"); + df.write().mode("append").parquet(tableLocation + "/data/_c2_trunc=AA/c3=AAAA"); + + waitUntilAfter(System.currentTimeMillis()); + + SparkActions actions = SparkActions.get(); + + DeleteOrphanFiles.Result result = + actions + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .execute(); + + assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); + } + + @TestTemplate + public void testHiddenPartitionPathsWithPartitionEvolution() { + Schema schema = + new Schema( + optional(1, "_c1", Types.IntegerType.get()), + optional(2, "_c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + PartitionSpec spec = PartitionSpec.builderFor(schema).truncate("_c2", 2).build(); + Table table = TABLES.create(schema, spec, properties, tableLocation); + + StructType structType = + new StructType() + .add("_c1", DataTypes.IntegerType) + .add("_c2", DataTypes.StringType) + .add("c3", DataTypes.StringType); + List records = Lists.newArrayList(RowFactory.create(1, "AAAAAAAAAA", "AAAA")); + Dataset df = spark.createDataFrame(records, structType).coalesce(1); + + df.select("_c1", "_c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + df.write().mode("append").parquet(tableLocation + "/data/_c2_trunc=AA"); + + table.updateSpec().addField("_c1").commit(); + + df.write().mode("append").parquet(tableLocation + "/data/_c2_trunc=AA/_c1=1"); + + waitUntilAfter(System.currentTimeMillis()); + + SparkActions actions = SparkActions.get(); + + DeleteOrphanFiles.Result result = + actions + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .execute(); + + assertThat(result.orphanFileLocations()).as("Should delete 2 files").hasSize(2); + } + + @TestTemplate + public void testHiddenPathsStartingWithPartitionNamesAreIgnored() throws IOException { + Schema schema = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "_c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + PartitionSpec spec = PartitionSpec.builderFor(schema).truncate("_c2", 2).identity("c3").build(); + Table table = TABLES.create(schema, spec, properties, tableLocation); + + StructType structType = + new StructType() + .add("c1", DataTypes.IntegerType) + .add("_c2", DataTypes.StringType) + .add("c3", DataTypes.StringType); + List records = Lists.newArrayList(RowFactory.create(1, "AAAAAAAAAA", "AAAA")); + Dataset df = spark.createDataFrame(records, structType).coalesce(1); + + df.select("c1", "_c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + Path dataPath = new Path(tableLocation + "/data"); + FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); + Path pathToFileInHiddenFolder = new Path(dataPath, "_c2_trunc/file.txt"); + fs.createNewFile(pathToFileInHiddenFolder); + + waitUntilAfter(System.currentTimeMillis()); + + SparkActions actions = SparkActions.get(); + + DeleteOrphanFiles.Result result = + actions + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .execute(); + + assertThat(result.orphanFileLocations()).as("Should delete 0 files").isEmpty(); + assertThat(fs.exists(pathToFileInHiddenFolder)).isTrue(); + } + + private List snapshotFiles(long snapshotId) { + return spark + .read() + .format("iceberg") + .option("snapshot-id", snapshotId) + .load(tableLocation + "#files") + .select("file_path") + .as(Encoders.STRING()) + .collectAsList(); + } + + @TestTemplate + public void testRemoveOrphanFilesWithRelativeFilePath() throws IOException { + assumeThat(usePrefixListing) + .as("Should not test both prefix listing and Hadoop file listing (redundant)") + .isEqualTo(false); + Table table = + TABLES.create( + SCHEMA, PartitionSpec.unpartitioned(), properties, tableDir.getAbsolutePath()); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + + df.select("c1", "c2", "c3") + .write() + .format("iceberg") + .mode("append") + .save(tableDir.getAbsolutePath()); + + List validFiles = + spark + .read() + .format("iceberg") + .load(tableLocation + "#files") + .select("file_path") + .as(Encoders.STRING()) + .collectAsList(); + assertThat(validFiles).as("Should be 1 valid file").hasSize(1); + String validFile = validFiles.get(0); + + df.write().mode("append").parquet(tableLocation + "/data"); + + Path dataPath = new Path(tableLocation + "/data"); + FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); + List allFiles = + Arrays.stream(fs.listStatus(dataPath, HiddenPathFilter.get())) + .filter(FileStatus::isFile) + .map(file -> file.getPath().toString()) + .collect(Collectors.toList()); + assertThat(allFiles).as("Should be 2 files").hasSize(2); + + List invalidFiles = Lists.newArrayList(allFiles); + invalidFiles.removeIf(file -> file.contains(validFile)); + assertThat(invalidFiles).as("Should be 1 invalid file").hasSize(1); + + waitUntilAfter(System.currentTimeMillis()); + + SparkActions actions = SparkActions.get(); + DeleteOrphanFiles.Result result = + actions + .deleteOrphanFiles(table) + .olderThan(System.currentTimeMillis()) + .deleteWith(s -> {}) + .execute(); + assertThat(result.orphanFileLocations()) + .as("Action should find 1 file") + .isEqualTo(invalidFiles); + assertThat(fs.exists(new Path(invalidFiles.get(0)))) + .as("Invalid file should be present") + .isTrue(); + } + + @TestTemplate + public void testRemoveOrphanFilesWithHadoopCatalog() throws InterruptedException { + HadoopCatalog catalog = new HadoopCatalog(new Configuration(), tableLocation); + String namespaceName = "testDb"; + String tableName = "testTb"; + + Namespace namespace = Namespace.of(namespaceName); + TableIdentifier tableIdentifier = TableIdentifier.of(namespace, tableName); + Table table = + catalog.createTable( + tableIdentifier, SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap()); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(table.location()); + + df.write().mode("append").parquet(table.location() + "/data"); + + waitUntilAfter(System.currentTimeMillis()); + + table.refresh(); + + DeleteOrphanFiles.Result result = + SparkActions.get() + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .execute(); + + assertThat(result.orphanFileLocations()).as("Should delete only 1 file").hasSize(1); + + Dataset resultDF = spark.read().format("iceberg").load(table.location()); + List actualRecords = + resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + assertThat(actualRecords).as("Rows must match").isEqualTo(records); + } + + @TestTemplate + public void testHiveCatalogTable() throws IOException { + TableIdentifier identifier = TableIdentifier.of("default", randomName("hivetestorphan")); + Table table = catalog.createTable(identifier, SCHEMA, SPEC, tableLocation, properties); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + + df.select("c1", "c2", "c3") + .write() + .format("iceberg") + .mode("append") + .save(identifier.toString()); + + String location = table.location().replaceFirst("file:", ""); + new File(location + "/data/trashfile").createNewFile(); + + DeleteOrphanFiles.Result result = + SparkActions.get() + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis() + 1000) + .execute(); + assertThat(result.orphanFileLocations()) + .as("trash file should be removed") + .contains("file:" + location + "/data/trashfile"); + } + + @TestTemplate + public void testGarbageCollectionDisabled() { + assumeThat(usePrefixListing) + .as("Should not test both prefix listing and Hadoop file listing (redundant)") + .isEqualTo(false); + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), properties, tableLocation); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); + + assertThatThrownBy(() -> SparkActions.get().deleteOrphanFiles(table).execute()) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Cannot delete orphan files: GC is disabled (deleting files may corrupt other tables)"); + } + + @TestTemplate + public void testCompareToFileList() throws IOException { + assumeThat(usePrefixListing) + .as("Should not test both prefix listing and Hadoop file listing (redundant)") + .isEqualTo(false); + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), properties, tableLocation); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + Path dataPath = new Path(tableLocation + "/data"); + FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); + List validFiles = + Arrays.stream(fs.listStatus(dataPath, HiddenPathFilter.get())) + .filter(FileStatus::isFile) + .map( + file -> + new FilePathLastModifiedRecord( + file.getPath().toString(), new Timestamp(file.getModificationTime()))) + .collect(Collectors.toList()); + + assertThat(validFiles).as("Should be 2 valid files").hasSize(2); + + df.write().mode("append").parquet(tableLocation + "/data"); + + List allFiles = + Arrays.stream(fs.listStatus(dataPath, HiddenPathFilter.get())) + .filter(FileStatus::isFile) + .map( + file -> + new FilePathLastModifiedRecord( + file.getPath().toString(), new Timestamp(file.getModificationTime()))) + .collect(Collectors.toList()); + + assertThat(allFiles).as("Should be 3 files").hasSize(3); + + List invalidFiles = Lists.newArrayList(allFiles); + invalidFiles.removeAll(validFiles); + List invalidFilePaths = + invalidFiles.stream() + .map(FilePathLastModifiedRecord::getFilePath) + .collect(Collectors.toList()); + assertThat(invalidFiles).as("Should be 1 invalid file").hasSize(1); + + // sleep for 1 second to ensure files will be old enough + waitUntilAfter(System.currentTimeMillis()); + + SparkActions actions = SparkActions.get(); + + Dataset compareToFileList = + spark + .createDataFrame(allFiles, FilePathLastModifiedRecord.class) + .withColumnRenamed("filePath", "file_path") + .withColumnRenamed("lastModified", "last_modified"); + + DeleteOrphanFiles.Result result1 = + actions + .deleteOrphanFiles(table) + .compareToFileList(compareToFileList) + .deleteWith(s -> {}) + .execute(); + assertThat(result1.orphanFileLocations()) + .as("Default olderThan interval should be safe") + .isEmpty(); + + DeleteOrphanFiles.Result result2 = + actions + .deleteOrphanFiles(table) + .compareToFileList(compareToFileList) + .olderThan(System.currentTimeMillis()) + .deleteWith(s -> {}) + .execute(); + assertThat(result2.orphanFileLocations()) + .as("Action should find 1 file") + .isEqualTo(invalidFilePaths); + assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) + .as("Invalid file should be present") + .isTrue(); + + DeleteOrphanFiles.Result result3 = + actions + .deleteOrphanFiles(table) + .compareToFileList(compareToFileList) + .olderThan(System.currentTimeMillis()) + .execute(); + assertThat(result3.orphanFileLocations()) + .as("Action should delete 1 file") + .isEqualTo(invalidFilePaths); + assertThat(fs.exists(new Path(invalidFilePaths.get(0)))) + .as("Invalid file should not be present") + .isFalse(); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(records); + expectedRecords.addAll(records); + + Dataset resultDF = spark.read().format("iceberg").load(tableLocation); + List actualRecords = + resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); + + List outsideLocationMockFiles = + Lists.newArrayList(new FilePathLastModifiedRecord("/tmp/mock1", new Timestamp(0L))); + + Dataset compareToFileListWithOutsideLocation = + spark + .createDataFrame(outsideLocationMockFiles, FilePathLastModifiedRecord.class) + .withColumnRenamed("filePath", "file_path") + .withColumnRenamed("lastModified", "last_modified"); + + DeleteOrphanFiles.Result result4 = + actions + .deleteOrphanFiles(table) + .compareToFileList(compareToFileListWithOutsideLocation) + .deleteWith(s -> {}) + .execute(); + assertThat(result4.orphanFileLocations()).as("Action should find nothing").isEmpty(); + } + + protected long waitUntilAfter(long timestampMillis) { + long current = System.currentTimeMillis(); + while (current <= timestampMillis) { + current = System.currentTimeMillis(); + } + return current; + } + + @TestTemplate + public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { + assumeThat(usePrefixListing) + .as("Should not test both prefix listing and Hadoop file listing (redundant)") + .isEqualTo(false); + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), properties, tableLocation); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + table.refresh(); + long snapshotId = table.currentSnapshot().snapshotId(); + long snapshotSequenceNumber = table.currentSnapshot().sequenceNumber(); + + File statsLocation = + new File(new URI(tableLocation)) + .toPath() + .resolve("data") + .resolve("some-stats-file") + .toFile(); + StatisticsFile statisticsFile; + try (PuffinWriter puffinWriter = Puffin.write(Files.localOutput(statsLocation)).build()) { + puffinWriter.add( + new Blob( + "some-blob-type", + ImmutableList.of(1), + snapshotId, + snapshotSequenceNumber, + ByteBuffer.wrap("blob content".getBytes(StandardCharsets.UTF_8)))); + puffinWriter.finish(); + statisticsFile = + new GenericStatisticsFile( + snapshotId, + statsLocation.toString(), + puffinWriter.fileSize(), + puffinWriter.footerSize(), + puffinWriter.writtenBlobsMetadata().stream() + .map(GenericBlobMetadata::from) + .collect(ImmutableList.toImmutableList())); + } + + Transaction transaction = table.newTransaction(); + transaction.updateStatistics().setStatistics(statisticsFile).commit(); + transaction.commitTransaction(); + + SparkActions.get() + .deleteOrphanFiles(table) + .olderThan(System.currentTimeMillis() + 1000) + .execute(); + + assertThat(statsLocation).as("stats file should exist").exists(); + assertThat(statsLocation.length()) + .as("stats file length") + .isEqualTo(statisticsFile.fileSizeInBytes()); + + transaction = table.newTransaction(); + transaction.updateStatistics().removeStatistics(statisticsFile.snapshotId()).commit(); + transaction.commitTransaction(); + + DeleteOrphanFiles.Result result = + SparkActions.get() + .deleteOrphanFiles(table) + .olderThan(System.currentTimeMillis() + 1000) + .execute(); + Iterable orphanFileLocations = result.orphanFileLocations(); + assertThat(orphanFileLocations).hasSize(1).containsExactly(statsLocation.toURI().toString()); + assertThat(statsLocation).as("stats file should be deleted").doesNotExist(); + } + + @TestTemplate + public void testPathsWithExtraSlashes() { + List validFiles = Lists.newArrayList("file:///dir1/dir2/file1"); + List actualFiles = Lists.newArrayList("file:///dir1/////dir2///file1"); + executeTest(validFiles, actualFiles, Lists.newArrayList()); + } + + @TestTemplate + public void testPathsWithValidFileHavingNoAuthority() { + List validFiles = Lists.newArrayList("hdfs:///dir1/dir2/file1"); + List actualFiles = Lists.newArrayList("hdfs://servicename/dir1/dir2/file1"); + executeTest(validFiles, actualFiles, Lists.newArrayList()); + } + + @TestTemplate + public void testPathsWithActualFileHavingNoAuthority() { + List validFiles = Lists.newArrayList("hdfs://servicename/dir1/dir2/file1"); + List actualFiles = Lists.newArrayList("hdfs:///dir1/dir2/file1"); + executeTest(validFiles, actualFiles, Lists.newArrayList()); + } + + @TestTemplate + public void testPathsWithEqualSchemes() { + List validFiles = Lists.newArrayList("scheme1://bucket1/dir1/dir2/file1"); + List actualFiles = Lists.newArrayList("scheme2://bucket1/dir1/dir2/file1"); + assertThatThrownBy( + () -> + executeTest( + validFiles, + actualFiles, + Lists.newArrayList(), + ImmutableMap.of(), + ImmutableMap.of(), + DeleteOrphanFiles.PrefixMismatchMode.ERROR)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Unable to determine whether certain files are orphan") + .hasMessageEndingWith("Conflicting authorities/schemes: [(scheme1, scheme2)]."); + + Map equalSchemes = Maps.newHashMap(); + equalSchemes.put("scheme1", "scheme"); + equalSchemes.put("scheme2", "scheme"); + executeTest( + validFiles, + actualFiles, + Lists.newArrayList(), + equalSchemes, + ImmutableMap.of(), + DeleteOrphanFiles.PrefixMismatchMode.ERROR); + } + + @TestTemplate + public void testPathsWithEqualAuthorities() { + List validFiles = Lists.newArrayList("hdfs://servicename1/dir1/dir2/file1"); + List actualFiles = Lists.newArrayList("hdfs://servicename2/dir1/dir2/file1"); + assertThatThrownBy( + () -> + executeTest( + validFiles, + actualFiles, + Lists.newArrayList(), + ImmutableMap.of(), + ImmutableMap.of(), + DeleteOrphanFiles.PrefixMismatchMode.ERROR)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Unable to determine whether certain files are orphan") + .hasMessageEndingWith("Conflicting authorities/schemes: [(servicename1, servicename2)]."); + + Map equalAuthorities = Maps.newHashMap(); + equalAuthorities.put("servicename1", "servicename"); + equalAuthorities.put("servicename2", "servicename"); + executeTest( + validFiles, + actualFiles, + Lists.newArrayList(), + ImmutableMap.of(), + equalAuthorities, + DeleteOrphanFiles.PrefixMismatchMode.ERROR); + } + + @TestTemplate + public void testRemoveOrphanFileActionWithDeleteMode() { + List validFiles = Lists.newArrayList("hdfs://servicename1/dir1/dir2/file1"); + List actualFiles = Lists.newArrayList("hdfs://servicename2/dir1/dir2/file1"); + + executeTest( + validFiles, + actualFiles, + Lists.newArrayList("hdfs://servicename2/dir1/dir2/file1"), + ImmutableMap.of(), + ImmutableMap.of(), + DeleteOrphanFiles.PrefixMismatchMode.DELETE); + } + + @TestTemplate + public void testDefaultToHadoopListing() { + assumeThat(usePrefixListing) + .as( + "This test verifies default listing behavior and does not require prefix listing to be enabled.") + .isEqualTo(false); + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), properties, tableLocation); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + DeleteOrphanFilesSparkAction deleteOrphanFilesSparkAction = + SparkActions.get().deleteOrphanFiles(table); + DeleteOrphanFilesSparkAction spyAction = Mockito.spy(deleteOrphanFilesSparkAction); + try (MockedStatic mockedStatic = Mockito.mockStatic(FileSystemWalker.class)) { + spyAction.execute(); + mockedStatic.verify( + () -> + FileSystemWalker.listDirRecursivelyWithHadoop( + anyString(), + anyMap(), + any(Predicate.class), + any(Configuration.class), + anyInt(), + anyInt(), + any(), + any())); + } + } + + protected String randomName(String prefix) { + return prefix + UUID.randomUUID().toString().replace("-", ""); + } + + private void executeTest( + List validFiles, List actualFiles, List expectedOrphanFiles) { + executeTest( + validFiles, + actualFiles, + expectedOrphanFiles, + ImmutableMap.of(), + ImmutableMap.of(), + DeleteOrphanFiles.PrefixMismatchMode.IGNORE); + } + + private void executeTest( + List validFiles, + List actualFiles, + List expectedOrphanFiles, + Map equalSchemes, + Map equalAuthorities, + DeleteOrphanFiles.PrefixMismatchMode mode) { + + StringToFileURI toFileUri = new StringToFileURI(equalSchemes, equalAuthorities); + + Dataset validFileDS = spark.createDataset(validFiles, Encoders.STRING()); + Dataset actualFileDS = spark.createDataset(actualFiles, Encoders.STRING()); + + Dataset orphanFileDS = + DeleteOrphanFilesSparkAction.findOrphanFiles( + toFileUri.apply(actualFileDS), toFileUri.apply(validFileDS), mode); + + List orphanFiles = orphanFileDS.collectAsList(); + orphanFileDS.unpersist(); + + assertThat(orphanFiles).isEqualTo(expectedOrphanFiles); + } + + @TestTemplate + public void testStreamResultsDeletion() throws IOException { + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), properties, tableLocation); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + + List validFiles = + spark + .read() + .format("iceberg") + .load(tableLocation + "#files") + .select("file_path") + .as(Encoders.STRING()) + .collectAsList(); + assertThat(validFiles).as("Should be 1 valid file").hasSize(1); + + for (int i = 0; i < 10; i++) { + df.write().mode("append").parquet(tableLocation + "/data"); + } + + Path dataPath = new Path(tableLocation + "/data"); + FileSystem fs = dataPath.getFileSystem(spark.sessionState().newHadoopConf()); + List allFiles = + Arrays.stream(fs.listStatus(dataPath, HiddenPathFilter.get())) + .filter(FileStatus::isFile) + .map(file -> file.getPath().toString()) + .collect(Collectors.toList()); + assertThat(allFiles).as("Should be 11 files").hasSize(11); + + List invalidFiles = Lists.newArrayList(allFiles); + invalidFiles.removeAll(validFiles); + assertThat(invalidFiles).as("Should be 10 invalid files").hasSize(10); + + waitUntilAfter(System.currentTimeMillis()); + + DeleteOrphanFiles.Result nonStreamingResult = + SparkActions.get() + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .deleteWith(s -> {}) + .execute(); + + assertThat(nonStreamingResult.orphanFileLocations()) + .as("Non-streaming dry-run should return all 10 orphan files") + .hasSize(10) + .containsExactlyInAnyOrderElementsOf(invalidFiles); + + DeleteOrphanFiles.Result streamingResult = + SparkActions.get() + .deleteOrphanFiles(table) + .usePrefixListing(usePrefixListing) + .olderThan(System.currentTimeMillis()) + .option(STREAM_RESULTS, "true") + .option(MAX_ORPHAN_FILE_SAMPLE_SIZE, "5") + .execute(); + + assertThat(streamingResult.orphanFileLocations()) + .as("Streaming with sample size 5 should return only 5 orphan files") + .hasSize(5); + + for (String invalidFile : invalidFiles) { + assertThat(fs.exists(new Path(invalidFile))).as("Orphan file should be deleted").isFalse(); + } + + Dataset resultDF = spark.read().format("iceberg").load(tableLocation); + List actualRecords = + resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + assertThat(actualRecords).isEqualTo(records); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java new file mode 100644 index 000000000000..5f98287951f1 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction3.java @@ -0,0 +1,194 @@ +/* + * 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.actions; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.util.stream.StreamSupport; +import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.spark.source.SparkTable; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.expressions.Transform; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestRemoveOrphanFilesAction3 extends TestRemoveOrphanFilesAction { + @TestTemplate + public void testSparkCatalogTable() throws Exception { + spark.conf().set("spark.sql.catalog.mycat", "org.apache.iceberg.spark.SparkCatalog"); + spark.conf().set("spark.sql.catalog.mycat.type", "hadoop"); + spark.conf().set("spark.sql.catalog.mycat.warehouse", tableLocation); + SparkCatalog cat = (SparkCatalog) spark.sessionState().catalogManager().catalog("mycat"); + + String[] database = {"default"}; + Identifier id = Identifier.of(database, randomName("table")); + Transform[] transforms = {}; + cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); + SparkTable table = (SparkTable) cat.loadTable(id); + + sql("INSERT INTO mycat.default.%s VALUES (1,1,1)", id.name()); + + String location = table.table().location().replaceFirst("file:", ""); + String trashFile = randomName("/data/trashfile"); + new File(location + trashFile).createNewFile(); + + DeleteOrphanFiles.Result results = + SparkActions.get() + .deleteOrphanFiles(table.table()) + .olderThan(System.currentTimeMillis() + 1000) + .execute(); + assertThat(results.orphanFileLocations()) + .as("trash file should be removed") + .contains("file:" + location + trashFile); + } + + @TestTemplate + public void testSparkCatalogNamedHadoopTable() throws Exception { + spark.conf().set("spark.sql.catalog.hadoop", "org.apache.iceberg.spark.SparkCatalog"); + spark.conf().set("spark.sql.catalog.hadoop.type", "hadoop"); + spark.conf().set("spark.sql.catalog.hadoop.warehouse", tableLocation); + SparkCatalog cat = (SparkCatalog) spark.sessionState().catalogManager().catalog("hadoop"); + + String[] database = {"default"}; + Identifier id = Identifier.of(database, randomName("table")); + Transform[] transforms = {}; + cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); + SparkTable table = (SparkTable) cat.loadTable(id); + + sql("INSERT INTO hadoop.default.%s VALUES (1,1,1)", id.name()); + + String location = table.table().location().replaceFirst("file:", ""); + String trashFile = randomName("/data/trashfile"); + new File(location + trashFile).createNewFile(); + + DeleteOrphanFiles.Result results = + SparkActions.get() + .deleteOrphanFiles(table.table()) + .olderThan(System.currentTimeMillis() + 1000) + .execute(); + assertThat(results.orphanFileLocations()) + .as("trash file should be removed") + .contains("file:" + location + trashFile); + } + + @TestTemplate + public void testSparkCatalogNamedHiveTable() throws Exception { + spark.conf().set("spark.sql.catalog.hive", "org.apache.iceberg.spark.SparkCatalog"); + spark.conf().set("spark.sql.catalog.hive.type", "hadoop"); + spark.conf().set("spark.sql.catalog.hive.warehouse", tableLocation); + SparkCatalog cat = (SparkCatalog) spark.sessionState().catalogManager().catalog("hive"); + + String[] database = {"default"}; + Identifier id = Identifier.of(database, randomName("table")); + Transform[] transforms = {}; + cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); + SparkTable table = (SparkTable) cat.loadTable(id); + + sql("INSERT INTO hive.default.%s VALUES (1,1,1)", id.name()); + + String location = table.table().location().replaceFirst("file:", ""); + String trashFile = randomName("/data/trashfile"); + new File(location + trashFile).createNewFile(); + + DeleteOrphanFiles.Result results = + SparkActions.get() + .deleteOrphanFiles(table.table()) + .olderThan(System.currentTimeMillis() + 1000) + .execute(); + + assertThat(StreamSupport.stream(results.orphanFileLocations().spliterator(), false)) + .as("trash file should be removed") + .anyMatch(file -> file.contains("file:" + location + trashFile)); + } + + @TestTemplate + public void testSparkSessionCatalogHadoopTable() throws Exception { + spark + .conf() + .set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog"); + spark.conf().set("spark.sql.catalog.spark_catalog.type", "hadoop"); + spark.conf().set("spark.sql.catalog.spark_catalog.warehouse", tableLocation); + SparkSessionCatalog cat = + (SparkSessionCatalog) spark.sessionState().catalogManager().v2SessionCatalog(); + + String[] database = {"default"}; + Identifier id = Identifier.of(database, randomName("table")); + Transform[] transforms = {}; + cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); + SparkTable table = (SparkTable) cat.loadTable(id); + + sql("INSERT INTO default.%s VALUES (1,1,1)", id.name()); + + String location = table.table().location().replaceFirst("file:", ""); + String trashFile = randomName("/data/trashfile"); + new File(location + trashFile).createNewFile(); + + DeleteOrphanFiles.Result results = + SparkActions.get() + .deleteOrphanFiles(table.table()) + .olderThan(System.currentTimeMillis() + 1000) + .execute(); + assertThat(results.orphanFileLocations()) + .as("trash file should be removed") + .contains("file:" + location + trashFile); + } + + @TestTemplate + public void testSparkSessionCatalogHiveTable() throws Exception { + spark + .conf() + .set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.SparkSessionCatalog"); + spark.conf().set("spark.sql.catalog.spark_catalog.type", "hive"); + SparkSessionCatalog cat = + (SparkSessionCatalog) spark.sessionState().catalogManager().v2SessionCatalog(); + + String[] database = {"default"}; + Identifier id = Identifier.of(database, "sessioncattest"); + Transform[] transforms = {}; + cat.dropTable(id); + cat.createTable(id, SparkSchemaUtil.convert(SCHEMA), transforms, properties); + SparkTable table = (SparkTable) cat.loadTable(id); + + sql("INSERT INTO default.sessioncattest VALUES (1,1,1)"); + + String location = table.table().location().replaceFirst("file:", ""); + String trashFile = randomName("/data/trashfile"); + new File(location + trashFile).createNewFile(); + + DeleteOrphanFiles.Result results = + SparkActions.get() + .deleteOrphanFiles(table.table()) + .olderThan(System.currentTimeMillis() + 1000) + .execute(); + assertThat(results.orphanFileLocations()) + .as("trash file should be removed") + .contains("file:" + location + trashFile); + } + + @AfterEach + public void resetSparkSessionCatalog() { + spark.conf().unset("spark.sql.catalog.spark_catalog"); + spark.conf().unset("spark.sql.catalog.spark_catalog.type"); + spark.conf().unset("spark.sql.catalog.spark_catalog.warehouse"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java new file mode 100644 index 000000000000..6d965f3dcc62 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -0,0 +1,2660 @@ +/* + * 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.actions; + +import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; +import static org.apache.iceberg.data.FileHelpers.encrypt; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.current_date; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.min; +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 static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.argThat; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doCallRealMethod; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.LongStream; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestReader; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.BinPackRewriteFilePlanner; +import org.apache.iceberg.actions.RewriteDataFiles; +import org.apache.iceberg.actions.RewriteDataFiles.Result; +import org.apache.iceberg.actions.RewriteDataFilesCommitManager; +import org.apache.iceberg.actions.RewriteFileGroup; +import org.apache.iceberg.actions.SizeBasedFileRewritePlanner; +import org.apache.iceberg.data.GenericFileWriterFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.BaseDVFileWriter; +import org.apache.iceberg.deletes.DVFileWriter; +import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.OutputFileFactory; +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.Sets; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.spark.FileRewriteCoordinator; +import org.apache.iceberg.spark.ScanTaskSetManager; +import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.spark.source.ThreeColumnRecord; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.DataTypes; +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; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.ArgumentMatcher; +import org.mockito.Mockito; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewriteDataFilesAction extends TestBase { + + @TempDir private File tableDir; + private static final int SCALE = 400000; + + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + private static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + + @Parameter private int formatVersion; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return org.apache.iceberg.TestHelpers.V2_AND_ABOVE; + } + + private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); + private final ScanTaskSetManager manager = ScanTaskSetManager.get(); + private String tableLocation = null; + + @BeforeAll + public static void setupSpark() { + // disable AQE as tests assume that writes generate a particular number of files + spark.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), "false"); + } + + @BeforeEach + public void setupTableLocation() { + this.tableLocation = tableDir.toURI().toString(); + } + + private RewriteDataFilesSparkAction basicRewrite(Table table) { + // Always compact regardless of input files + table.refresh(); + return actions() + .rewriteDataFiles(table) + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1"); + } + + @TestTemplate + public void testEmptyTable() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + assertThat(table.currentSnapshot()).as("Table must be empty").isNull(); + + basicRewrite(table).execute(); + + assertThat(table.currentSnapshot()).as("Table must stay empty").isNull(); + } + + @TestTemplate + public void testBinPackUnpartitionedTable() { + Table table = createTable(4); + shouldHaveFiles(table, 4); + List expectedRecords = currentData(); + long dataSizeBefore = testDataSize(table); + + Result result = basicRewrite(table).execute(); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 4 data files") + .isEqualTo(4); + assertThat(result.addedDataFilesCount()).as("Action should add 1 data file").isOne(); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + shouldHaveFiles(table, 1); + List actual = currentData(); + + assertEquals("Rows must match", expectedRecords, actual); + } + + @TestTemplate + public void testBinPackPartitionedTable() { + Table table = createTablePartitioned(4, 2); + shouldHaveFiles(table, 8); + List expectedRecords = currentData(); + long dataSizeBefore = testDataSize(table); + + Result result = basicRewrite(table).execute(); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 8 data files") + .isEqualTo(8); + assertThat(result.addedDataFilesCount()).as("Action should add 4 data file").isEqualTo(4); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + shouldHaveFiles(table, 4); + List actualRecords = currentData(); + + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + @TestTemplate + public void testBinPackWithFilter() { + Table table = createTablePartitioned(4, 2); + shouldHaveFiles(table, 8); + List expectedRecords = currentData(); + long dataSizeBefore = testDataSize(table); + + Result result = + basicRewrite(table) + .filter(Expressions.equal("c1", 1)) + .filter(Expressions.startsWith("c2", "foo")) + .execute(); + + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 2 data files") + .isEqualTo(2); + assertThat(result.addedDataFilesCount()).as("Action should add 1 data file").isOne(); + assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); + + shouldHaveFiles(table, 7); + + List actualRecords = currentData(); + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + @TestTemplate + public void testBinPackWithFilterOnBucketExpression() { + Table table = createTablePartitioned(4, 2); + + shouldHaveFiles(table, 8); + List expectedRecords = currentData(); + long dataSizeBefore = testDataSize(table); + + Result result = + basicRewrite(table) + .filter(Expressions.equal("c1", 1)) + .filter(Expressions.equal(Expressions.bucket("c2", 2), 0)) + .execute(); + + assertThat(result) + .extracting(Result::rewrittenDataFilesCount, Result::addedDataFilesCount) + .as("Action should rewrite 2 data files into 1 data file") + .contains(2, 1); + assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); + + shouldHaveFiles(table, 7); + + List actualRecords = currentData(); + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + @TestTemplate + public void testBinPackAfterPartitionChange() { + Table table = createTable(); + + writeRecords(20, SCALE, 20); + shouldHaveFiles(table, 20); + table.updateSpec().addField(Expressions.ref("c1")).commit(); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + RewriteDataFiles.Result result = + basicRewrite(table) + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") + .option( + SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, + Integer.toString(averageFileSize(table) + 1000)) + .option( + RewriteDataFiles.TARGET_FILE_SIZE_BYTES, + // Increase max file size for V3 to account for additional row lineage fields + Integer.toString(averageFileSize(table) + (formatVersion >= 3 ? 12000 : 1100))) + .execute(); + + assertThat(result.rewriteResults()) + .as("Should have 1 fileGroup because all files were not correctly partitioned") + .hasSize(1); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + shouldHaveFiles(table, 20); + } + + @TestTemplate + public void testDataFilesRewrittenWithMaxDeleteRatio() throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + Table table = createTable(); + int numDataFiles = 5; + // 100 / 5 = 20 records per data file + writeRecords(numDataFiles, 100); + // delete > 100% of records for each data file + int numPositionsToDelete = 1000; + table.refresh(); + List dataFiles = TestHelpers.dataFiles(table); + assertThat(dataFiles).hasSize(numDataFiles); + + RowDelta rowDelta = table.newRowDelta(); + for (DataFile dataFile : dataFiles) { + if (formatVersion >= 3) { + writeDV(table, dataFile.partition(), dataFile.location(), numPositionsToDelete) + .forEach(rowDelta::addDeletes); + } else { + writePosDeletes(table, dataFile.partition(), dataFile.location(), 4, numPositionsToDelete) + .forEach(rowDelta::addDeletes); + } + } + + rowDelta.commit(); + + Set deleteFiles = TestHelpers.deleteFiles(table); + int expectedDataFiles = formatVersion >= 3 ? numDataFiles : numDataFiles * 4; + assertThat(deleteFiles).hasSize(expectedDataFiles); + + // there are 5 data files with a delete ratio of > 100% each, so all data files should be + // rewritten. Set MIN_INPUT_FILES > to the number of data files so that compaction is only + // triggered when the delete ratio of >= 30% is hit + RewriteDataFiles.Result result = + SparkActions.get(spark) + .rewriteDataFiles(table) + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "10") + .option(SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, "0") + .execute(); + + assertThat(result.rewrittenDataFilesCount()).isEqualTo(numDataFiles); + + table.refresh(); + List newDataFiles = TestHelpers.dataFiles(table); + assertThat(newDataFiles).isEmpty(); + + Set newDeleteFiles = TestHelpers.deleteFiles(table); + assertThat(newDeleteFiles).isEmpty(); + } + + @TestTemplate + public void testDataFilesRewrittenWithHighDeleteRatio() throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + Table table = createTable(); + int numDataFiles = 5; + // 100 / 5 = 20 records per data file + writeRecords(numDataFiles, 100); + // delete 40% of records for each data file + int numPositionsToDelete = 8; + table.refresh(); + List dataFiles = TestHelpers.dataFiles(table); + assertThat(dataFiles).hasSize(numDataFiles); + + RowDelta rowDelta = table.newRowDelta(); + for (DataFile dataFile : dataFiles) { + if (formatVersion >= 3) { + writeDV(table, dataFile.partition(), dataFile.location(), numPositionsToDelete) + .forEach(rowDelta::addDeletes); + } else { + writePosDeletes(table, dataFile.partition(), dataFile.location(), 4, numPositionsToDelete) + .forEach(rowDelta::addDeletes); + } + } + + rowDelta.commit(); + + Set deleteFiles = TestHelpers.deleteFiles(table); + int expectedDataFiles = formatVersion >= 3 ? numDataFiles : numDataFiles * 4; + assertThat(deleteFiles).hasSize(expectedDataFiles); + + // there are 5 data files with a delete ratio of 40% each, so all data files should be + // rewritten. Set MIN_INPUT_FILES > to the number of data files so that compaction is only + // triggered when the delete ratio of >= 30% is hit + RewriteDataFiles.Result result = + SparkActions.get(spark) + .rewriteDataFiles(table) + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "10") + .option(SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, "0") + .execute(); + + assertThat(result.rewrittenDataFilesCount()).isEqualTo(numDataFiles); + + table.refresh(); + List newDataFiles = TestHelpers.dataFiles(table); + assertThat(newDataFiles).hasSize(1); + + Set newDeleteFiles = TestHelpers.deleteFiles(table); + assertThat(newDeleteFiles).isEmpty(); + } + + @TestTemplate + public void testDataFilesNotRewrittenWithLowDeleteRatio() throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + Table table = createTable(); + int numDataFiles = 5; + // 100 / 5 = 20 records per data file + writeRecords(numDataFiles, 100); + // delete 25% of records for each data file + int numPositionsToDelete = 5; + table.refresh(); + List dataFiles = TestHelpers.dataFiles(table); + assertThat(dataFiles).hasSize(numDataFiles); + + RowDelta rowDelta = table.newRowDelta(); + for (DataFile dataFile : dataFiles) { + if (formatVersion >= 3) { + writeDV(table, dataFile.partition(), dataFile.location(), numPositionsToDelete) + .forEach(rowDelta::addDeletes); + } else { + writePosDeletes(table, dataFile.partition(), dataFile.location(), 5, numPositionsToDelete) + .forEach(rowDelta::addDeletes); + } + } + + rowDelta.commit(); + + Set deleteFiles = TestHelpers.deleteFiles(table); + int expectedDataFiles = formatVersion >= 3 ? numDataFiles : numDataFiles * 5; + assertThat(deleteFiles).hasSize(expectedDataFiles); + + // there are 5 data files with a delete ratio of 25% each, so data files should not be + // rewritten. Set MIN_INPUT_FILES > to the number of data files so that compaction is only + // triggered when the delete ratio of >= 30% is hit + RewriteDataFiles.Result result = + SparkActions.get(spark) + .rewriteDataFiles(table) + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "10") + .option(SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, "0") + .execute(); + + assertThat(result.rewrittenDataFilesCount()).isEqualTo(0); + + table.refresh(); + List newDataFiles = TestHelpers.dataFiles(table); + assertThat(newDataFiles).hasSameSizeAs(dataFiles); + + Set newDeleteFiles = TestHelpers.deleteFiles(table); + assertThat(newDeleteFiles).hasSameSizeAs(deleteFiles); + } + + @TestTemplate + public void testBinPackWithV2PositionDeletes() throws IOException { + assumeThat(formatVersion).isEqualTo(2); + Table table = createTablePartitioned(4, 2); + shouldHaveFiles(table, 8); + table.refresh(); + + List dataFiles = TestHelpers.dataFiles(table); + int total = (int) dataFiles.stream().mapToLong(ContentFile::recordCount).sum(); + + RowDelta rowDelta = table.newRowDelta(); + // add 1 delete file for data files 0, 1, 2 + for (int i = 0; i < 3; i++) { + writePosDeletesToFile(table, dataFiles.get(i), 1).forEach(rowDelta::addDeletes); + } + + // add 2 delete files for data files 3, 4 + for (int i = 3; i < 5; i++) { + writePosDeletesToFile(table, dataFiles.get(i), 2).forEach(rowDelta::addDeletes); + } + + rowDelta.commit(); + table.refresh(); + List expectedRecords = currentData(); + long dataSizeBefore = testDataSize(table); + Result result = + actions() + .rewriteDataFiles(table) + // do not include any file based on bin pack file size configs + .option(BinPackRewriteFilePlanner.MIN_FILE_SIZE_BYTES, "0") + .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) + .option(BinPackRewriteFilePlanner.MAX_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE)) + .option(BinPackRewriteFilePlanner.DELETE_FILE_THRESHOLD, "2") + .execute(); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 2 data files") + .isEqualTo(2); + assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); + + List actualRecords = currentData(); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertThat(actualRecords).as("7 rows are removed").hasSize(total - 7); + } + + @TestTemplate + public void testBinPackWithDVs() throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + Table table = createTablePartitioned(4, 2); + shouldHaveFiles(table, 8); + table.refresh(); + List initialRecords = currentDataWithLineage(); + Set rowIds = + initialRecords.stream().map(record -> (Long) record[0]).collect(Collectors.toSet()); + Set lastUpdatedSequenceNumbers = + initialRecords.stream().map(record -> (Long) record[1]).collect(Collectors.toSet()); + assertThat(rowIds) + .isEqualTo(LongStream.range(0, initialRecords.size()).boxed().collect(Collectors.toSet())); + assertThat(lastUpdatedSequenceNumbers).allMatch(sequenceNumber -> sequenceNumber.equals(1L)); + List dataFiles = TestHelpers.dataFiles(table); + int total = (int) dataFiles.stream().mapToLong(ContentFile::recordCount).sum(); + + RowDelta rowDelta = table.newRowDelta(); + Set rowIdsBeingRemoved = Sets.newHashSet(); + + // add 1 DV for data files 0, 1, 2 + for (int i = 0; i < 3; i++) { + writeDV(table, dataFiles.get(i).partition(), dataFiles.get(i).location(), 1) + .forEach(rowDelta::addDeletes); + rowIdsBeingRemoved.add(dataFiles.get(i).firstRowId()); + } + + // delete 2 positions for data files 3, 4 + for (int i = 3; i < 5; i++) { + writeDV(table, dataFiles.get(i).partition(), dataFiles.get(i).location(), 2) + .forEach(rowDelta::addDeletes); + long dataFileFirstRowId = dataFiles.get(i).firstRowId(); + rowIdsBeingRemoved.add(dataFileFirstRowId); + rowIdsBeingRemoved.add(dataFileFirstRowId + 1); + } + + rowDelta.commit(); + table.refresh(); + List recordsWithLineageAfterDelete = currentDataWithLineage(); + rowIds.removeAll(rowIdsBeingRemoved); + assertThat(rowIds) + .isEqualTo( + recordsWithLineageAfterDelete.stream() + .map(record -> (Long) record[0]) + .collect(Collectors.toSet())); + + long dataSizeBefore = testDataSize(table); + + Result result = + actions() + .rewriteDataFiles(table) + // do not include any file based on bin pack file size configs + .option(BinPackRewriteFilePlanner.MIN_FILE_SIZE_BYTES, "0") + .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) + .option(BinPackRewriteFilePlanner.MAX_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE)) + // set DELETE_FILE_THRESHOLD to 1 since DVs only produce one delete file per data file + .option(BinPackRewriteFilePlanner.DELETE_FILE_THRESHOLD, "1") + .execute(); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 5 data files") + .isEqualTo(5); + assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); + List actualRecordsWithLineage = currentDataWithLineage(); + assertEquals("Rows must match", recordsWithLineageAfterDelete, actualRecordsWithLineage); + assertThat(actualRecordsWithLineage).as("7 rows are removed").hasSize(total - 7); + } + + @TestTemplate + public void removeDanglingDVsFromDeleteManifest() throws Exception { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + Table table = createTable(); + int numDataFiles = 5; + // 100 / 5 = 20 records per data file + writeRecords(numDataFiles, 100); + int numPositionsToDelete = 10; + table.refresh(); + List dataFiles = TestHelpers.dataFiles(table); + assertThat(dataFiles).hasSize(numDataFiles); + + RowDelta rowDelta = table.newRowDelta(); + for (DataFile dataFile : dataFiles) { + writeDV(table, dataFile.partition(), dataFile.location(), numPositionsToDelete) + .forEach(rowDelta::addDeletes); + } + + rowDelta.commit(); + + Set deleteFiles = TestHelpers.deleteFiles(table); + assertThat(deleteFiles).hasSize(numDataFiles); + + Set validDataFilePaths = + TestHelpers.dataFiles(table).stream() + .map(ContentFile::location) + .collect(Collectors.toSet()); + for (ManifestFile manifestFile : table.currentSnapshot().deleteManifests(table.io())) { + ManifestReader reader = + ManifestFiles.readDeleteManifest( + manifestFile, table.io(), ((BaseTable) table).operations().current().specsById()); + for (DeleteFile deleteFile : reader) { + // make sure there are no orphaned DVs + assertThat(validDataFilePaths).contains(deleteFile.referencedDataFile()); + } + } + + // all data files should be rewritten. Set MIN_INPUT_FILES > to the number of data files so that + // compaction is only triggered when the delete ratio of >= 30% is hit + RewriteDataFiles.Result result = + SparkActions.get(spark) + .rewriteDataFiles(table) + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "10") + .option(SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, "0") + .execute(); + + assertThat(result.rewrittenDataFilesCount()).isEqualTo(numDataFiles); + assertThat(result.removedDeleteFilesCount()).isEqualTo(numDataFiles); + + table.refresh(); + assertThat(TestHelpers.dataFiles(table)).hasSize(1); + assertThat(TestHelpers.deleteFiles(table)).isEmpty(); + + validDataFilePaths = + TestHelpers.dataFiles(table).stream() + .map(ContentFile::location) + .collect(Collectors.toSet()); + for (ManifestFile manifestFile : table.currentSnapshot().deleteManifests(table.io())) { + ManifestReader reader = + ManifestFiles.readDeleteManifest( + manifestFile, table.io(), ((BaseTable) table).operations().current().specsById()); + for (DeleteFile deleteFile : reader) { + // make sure there are no orphaned DVs + assertThat(validDataFilePaths).contains(deleteFile.referencedDataFile()); + } + } + } + + @TestTemplate + public void testRemoveDangledEqualityDeletesPartitionEvolution() { + Table table = + TABLES.create( + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), + tableLocation); + + // data seq = 1, write 4 files in 2 partitions + List records1 = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), new ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")); + writeRecords(records1); + List records2 = + Lists.newArrayList( + new ThreeColumnRecord(0, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(0, "DDDDDDDDDD", "DDDD")); + writeRecords(records2); + table.refresh(); + shouldHaveFiles(table, 4); + + // data seq = 2 & 3, write 2 equality deletes in both partitions + writeEqDeleteRecord(table, "c1", 1, "c3", "AAAA"); + writeEqDeleteRecord(table, "c1", 2, "c3", "CCCC"); + table.refresh(); + Set existingDeletes = TestHelpers.deleteFiles(table); + assertThat(existingDeletes) + .as("Only one equality delete c1=1 is used in query planning") + .hasSize(1); + + // partition evolution + table.refresh(); + table.updateSpec().addField(Expressions.ref("c3")).commit(); + + // data seq = 4, write 2 new data files in both partitions for evolved spec + List records3 = + Lists.newArrayList( + new ThreeColumnRecord(1, "A", "CCCC"), new ThreeColumnRecord(2, "D", "DDDD")); + writeRecords(records3); + + List originalData = currentData(); + + RewriteDataFiles.Result result = + basicRewrite(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .filter(Expressions.equal("c1", 1)) + .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") + .execute(); + + existingDeletes = TestHelpers.deleteFiles(table); + assertThat(existingDeletes).as("Shall pruned dangling deletes after rewrite").hasSize(0); + + assertThat(result) + .extracting( + Result::addedDataFilesCount, + Result::rewrittenDataFilesCount, + Result::removedDeleteFilesCount) + .as("Should compact 3 data files into 2 and remove both dangled equality delete file") + .containsExactly(2, 3, 2); + shouldHaveMinSequenceNumberInPartition(table, "data_file.partition.c1 == 1", 5); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 7); + shouldHaveFiles(table, 5); + } + + @TestTemplate + public void testRemoveDangledPositionDeletesPartitionEvolution() throws IOException { + Table table = + TABLES.create( + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), + tableLocation); + + // data seq = 1, write 4 files in 2 partitions + writeRecords(2, 2, 2); + List dataFilesBefore = TestHelpers.dataFiles(table, null); + shouldHaveFiles(table, 4); + + DeleteFile deleteFile; + // data seq = 2, write 1 position deletes in c1=1 + DataFile dataFile = dataFilesBefore.get(3); + if (formatVersion >= 3) { + deleteFile = writeDV(table, dataFile.partition(), dataFile.location(), 1).get(0); + } else { + deleteFile = writePosDeletesToFile(table, dataFile, 1).get(0); + } + table.newRowDelta().addDeletes(deleteFile).commit(); + + // partition evolution + table.updateSpec().addField(Expressions.ref("c3")).commit(); + + // data seq = 3, write 1 new data files in c1=1 for evolved spec + writeRecords(1, 1, 1); + shouldHaveFiles(table, 5); + List expectedRecords = currentData(); + + Result result = + actions() + .rewriteDataFiles(table) + .filter(Expressions.equal("c1", 1)) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") + .execute(); + + assertThat(result) + .extracting( + Result::addedDataFilesCount, + Result::rewrittenDataFilesCount, + Result::removedDeleteFilesCount) + .as("Should rewrite 2 data files into 1 and remove 1 dangled position delete file") + .containsExactly(1, 2, 1); + shouldHaveMinSequenceNumberInPartition(table, "data_file.partition.c1 == 1", 3); + + // v3 removes orphaned DVs during rewrite already, so there should be one snapshot less + int expectedSnapshots = formatVersion >= 3 ? 4 : 5; + shouldHaveSnapshots(table, expectedSnapshots); + assertThat(table.currentSnapshot().summary()).containsEntry("total-position-deletes", "0"); + assertEquals("Rows must match", expectedRecords, currentData()); + } + + @TestTemplate + public void testBinPackWithDeleteAllData() throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + Table table = createTablePartitioned(1, 1, 1); + shouldHaveFiles(table, 1); + table.refresh(); + + List dataFiles = TestHelpers.dataFiles(table); + int total = (int) dataFiles.stream().mapToLong(ContentFile::recordCount).sum(); + + RowDelta rowDelta = table.newRowDelta(); + DataFile dataFile = dataFiles.get(0); + // remove all data + if (formatVersion >= 3) { + writeDV(table, dataFile.partition(), dataFile.location(), total) + .forEach(rowDelta::addDeletes); + } else { + writePosDeletesToFile(table, dataFile, total).forEach(rowDelta::addDeletes); + } + + rowDelta.commit(); + table.refresh(); + List expectedRecords = currentData(); + long dataSizeBefore = testDataSize(table); + + Result result = + actions() + .rewriteDataFiles(table) + .option(BinPackRewriteFilePlanner.DELETE_FILE_THRESHOLD, "1") + .execute(); + assertThat(result.rewrittenDataFilesCount()).as("Action should rewrite 1 data files").isOne(); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + if (formatVersion >= 3) { + assertThat(result.removedDeleteFilesCount()).isEqualTo(dataFiles.size()); + } + + List actualRecords = currentData(); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertThat(table.currentSnapshot().dataManifests(table.io()).get(0).existingFilesCount()) + .as("Data manifest should not have existing data file") + .isZero(); + + assertThat((long) table.currentSnapshot().dataManifests(table.io()).get(0).deletedFilesCount()) + .as("Data manifest should have 1 delete data file") + .isEqualTo(1L); + + assertThat(table.currentSnapshot().deleteManifests(table.io()).get(0).addedRowsCount()) + .as("Delete manifest added row count should equal total count") + // v3 removes orphaned DVs during rewrite + .isEqualTo(formatVersion >= 3 ? 0 : 1); + } + + @TestTemplate + public void testBinPackWithStartingSequenceNumber() { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + Table table = createTablePartitioned(4, 2); + shouldHaveFiles(table, 8); + List expectedRecords = currentData(); + table.refresh(); + long oldSequenceNumber = table.currentSnapshot().sequenceNumber(); + long dataSizeBefore = testDataSize(table); + + Result result = + basicRewrite(table).option(RewriteDataFiles.USE_STARTING_SEQUENCE_NUMBER, "true").execute(); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 8 data files") + .isEqualTo(8); + assertThat(result.addedDataFilesCount()).as("Action should add 4 data files").isEqualTo(4); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + shouldHaveFiles(table, 4); + List actualRecords = currentData(); + assertEquals("Rows must match", expectedRecords, actualRecords); + + table.refresh(); + assertThat(table.currentSnapshot().sequenceNumber()) + .as("Table sequence number should be incremented") + .isGreaterThan(oldSequenceNumber); + + Dataset rows = SparkTableUtil.loadMetadataTable(spark, table, MetadataTableType.ENTRIES); + for (Row row : rows.collectAsList()) { + if (row.getInt(0) == 1) { + assertThat(row.getLong(2)) + .as("Expect old sequence number for added entries") + .isEqualTo(oldSequenceNumber); + } + } + } + + @TestTemplate + public void testBinPackWithStartingSequenceNumberV1Compatibility() { + Map properties = ImmutableMap.of(TableProperties.FORMAT_VERSION, "1"); + Table table = createTablePartitioned(4, 2, SCALE, properties); + shouldHaveFiles(table, 8); + List expectedRecords = currentData(); + table.refresh(); + long oldSequenceNumber = table.currentSnapshot().sequenceNumber(); + assertThat(oldSequenceNumber).as("Table sequence number should be 0").isZero(); + long dataSizeBefore = testDataSize(table); + + Result result = + basicRewrite(table).option(RewriteDataFiles.USE_STARTING_SEQUENCE_NUMBER, "true").execute(); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 8 data files") + .isEqualTo(8); + assertThat(result.addedDataFilesCount()).as("Action should add 4 data files").isEqualTo(4); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + shouldHaveFiles(table, 4); + List actualRecords = currentData(); + assertEquals("Rows must match", expectedRecords, actualRecords); + + table.refresh(); + assertThat(table.currentSnapshot().sequenceNumber()) + .as("Table sequence number should still be 0") + .isEqualTo(oldSequenceNumber); + + Dataset rows = SparkTableUtil.loadMetadataTable(spark, table, MetadataTableType.ENTRIES); + for (Row row : rows.collectAsList()) { + assertThat(row.getLong(2)) + .as("Expect sequence number 0 for all entries") + .isEqualTo(oldSequenceNumber); + } + } + + @TestTemplate + public void testRewriteLargeTableHasResiduals() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).build(); + Map options = + ImmutableMap.of( + TableProperties.FORMAT_VERSION, + String.valueOf(formatVersion), + TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, + "100"); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + // all records belong to the same partition + List records = Lists.newArrayList(); + for (int i = 0; i < 100; i++) { + records.add(new ThreeColumnRecord(i, String.valueOf(i), String.valueOf(i % 4))); + } + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); + writeDF(df); + + List expectedRecords = currentData(); + + table.refresh(); + + CloseableIterable tasks = + table.newScan().ignoreResiduals().filter(Expressions.equal("c3", "0")).planFiles(); + for (FileScanTask task : tasks) { + assertThat(task.residual()) + .as("Residuals must be ignored") + .isEqualTo(Expressions.alwaysTrue()); + } + + shouldHaveFiles(table, 2); + + long dataSizeBefore = testDataSize(table); + Result result = basicRewrite(table).filter(Expressions.equal("c3", "0")).execute(); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 2 data files") + .isEqualTo(2); + assertThat(result.addedDataFilesCount()).as("Action should add 1 data file").isOne(); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + List actualRecords = currentData(); + + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + @TestTemplate + public void testBinPackSplitLargeFile() { + Table table = createTable(1); + shouldHaveFiles(table, 1); + + List expectedRecords = currentData(); + long targetSize = testDataSize(table) / 2; + + long dataSizeBefore = testDataSize(table); + Result result = + basicRewrite(table) + .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Long.toString(targetSize)) + .option( + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, + Long.toString(targetSize * 2 - 2000)) + .execute(); + + assertThat(result.rewrittenDataFilesCount()).as("Action should delete 1 data files").isOne(); + assertThat(result.addedDataFilesCount()).as("Action should add 2 data files").isEqualTo(2); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + shouldHaveFiles(table, 2); + + List actualRecords = currentData(); + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + @TestTemplate + public void testBinPackCombineMixedFiles() { + Table table = createTable(1); // 400000 + shouldHaveFiles(table, 1); + + // Add one more small file, and one large file + writeRecords(1, SCALE); + writeRecords(1, SCALE * 3); + shouldHaveFiles(table, 3); + + List expectedRecords = currentData(); + + int targetSize = averageFileSize(table); + + long dataSizeBefore = testDataSize(table); + Result result = + basicRewrite(table) + .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(targetSize + 1000)) + .option( + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, + // Increase max file size for V3 to account for additional row lineage fields + Integer.toString(targetSize + (formatVersion >= 3 ? 1850000 : 80000))) + .option( + SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, + Integer.toString(targetSize - 1000)) + .execute(); + + assertThat(result.rewrittenDataFilesCount()) + .as("Action should delete 3 data files") + .isEqualTo(3); + // Should Split the big files into 3 pieces, one of which should be combined with the two + // smaller files + assertThat(result.addedDataFilesCount()).as("Action should add 3 data files").isEqualTo(3); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + shouldHaveFiles(table, 3); + + List actualRecords = currentData(); + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + @TestTemplate + public void testBinPackCombineMediumFiles() { + Table table = createTable(4); + shouldHaveFiles(table, 4); + + List expectedRecords = currentData(); + int targetSize = ((int) testDataSize(table) / 3); + // The test is to see if we can combine parts of files to make files of the correct size + + long dataSizeBefore = testDataSize(table); + Result result = + basicRewrite(table) + .option(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(targetSize)) + .option( + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, + // Increase max file size for V3 to account for additional row lineage fields + Integer.toString((int) (targetSize * (formatVersion >= 3 ? 2 : 1.8)))) + .option( + SizeBasedFileRewritePlanner.MIN_FILE_SIZE_BYTES, + Integer.toString(targetSize - 100)) // All files too small + .execute(); + + assertThat(result.rewrittenDataFilesCount()) + .as("Action should delete 4 data files") + .isEqualTo(4); + assertThat(result.addedDataFilesCount()).as("Action should add 3 data files").isEqualTo(3); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + shouldHaveFiles(table, 3); + + List actualRecords = currentData(); + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + @TestTemplate + public void testPartialProgressEnabled() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + table.updateProperties().set(COMMIT_NUM_RETRIES, "10").commit(); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + // Perform a rewrite but only allow 2 files to be compacted at a time + RewriteDataFiles.Result result = + basicRewrite(table) + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "10") + .execute(); + + assertThat(result.rewriteResults()).as("Should have 10 fileGroups").hasSize(10); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + table.refresh(); + + shouldHaveSnapshots(table, 11); + shouldHaveACleanCache(table); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + } + + @TestTemplate + public void testMultipleGroups() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + // Perform a rewrite but only allow 2 files to be compacted at a time + RewriteDataFiles.Result result = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") + .execute(); + + assertThat(result.rewriteResults()).as("Should have 10 fileGroups").hasSize(10); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + } + + @TestTemplate + public void testPartialProgressMaxCommits() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + // Perform a rewrite but only allow 2 files to be compacted at a time + RewriteDataFiles.Result result = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3") + .execute(); + + assertThat(result.rewriteResults()).as("Should have 10 fileGroups").hasSize(10); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 4); + shouldHaveACleanCache(table); + } + + @TestTemplate + public void testSingleCommitWithRewriteFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + RewriteDataFilesSparkAction realRewrite = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)); + + RewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite); + + // Fail groups 1, 3, and 7 during rewrite + GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); + doThrow(new RuntimeException("Rewrite Failed")) + .when(spyRewrite) + .rewriteFiles(any(), argThat(failGroup)); + + assertThatThrownBy(spyRewrite::execute) + .isInstanceOf(RuntimeException.class) + .hasMessage("Rewrite Failed"); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 1); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @TestTemplate + public void testSingleCommitWithCommitFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + RewriteDataFilesSparkAction realRewrite = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)); + + RewriteDataFilesSparkAction spyRewrite = spy(realRewrite); + RewriteDataFilesCommitManager util = spy(new RewriteDataFilesCommitManager(table)); + + // Fail to commit + doThrow(new CommitFailedException("Commit Failure")).when(util).commitFileGroups(any()); + + doReturn(util).when(spyRewrite).commitManager(table.currentSnapshot().snapshotId()); + + assertThatThrownBy(spyRewrite::execute) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Cannot commit rewrite"); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 1); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @TestTemplate + public void testCommitFailsWithUncleanableFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + RewriteDataFilesSparkAction realRewrite = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)); + + RewriteDataFilesSparkAction spyRewrite = spy(realRewrite); + RewriteDataFilesCommitManager util = spy(new RewriteDataFilesCommitManager(table)); + + // Fail to commit with an arbitrary failure and validate that orphans are not cleaned up + doThrow(new RuntimeException("Arbitrary Failure")).when(util).commitFileGroups(any()); + + doReturn(util).when(spyRewrite).commitManager(table.currentSnapshot().snapshotId()); + + assertThatThrownBy(spyRewrite::execute) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Arbitrary Failure"); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 1); + shouldHaveOrphans(table); + shouldHaveACleanCache(table); + } + + @TestTemplate + public void testParallelSingleCommitWithRewriteFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + RewriteDataFilesSparkAction realRewrite = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) + .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3"); + + RewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite); + + // Fail groups 1, 3, and 7 during rewrite + GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); + doThrow(new CommitFailedException("Rewrite Failed")) + .when(spyRewrite) + .rewriteFiles(any(), argThat(failGroup)); + + assertThatThrownBy(spyRewrite::execute) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Rewrite Failed"); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 1); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @TestTemplate + public void testPartialProgressWithRewriteFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + RewriteDataFilesSparkAction realRewrite = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3"); + + RewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite); + + // Fail groups 1, 3, and 7 during rewrite + GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); + doThrow(new RuntimeException("Rewrite Failed")) + .when(spyRewrite) + .rewriteFiles(any(), argThat(failGroup)); + + RewriteDataFiles.Result result = spyRewrite.execute(); + + assertThat(result.rewriteResults()).hasSize(7); + assertThat(result.rewriteFailures()).hasSize(3); + assertThat(result.failedDataFilesCount()).isEqualTo(6); + assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2. + // removing 3 groups leaves us with only 2 new commits, 4 and 3 + shouldHaveSnapshots(table, 3); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @TestTemplate + public void testParallelPartialProgressWithRewriteFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + RewriteDataFilesSparkAction realRewrite = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) + .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3") + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3"); + + RewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite); + + // Fail groups 1, 3, and 7 during rewrite + GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); + doThrow(new RuntimeException("Rewrite Failed")) + .when(spyRewrite) + .rewriteFiles(any(), argThat(failGroup)); + + RewriteDataFiles.Result result = spyRewrite.execute(); + + assertThat(result.rewriteResults()).hasSize(7); + assertThat(result.rewriteFailures()).hasSize(3); + assertThat(result.failedDataFilesCount()).isEqualTo(6); + assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + // With 10 original groups and max commits of 3, we have 4 groups per commit. + // Removing 3 groups, we are left with 4 groups and 3 groups in two commits. + shouldHaveSnapshots(table, 3); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @TestTemplate + public void testParallelPartialProgressWithCommitFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + RewriteDataFilesSparkAction realRewrite = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) + .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3") + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3"); + + RewriteDataFilesSparkAction spyRewrite = spy(realRewrite); + RewriteDataFilesCommitManager util = spy(new RewriteDataFilesCommitManager(table)); + + // First and Third commits work, second does not + doCallRealMethod() + .doThrow(new CommitFailedException("Commit Failed")) + .doCallRealMethod() + .when(util) + .commitFileGroups(any()); + + doReturn(util).when(spyRewrite).commitManager(table.currentSnapshot().snapshotId()); + + RewriteDataFiles.Result result = spyRewrite.execute(); + + // Commit 1: 4/4 + Commit 2 failed 0/4 + Commit 3: 2/2 == 6 out of 10 total groups committed + assertThat(result.rewriteResults()).as("Should have 6 fileGroups").hasSize(6); + assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + // Only 2 new commits because we broke one + shouldHaveSnapshots(table, 3); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @TestTemplate + public void testParallelPartialProgressWithMaxFailedCommits() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + RewriteDataFilesSparkAction realRewrite = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) + .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3") + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_FAILED_COMMITS, "0"); + + RewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite); + + // Fail groups 1, 3, and 7 during rewrite + GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); + doThrow(new RuntimeException("Rewrite Failed")) + .when(spyRewrite) + .rewriteFiles(any(), argThat(failGroup)); + + assertThatThrownBy(() -> spyRewrite.execute()) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "1 rewrite commits failed. This is more than the maximum allowed failures of 0"); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + // With 10 original groups and max commits of 3, we have 4 groups per commit. + // Removing 3 groups, we are left with 4 groups and 3 groups in two commits. + // Adding max allowed failed commits doesn't change the number of successful commits. + shouldHaveSnapshots(table, 3); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @TestTemplate + public void testParallelPartialProgressWithMaxCommitsLargerThanTotalGroupCount() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + RewriteDataFilesSparkAction rewrite = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) + .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3") + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + // Since we can have at most one commit per file group and there are only 10 file + // groups, actual number of commits is 10 + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "20") + // Setting max-failed-commits to 1 to tolerate random commit failure + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_FAILED_COMMITS, "1"); + rewrite.execute(); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + table.refresh(); + assertThat(table.snapshots()) + .as("Table did not have the expected number of snapshots") + // To tolerate 1 random commit failure + .hasSizeGreaterThanOrEqualTo(10); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + + @TestTemplate + public void testInvalidOptions() { + Table table = createTable(20); + + assertThatThrownBy( + () -> + basicRewrite(table) + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "-5") + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot set partial-progress.max-commits to -5, " + + "the value must be positive when partial-progress.enabled is true"); + + assertThatThrownBy( + () -> + basicRewrite(table) + .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "-5") + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot set max-concurrent-file-group-rewrites to -5, the value must be positive."); + + assertThatThrownBy(() -> basicRewrite(table).option("foobarity", "-5").execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot use options [foobarity], they are not supported by the action or the rewriter BIN-PACK"); + + assertThatThrownBy( + () -> basicRewrite(table).option(RewriteDataFiles.REWRITE_JOB_ORDER, "foo").execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid rewrite job order name: foo"); + + assertThatThrownBy( + () -> + basicRewrite(table) + .sort(SortOrder.builderFor(table.schema()).asc("c2").build()) + .option(SparkShufflingFileRewriteRunner.SHUFFLE_PARTITIONS_PER_FILE, "5") + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("requires enabling Iceberg Spark session extensions"); + } + + @TestTemplate + public void testSortMultipleGroups() { + Table table = createTable(20); + shouldHaveFiles(table, 20); + table.replaceSortOrder().asc("c2").commit(); + shouldHaveLastCommitUnsorted(table, "c2"); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + // Perform a rewrite but only allow 2 files to be compacted at a time + RewriteDataFiles.Result result = + basicRewrite(table) + .sort() + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) + .execute(); + + assertThat(result.rewriteResults()).as("Should have 10 fileGroups").hasSize(10); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + } + + @TestTemplate + public void testSimpleSort() { + Table table = createTable(20); + shouldHaveFiles(table, 20); + table.replaceSortOrder().asc("c2").commit(); + shouldHaveLastCommitUnsorted(table, "c2"); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + RewriteDataFiles.Result result = + basicRewrite(table) + .sort() + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(averageFileSize(table))) + .execute(); + + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + shouldHaveMultipleFiles(table); + shouldHaveLastCommitSorted(table, "c2"); + } + + @TestTemplate + public void testSortAfterPartitionChange() { + Table table = createTable(20); + shouldHaveFiles(table, 20); + table.updateSpec().addField(Expressions.bucket("c1", 4)).commit(); + table.replaceSortOrder().asc("c2").commit(); + shouldHaveLastCommitUnsorted(table, "c2"); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + RewriteDataFiles.Result result = + basicRewrite(table) + .sort() + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(averageFileSize(table))) + .execute(); + + assertThat(result.rewriteResults()) + .as("Should have 1 fileGroups because all files were not correctly partitioned") + .hasSize(1); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + shouldHaveMultipleFiles(table); + shouldHaveLastCommitSorted(table, "c2"); + } + + @TestTemplate + public void testSortCustomSortOrder() { + Table table = createTable(20); + shouldHaveLastCommitUnsorted(table, "c2"); + shouldHaveFiles(table, 20); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + RewriteDataFiles.Result result = + basicRewrite(table) + .sort(SortOrder.builderFor(table.schema()).asc("c2").build()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + RewriteDataFiles.TARGET_FILE_SIZE_BYTES, Integer.toString(averageFileSize(table))) + .execute(); + + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + shouldHaveMultipleFiles(table); + shouldHaveLastCommitSorted(table, "c2"); + } + + @TestTemplate + public void testSortCustomSortOrderRequiresRepartition() { + int partitions = 4; + Table table = createTable(); + writeRecords(20, SCALE, partitions); + shouldHaveLastCommitUnsorted(table, "c3"); + + // Add a partition column so this requires repartitioning + table.updateSpec().addField("c1").commit(); + // Add a sort order which our repartitioning needs to ignore + table.replaceSortOrder().asc("c2").apply(); + shouldHaveFiles(table, 20); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + RewriteDataFiles.Result result = + basicRewrite(table) + .sort(SortOrder.builderFor(table.schema()).asc("c3").build()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + RewriteDataFiles.TARGET_FILE_SIZE_BYTES, + Integer.toString(averageFileSize(table) / partitions)) + .execute(); + + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + shouldHaveMultipleFiles(table); + shouldHaveLastCommitUnsorted(table, "c2"); + shouldHaveLastCommitSorted(table, "c3"); + } + + @TestTemplate + public void testAutoSortShuffleOutput() { + Table table = createTable(20); + shouldHaveLastCommitUnsorted(table, "c2"); + shouldHaveFiles(table, 20); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + RewriteDataFiles.Result result = + basicRewrite(table) + .sort(SortOrder.builderFor(table.schema()).asc("c2").build()) + .option( + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, + Integer.toString((averageFileSize(table) / 2) + 2)) + // Divide files in 2 + .option( + RewriteDataFiles.TARGET_FILE_SIZE_BYTES, + Integer.toString(averageFileSize(table) / 2)) + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") + .execute(); + + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); + assertThat(table.currentSnapshot().addedDataFiles(table.io())) + .as("Should have written 40+ files") + .hasSizeGreaterThanOrEqualTo(40); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + shouldHaveMultipleFiles(table); + shouldHaveLastCommitSorted(table, "c2"); + } + + @TestTemplate + public void testCommitStateUnknownException() { + Table table = createTable(20); + shouldHaveFiles(table, 20); + + List originalData = currentData(); + + RewriteDataFilesSparkAction action = basicRewrite(table); + RewriteDataFilesSparkAction spyAction = spy(action); + RewriteDataFilesCommitManager util = spy(new RewriteDataFilesCommitManager(table)); + + doAnswer( + invocationOnMock -> { + invocationOnMock.callRealMethod(); + throw new CommitStateUnknownException(new RuntimeException("Unknown State")); + }) + .when(util) + .commitFileGroups(any()); + + doReturn(util).when(spyAction).commitManager(table.currentSnapshot().snapshotId()); + + assertThatThrownBy(spyAction::execute) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith( + "Unknown State\n" + "Cannot determine whether the commit was successful or not"); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); // Commit actually Succeeded + } + + @TestTemplate + public void testZOrderSort() { + int originalFiles = 20; + Table table = createTable(originalFiles); + shouldHaveLastCommitUnsorted(table, "c2"); + shouldHaveFiles(table, originalFiles); + + List originalData = currentData(); + double originalFilesC2 = percentFilesRequired(table, "c2", "foo23"); + double originalFilesC3 = percentFilesRequired(table, "c3", "bar21"); + double originalFilesC2C3 = + percentFilesRequired(table, new String[] {"c2", "c3"}, new String[] {"foo23", "bar23"}); + + assertThat(originalFilesC2).as("Should require all files to scan c2").isGreaterThan(0.99); + assertThat(originalFilesC3).as("Should require all files to scan c3").isGreaterThan(0.99); + + long dataSizeBefore = testDataSize(table); + RewriteDataFiles.Result result = + basicRewrite(table) + .zOrder("c2", "c3") + .option( + SizeBasedFileRewritePlanner.MAX_FILE_SIZE_BYTES, + Integer.toString((averageFileSize(table) / 2) + 2)) + // Divide files in 2 + .option( + RewriteDataFiles.TARGET_FILE_SIZE_BYTES, + Integer.toString(averageFileSize(table) / 2)) + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") + .execute(); + + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + assertThat(table.currentSnapshot().addedDataFiles(table.io())) + .as("Should have written 40+ files") + .hasSizeGreaterThanOrEqualTo(40); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + + double filesScannedC2 = percentFilesRequired(table, "c2", "foo23"); + double filesScannedC3 = percentFilesRequired(table, "c3", "bar21"); + double filesScannedC2C3 = + percentFilesRequired(table, new String[] {"c2", "c3"}, new String[] {"foo23", "bar23"}); + + assertThat(originalFilesC2) + .as("Should have reduced the number of files required for c2") + .isGreaterThan(filesScannedC2); + assertThat(originalFilesC3) + .as("Should have reduced the number of files required for c3") + .isGreaterThan(filesScannedC3); + assertThat(originalFilesC2C3) + .as("Should have reduced the number of files required for c2,c3 predicate") + .isGreaterThan(filesScannedC2C3); + } + + @TestTemplate + public void testZOrderAllTypesSort() { + spark.conf().set("spark.sql.ansi.enabled", "false"); + Table table = createTypeTestTable(); + shouldHaveFiles(table, 10); + + List originalRaw = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SPLIT_SIZE, 1024 * 1024 * 64) + .option(SparkReadOptions.FILE_OPEN_COST, 0) + .load(tableLocation) + .coalesce(1) + .sort("longCol") + .collectAsList(); + List originalData = rowsToJava(originalRaw); + long dataSizeBefore = testDataSize(table); + + // TODO add in UUID when it is supported in Spark + RewriteDataFiles.Result result = + basicRewrite(table) + .zOrder( + "longCol", + "intCol", + "floatCol", + "doubleCol", + "dateCol", + "timestampCol", + "stringCol", + "binaryCol", + "booleanCol") + .option(SizeBasedFileRewritePlanner.MIN_INPUT_FILES, "1") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + assertThat(table.currentSnapshot().addedDataFiles(table.io())) + .as("Should have written 1 file") + .hasSize(1); + + table.refresh(); + + List postRaw = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SPLIT_SIZE, 1024 * 1024 * 64) + .option(SparkReadOptions.FILE_OPEN_COST, 0) + .load(tableLocation) + .coalesce(1) + .sort("longCol") + .collectAsList(); + List postRewriteData = rowsToJava(postRaw); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + } + + @TestTemplate + public void testInvalidAPIUsage() { + Table table = createTable(1); + + SortOrder sortOrder = SortOrder.builderFor(table.schema()).asc("c2").build(); + + assertThatThrownBy(() -> actions().rewriteDataFiles(table).binPack().sort()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot set rewrite mode, it has already been set to "); + + assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot set rewrite mode, it has already been set to "); + + assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot set rewrite mode, it has already been set to "); + } + + @TestTemplate + public void testSnapshotProperty() { + Table table = createTable(4); + Result ignored = basicRewrite(table).snapshotProperty("key", "value").execute(); + assertThat(table.currentSnapshot().summary()) + .containsAllEntriesOf(ImmutableMap.of("key", "value")); + // make sure internal produced properties are not lost + String[] commitMetricsKeys = + new String[] { + SnapshotSummary.ADDED_FILES_PROP, + SnapshotSummary.DELETED_FILES_PROP, + SnapshotSummary.TOTAL_DATA_FILES_PROP, + SnapshotSummary.CHANGED_PARTITION_COUNT_PROP + }; + assertThat(table.currentSnapshot().summary()).containsKeys(commitMetricsKeys); + } + + @TestTemplate + public void testBinPackRewriterWithSpecificUnparitionedOutputSpec() { + Table table = createTable(10); + shouldHaveFiles(table, 10); + int outputSpecId = table.spec().specId(); + table.updateSpec().addField(Expressions.truncate("c2", 2)).commit(); + + long dataSizeBefore = testDataSize(table); + long count = currentData().size(); + + RewriteDataFiles.Result result = + basicRewrite(table) + .option(RewriteDataFiles.OUTPUT_SPEC_ID, String.valueOf(outputSpecId)) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .binPack() + .execute(); + + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + assertThat(currentData()).hasSize((int) count); + shouldRewriteDataFilesWithPartitionSpec(table, outputSpecId); + } + + @TestTemplate + public void testBinPackRewriterWithSpecificOutputSpec() { + Table table = createTable(10); + shouldHaveFiles(table, 10); + table.updateSpec().addField(Expressions.truncate("c2", 2)).commit(); + int outputSpecId = table.spec().specId(); + table.updateSpec().addField(Expressions.bucket("c3", 2)).commit(); + + long dataSizeBefore = testDataSize(table); + long count = currentData().size(); + + RewriteDataFiles.Result result = + basicRewrite(table) + .option(RewriteDataFiles.OUTPUT_SPEC_ID, String.valueOf(outputSpecId)) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .binPack() + .execute(); + + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + assertThat(currentData()).hasSize((int) count); + shouldRewriteDataFilesWithPartitionSpec(table, outputSpecId); + } + + @TestTemplate + public void testBinpackRewriteWithInvalidOutputSpecId() { + Table table = createTable(10); + shouldHaveFiles(table, 10); + assertThatThrownBy( + () -> + actions() + .rewriteDataFiles(table) + .option(RewriteDataFiles.OUTPUT_SPEC_ID, String.valueOf(1234)) + .binPack() + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot use output spec id 1234 because the table does not contain a reference to this spec-id."); + } + + @TestTemplate + public void testSortRewriterWithSpecificOutputSpecId() { + Table table = createTable(10); + shouldHaveFiles(table, 10); + table.updateSpec().addField(Expressions.truncate("c2", 2)).commit(); + int outputSpecId = table.spec().specId(); + table.updateSpec().addField(Expressions.bucket("c3", 2)).commit(); + + long dataSizeBefore = testDataSize(table); + long count = currentData().size(); + + RewriteDataFiles.Result result = + basicRewrite(table) + .option(RewriteDataFiles.OUTPUT_SPEC_ID, String.valueOf(outputSpecId)) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .sort(SortOrder.builderFor(table.schema()).asc("c2").asc("c3").build()) + .execute(); + + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + assertThat(currentData()).hasSize((int) count); + shouldRewriteDataFilesWithPartitionSpec(table, outputSpecId); + } + + @TestTemplate + public void testZOrderRewriteWithSpecificOutputSpecId() { + Table table = createTable(10); + shouldHaveFiles(table, 10); + table.updateSpec().addField(Expressions.truncate("c2", 2)).commit(); + int outputSpecId = table.spec().specId(); + table.updateSpec().addField(Expressions.bucket("c3", 2)).commit(); + + long dataSizeBefore = testDataSize(table); + long count = currentData().size(); + + RewriteDataFiles.Result result = + basicRewrite(table) + .option(RewriteDataFiles.OUTPUT_SPEC_ID, String.valueOf(outputSpecId)) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .zOrder("c2", "c3") + .execute(); + + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + assertThat(currentData()).hasSize((int) count); + shouldRewriteDataFilesWithPartitionSpec(table, outputSpecId); + } + + @TestTemplate + public void testUnpartitionedRewriteDataFilesPreservesLineage() throws NoSuchTableException { + assumeThat(formatVersion).isGreaterThan(2); + + // Verify the initial row IDs and sequence numbers + Table table = createTable(4); + shouldHaveFiles(table, 4); + List expectedRecordsWithLineage = currentDataWithLineage(); + List rowIds = + expectedRecordsWithLineage.stream() + .map(record -> (Long) record[0]) + .collect(Collectors.toList()); + List lastUpdatedSequenceNumbers = + expectedRecordsWithLineage.stream() + .map(record -> (Long) record[1]) + .collect(Collectors.toList()); + assertThat(rowIds) + .isEqualTo( + LongStream.range(0, expectedRecordsWithLineage.size()) + .boxed() + .collect(Collectors.toList())); + assertThat(lastUpdatedSequenceNumbers).allMatch(sequenceNumber -> sequenceNumber.equals(1L)); + + // Perform and validate compaction + long dataSizeBefore = testDataSize(table); + Result result = basicRewrite(table).execute(); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 4 data files") + .isEqualTo(4); + assertThat(result.addedDataFilesCount()).as("Action should add 1 data file").isOne(); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + shouldHaveFiles(table, 1); + List actualRecordsWithLineage = currentDataWithLineage(); + assertEquals("Rows must match", expectedRecordsWithLineage, actualRecordsWithLineage); + } + + @TestTemplate + public void testRewriteDataFilesPreservesLineage() throws NoSuchTableException { + assumeThat(formatVersion).isGreaterThan(2); + + Table table = createTablePartitioned(4 /* partitions */, 2 /* files per partition */); + shouldHaveFiles(table, 8); + + // Verify the initial row IDs and sequence numbers + List expectedRecords = currentDataWithLineage(); + List rowIds = + expectedRecords.stream().map(record -> (Long) record[0]).collect(Collectors.toList()); + List lastUpdatedSequenceNumbers = + expectedRecords.stream().map(record -> (Long) record[1]).collect(Collectors.toList()); + assertThat(rowIds) + .isEqualTo( + LongStream.range(0, expectedRecords.size()).boxed().collect(Collectors.toList())); + assertThat(lastUpdatedSequenceNumbers).allMatch(sequenceNumber -> sequenceNumber.equals(1L)); + + // Perform and validate compaction + long dataSizeBefore = testDataSize(table); + Result result = basicRewrite(table).execute(); + assertThat(result.rewrittenDataFilesCount()) + .as("Action should rewrite 8 data files") + .isEqualTo(8); + assertThat(result.addedDataFilesCount()).as("Action should add 4 data file").isEqualTo(4); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + shouldHaveFiles(table, 4); + List actualRecordsWithLineage = currentDataWithLineage(); + assertEquals("Rows must match", expectedRecords, actualRecordsWithLineage); + } + + @TestTemplate + public void testExecutorCacheForDeleteFilesDisabled() { + Table table = createTablePartitioned(1, 1); + RewriteDataFilesSparkAction action = SparkActions.get(spark).rewriteDataFiles(table); + + // The constructor should have set the configuration to false + SparkReadConf readConf = new SparkReadConf(action.spark(), table, Collections.emptyMap()); + assertThat(readConf.cacheDeleteFilesOnExecutors()) + .as("Executor cache for delete files should be disabled in RewriteDataFilesSparkAction") + .isFalse(); + } + + @TestTemplate + public void testZOrderUDFWithDateType() { + SparkZOrderUDF zorderUDF = new SparkZOrderUDF(1, 16, 1024); + Dataset result = + spark + .sql("SELECT DATE '2025-01-01' as test_col") + .withColumn( + "zorder_result", + zorderUDF.sortedLexicographically(col("test_col"), DataTypes.DateType)); + + assertThat(result.schema().apply("zorder_result").dataType()).isEqualTo(DataTypes.BinaryType); + List rows = result.collectAsList(); + Row row = rows.get(0); + byte[] zorderBytes = row.getAs("zorder_result"); + assertThat(zorderBytes).isNotNull().isNotEmpty(); + } + + protected void shouldRewriteDataFilesWithPartitionSpec(Table table, int outputSpecId) { + List rewrittenFiles = currentDataFiles(table); + assertThat(rewrittenFiles).allMatch(file -> file.specId() == outputSpecId); + assertThat(rewrittenFiles) + .allMatch( + file -> + ((PartitionData) file.partition()) + .getPartitionType() + .equals(table.specs().get(outputSpecId).partitionType())); + } + + protected List currentDataFiles(Table table) { + return Streams.stream(table.newScan().planFiles()) + .map(FileScanTask::file) + .collect(Collectors.toList()); + } + + protected List currentData() { + return rowsToJava( + spark + .read() + .option(SparkReadOptions.SPLIT_SIZE, 1024 * 1024 * 64) + .option(SparkReadOptions.FILE_OPEN_COST, 0) + .format("iceberg") + .load(tableLocation) + .coalesce(1) + .sort("c1", "c2", "c3") + .collectAsList()); + } + + protected List currentDataWithLineage() { + return rowsToJava( + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SPLIT_SIZE, 1024 * 1024 * 64) + .option(SparkReadOptions.FILE_OPEN_COST, 0) + .load(tableLocation) + .coalesce(1) + .sort("_row_id") + .selectExpr("_row_id", "_last_updated_sequence_number", "*") + .collectAsList()); + } + + protected long testDataSize(Table table) { + return Streams.stream(table.newScan().planFiles()).mapToLong(FileScanTask::length).sum(); + } + + protected void shouldHaveMultipleFiles(Table table) { + table.refresh(); + int numFiles = Iterables.size(table.newScan().planFiles()); + assertThat(numFiles) + .as(String.format("Should have multiple files, had %d", numFiles)) + .isGreaterThan(1); + } + + protected void shouldHaveFiles(Table table, int numExpected) { + table.refresh(); + List files = + StreamSupport.stream(table.newScan().planFiles().spliterator(), false) + .collect(Collectors.toList()); + assertThat(files.size()).as("Did not have the expected number of files").isEqualTo(numExpected); + } + + protected long shouldHaveMinSequenceNumberInPartition( + Table table, String partitionFilter, long expected) { + long actual = + SparkTableUtil.loadMetadataTable(spark, table, MetadataTableType.ENTRIES) + .filter("status != 2") + .filter(partitionFilter) + .select("sequence_number") + .agg(min("sequence_number")) + .as(Encoders.LONG()) + .collectAsList() + .get(0); + assertThat(actual).as("Did not have the expected min sequence number").isEqualTo(expected); + return actual; + } + + protected void shouldHaveSnapshots(Table table, int expectedSnapshots) { + table.refresh(); + assertThat(table.snapshots()) + .as("Table did not have the expected number of snapshots") + .hasSize(expectedSnapshots); + } + + protected void shouldHaveNoOrphans(Table table) { + assertThat( + actions() + .deleteOrphanFiles(table) + .olderThan(System.currentTimeMillis()) + .execute() + .orphanFileLocations()) + .as("Should not have found any orphan files") + .isEmpty(); + } + + protected void shouldHaveOrphans(Table table) { + assertThat( + actions() + .deleteOrphanFiles(table) + .olderThan(System.currentTimeMillis()) + .execute() + .orphanFileLocations()) + .as("Should have found orphan files") + .isNotEmpty(); + } + + protected void shouldHaveACleanCache(Table table) { + assertThat(cacheContents(table)).as("Should not have any entries in cache").isEmpty(); + } + + protected void shouldHaveLastCommitSorted(Table table, String column) { + List, Pair>> overlappingFiles = checkForOverlappingFiles(table, column); + + assertThat(overlappingFiles).as("Found overlapping files").isEmpty(); + } + + protected void shouldHaveLastCommitUnsorted(Table table, String column) { + List, Pair>> overlappingFiles = checkForOverlappingFiles(table, column); + + assertThat(overlappingFiles).as("Found no overlapping files").isNotEmpty(); + } + + private Pair boundsOf(DataFile file, NestedField field, Class javaClass) { + int columnId = field.fieldId(); + return Pair.of( + javaClass.cast(Conversions.fromByteBuffer(field.type(), file.lowerBounds().get(columnId))), + javaClass.cast(Conversions.fromByteBuffer(field.type(), file.upperBounds().get(columnId)))); + } + + private List, Pair>> checkForOverlappingFiles( + Table table, String column) { + table.refresh(); + NestedField field = table.schema().caseInsensitiveFindField(column); + Class javaClass = (Class) field.type().typeId().javaClass(); + + Snapshot snapshot = table.currentSnapshot(); + Map> filesByPartition = + Streams.stream(snapshot.addedDataFiles(table.io())) + .collect(Collectors.groupingBy(DataFile::partition)); + + Stream, Pair>> overlaps = + filesByPartition.entrySet().stream() + .flatMap( + entry -> { + List datafiles = entry.getValue(); + Preconditions.checkArgument( + datafiles.size() > 1, + "This test is checking for overlaps in a situation where no overlaps can actually occur because the " + + "partition %s does not contain multiple datafiles", + entry.getKey()); + + List, Pair>> boundComparisons = + Lists.cartesianProduct(datafiles, datafiles).stream() + .filter(tuple -> tuple.get(0) != tuple.get(1)) + .map( + tuple -> + Pair.of( + boundsOf(tuple.get(0), field, javaClass), + boundsOf(tuple.get(1), field, javaClass))) + .collect(Collectors.toList()); + + Comparator comparator = Comparators.forType(field.type().asPrimitiveType()); + + List, Pair>> overlappingFiles = + boundComparisons.stream() + .filter( + filePair -> { + Pair left = filePair.first(); + T lMin = left.first(); + T lMax = left.second(); + Pair right = filePair.second(); + T rMin = right.first(); + T rMax = right.second(); + boolean boundsDoNotOverlap = + // Min and Max of a range are greater than or equal to the max + // value of the other range + (comparator.compare(rMax, lMax) >= 0 + && comparator.compare(rMin, lMax) >= 0) + || (comparator.compare(lMax, rMax) >= 0 + && comparator.compare(lMin, rMax) >= 0); + + return !boundsDoNotOverlap; + }) + .collect(Collectors.toList()); + return overlappingFiles.stream(); + }); + + return overlaps.collect(Collectors.toList()); + } + + protected Table createTable() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + table + .updateProperties() + .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, Integer.toString(20 * 1024)) + .commit(); + assertThat(table.currentSnapshot()).as("Table must be empty").isNull(); + return table; + } + + /** + * Create a table with a certain number of files, returns the size of a file + * + * @param files number of files to create + * @return the created table + */ + protected Table createTable(int files) { + Table table = createTable(); + writeRecords(files, SCALE); + return table; + } + + protected Table createTablePartitioned( + int partitions, int files, int numRecords, Map options) { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 2).build(); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + assertThat(table.currentSnapshot()).as("Table must be empty").isNull(); + + writeRecords(files, numRecords, partitions); + return table; + } + + protected Table createTablePartitioned(int partitions, int files) { + return createTablePartitioned( + partitions, + files, + SCALE, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion))); + } + + protected Table createTablePartitioned(int partitions, int files, int numRecords) { + return createTablePartitioned( + partitions, + files, + numRecords, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion))); + } + + private Table createTypeTestTable() { + Schema schema = + new Schema( + required(1, "longCol", Types.LongType.get()), + required(2, "intCol", Types.IntegerType.get()), + required(3, "floatCol", Types.FloatType.get()), + optional(4, "doubleCol", Types.DoubleType.get()), + optional(5, "dateCol", Types.DateType.get()), + optional(6, "timestampCol", Types.TimestampType.withZone()), + optional(7, "stringCol", Types.StringType.get()), + optional(8, "booleanCol", Types.BooleanType.get()), + optional(9, "binaryCol", Types.BinaryType.get())); + + Map options = + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + Table table = TABLES.create(schema, PartitionSpec.unpartitioned(), options, tableLocation); + + spark + .range(0, 10, 1, 10) + .withColumnRenamed("id", "longCol") + .withColumn("intCol", expr("CAST(longCol AS INT)")) + .withColumn("floatCol", expr("CAST(longCol AS FLOAT)")) + .withColumn("doubleCol", expr("CAST(longCol AS DOUBLE)")) + .withColumn("dateCol", date_add(current_date(), 1)) + .withColumn("timestampCol", expr("TO_TIMESTAMP(dateCol)")) + .withColumn("stringCol", expr("CAST(dateCol AS STRING)")) + .withColumn("booleanCol", expr("longCol > 5")) + .withColumn("binaryCol", expr("CAST(longCol AS BINARY)")) + .write() + .format("iceberg") + .mode("append") + .save(tableLocation); + + return table; + } + + protected int averageFileSize(Table table) { + table.refresh(); + return (int) + Streams.stream(table.newScan().planFiles()) + .mapToLong(FileScanTask::length) + .average() + .getAsDouble(); + } + + private void writeRecords(List records) { + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); + writeDF(df); + } + + private void writeRecords(int files, int numRecords) { + writeRecords(files, numRecords, 0); + } + + private void writeRecords(int files, int numRecords, int partitions) { + List records = Lists.newArrayList(); + int rowDimension = (int) Math.ceil(Math.sqrt(numRecords)); + List> data = + IntStream.range(0, rowDimension) + .boxed() + .flatMap(x -> IntStream.range(0, rowDimension).boxed().map(y -> Pair.of(x, y))) + .collect(Collectors.toList()); + Collections.shuffle(data, new Random(42)); + if (partitions > 0) { + data.forEach( + i -> + records.add( + new ThreeColumnRecord( + i.first() % partitions, "foo" + i.first(), "bar" + i.second()))); + } else { + data.forEach( + i -> + records.add(new ThreeColumnRecord(i.first(), "foo" + i.first(), "bar" + i.second()))); + } + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).repartition(files); + writeDF(df); + } + + private void writeDF(Dataset df) { + df.select("c1", "c2", "c3") + .sortWithinPartitions("c1", "c2") + .write() + .format("iceberg") + .mode("append") + .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") + .save(tableLocation); + } + + private List writePosDeletesToFile( + Table table, DataFile dataFile, int outputDeleteFiles) { + return writePosDeletes(table, dataFile.partition(), dataFile.location(), outputDeleteFiles); + } + + private List writePosDeletes( + Table table, StructLike partition, String path, int outputDeleteFiles) { + List results = Lists.newArrayList(); + int rowPosition = 0; + for (int file = 0; file < outputDeleteFiles; file++) { + OutputFile outputFile = + table + .io() + .newOutputFile( + table + .locationProvider() + .newDataLocation( + FileFormat.PARQUET.addExtension(UUID.randomUUID().toString()))); + + PositionDeleteWriter posDeleteWriter = + new GenericFileWriterFactory.Builder(table) + .deleteFileFormat(FileFormat.PARQUET) + .writerProperties(ImmutableMap.of(TableProperties.DEFAULT_WRITE_METRICS_MODE, "full")) + .build() + .newPositionDeleteWriter(encrypt(outputFile), table.spec(), partition); + + PositionDelete posDelete = PositionDelete.create(); + posDeleteWriter.write(posDelete.set(path, rowPosition, null)); + try { + posDeleteWriter.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + results.add(posDeleteWriter.toDeleteFile()); + rowPosition++; + } + + return results; + } + + private List writePosDeletes( + Table table, + StructLike partition, + String path, + int outputDeleteFiles, + int totalPositionsToDelete) { + List results = Lists.newArrayList(); + for (int file = 0; file < outputDeleteFiles; file++) { + OutputFile outputFile = + table + .io() + .newOutputFile( + table + .locationProvider() + .newDataLocation( + FileFormat.PARQUET.addExtension(UUID.randomUUID().toString()))); + + PositionDeleteWriter posDeleteWriter = + new GenericFileWriterFactory.Builder(table) + .deleteFileFormat(FileFormat.PARQUET) + .writerProperties(ImmutableMap.of(TableProperties.DEFAULT_WRITE_METRICS_MODE, "full")) + .build() + .newPositionDeleteWriter(encrypt(outputFile), table.spec(), partition); + + PositionDelete posDelete = PositionDelete.create(); + int positionsPerDeleteFile = totalPositionsToDelete / outputDeleteFiles; + + for (int position = file * positionsPerDeleteFile; + position < (file + 1) * positionsPerDeleteFile; + position++) { + posDeleteWriter.write(posDelete.set(path, position, null)); + } + + try { + posDeleteWriter.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + results.add(posDeleteWriter.toDeleteFile()); + } + + return results; + } + + private List writeDV( + Table table, StructLike partition, String path, int numPositionsToDelete) throws IOException { + OutputFileFactory fileFactory = + OutputFileFactory.builderFor(table, 1, 1).format(FileFormat.PUFFIN).build(); + DVFileWriter writer = new BaseDVFileWriter(fileFactory, p -> null); + try (DVFileWriter closeableWriter = writer) { + for (int row = 0; row < numPositionsToDelete; row++) { + closeableWriter.delete(path, row, table.spec(), partition); + } + } + + return writer.result().deleteFiles(); + } + + private void writeEqDeleteRecord( + Table table, String partCol, Object partVal, String delCol, Object delVal) { + List equalityFieldIds = Lists.newArrayList(table.schema().findField(delCol).fieldId()); + Schema eqDeleteRowSchema = table.schema().select(delCol); + Record partitionRecord = + GenericRecord.create(table.schema().select(partCol)) + .copy(ImmutableMap.of(partCol, partVal)); + Record record = GenericRecord.create(eqDeleteRowSchema).copy(ImmutableMap.of(delCol, delVal)); + writeEqDeleteRecord(table, equalityFieldIds, partitionRecord, eqDeleteRowSchema, record); + } + + private void writeEqDeleteRecord( + Table table, + List equalityFieldIds, + Record partitionRecord, + Schema eqDeleteRowSchema, + Record deleteRecord) { + OutputFileFactory fileFactory = + OutputFileFactory.builderFor(table, 1, 1).format(FileFormat.PARQUET).build(); + + EncryptedOutputFile file = + createEncryptedOutputFile(createPartitionKey(table, partitionRecord), fileFactory); + + EqualityDeleteWriter eqDeleteWriter = + new GenericFileWriterFactory.Builder(table) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(eqDeleteRowSchema) + .deleteFileFormat(FileFormat.PARQUET) + .build() + .newEqualityDeleteWriter( + file, table.spec(), createPartitionKey(table, partitionRecord)); + + try (EqualityDeleteWriter clsEqDeleteWriter = eqDeleteWriter) { + clsEqDeleteWriter.write(deleteRecord); + } catch (Exception e) { + throw new RuntimeException(e); + } + table.newRowDelta().addDeletes(eqDeleteWriter.toDeleteFile()).commit(); + } + + private PartitionKey createPartitionKey(Table table, Record record) { + if (table.spec().isUnpartitioned()) { + return null; + } + + PartitionKey partitionKey = new PartitionKey(table.spec(), table.schema()); + partitionKey.partition(record); + + return partitionKey; + } + + private EncryptedOutputFile createEncryptedOutputFile( + PartitionKey partition, OutputFileFactory fileFactory) { + if (partition == null) { + return fileFactory.newOutputFile(); + } else { + return fileFactory.newOutputFile(partition); + } + } + + private SparkActions actions() { + return SparkActions.get(); + } + + private Set cacheContents(Table table) { + return ImmutableSet.builder() + .addAll(manager.fetchSetIds(table)) + .addAll(coordinator.fetchSetIds(table)) + .build(); + } + + private double percentFilesRequired(Table table, String col, String value) { + return percentFilesRequired(table, new String[] {col}, new String[] {value}); + } + + private double percentFilesRequired(Table table, String[] cols, String[] values) { + Preconditions.checkArgument(cols.length == values.length); + Expression restriction = Expressions.alwaysTrue(); + for (int i = 0; i < cols.length; i++) { + restriction = Expressions.and(restriction, Expressions.equal(cols[i], values[i])); + } + int totalFiles = Iterables.size(table.newScan().planFiles()); + int filteredFiles = Iterables.size(table.newScan().filter(restriction).planFiles()); + return (double) filteredFiles / (double) totalFiles; + } + + class GroupInfoMatcher implements ArgumentMatcher { + private final Set groupIDs; + + GroupInfoMatcher(Integer... globalIndex) { + this.groupIDs = ImmutableSet.copyOf(globalIndex); + } + + @Override + public boolean matches(RewriteFileGroup argument) { + return groupIDs.contains(argument.info().globalIndex()); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java new file mode 100644 index 000000000000..4f95416878e2 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -0,0 +1,1312 @@ +/* + * 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.actions; + +import static org.apache.iceberg.ValidationHelpers.dataSeqs; +import static org.apache.iceberg.ValidationHelpers.fileSeqs; +import static org.apache.iceberg.ValidationHelpers.files; +import static org.apache.iceberg.ValidationHelpers.snapshotIds; +import static org.apache.iceberg.ValidationHelpers.validateDataManifest; +import static org.apache.iceberg.types.Types.NestedField.optional; +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 static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileGenerationUtil; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Files; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.actions.RewriteManifests; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.CloseableIterable; +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.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.SparkTableUtil; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.spark.source.ThreeColumnRecord; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewriteManifestsAction extends TestBase { + + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + private static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + + @Parameters( + name = + "snapshotIdInheritanceEnabled = {0}, useCaching = {1}, shouldStageManifests = {2}, formatVersion = {3}") + public static Object[] parameters() { + return new Object[][] { + new Object[] {"true", "true", false, 1}, + new Object[] {"false", "true", true, 1}, + new Object[] {"true", "false", false, 2}, + new Object[] {"false", "false", false, 2}, + new Object[] {"true", "false", false, 3}, + new Object[] {"false", "false", false, 3} + }; + } + + @Parameter private String snapshotIdInheritanceEnabled; + + @Parameter(index = 1) + private String useCaching; + + @Parameter(index = 2) + private boolean shouldStageManifests; + + @Parameter(index = 3) + private int formatVersion; + + private String tableLocation = null; + + @TempDir private Path temp; + @TempDir private File tableDir; + + @BeforeEach + public void setupTableLocation() throws Exception { + this.tableLocation = tableDir.toURI().toString(); + } + + @TestTemplate + public void testRewriteManifestsPreservesOptionalFields() throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(2); + + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + DataFile dataFile1 = newDataFile(table, "c1=0"); + DataFile dataFile2 = newDataFile(table, "c1=0"); + DataFile dataFile3 = newDataFile(table, "c1=0"); + table + .newFastAppend() + .appendFile(dataFile1) + .appendFile(dataFile2) + .appendFile(dataFile3) + .commit(); + + DeleteFile deleteFile1 = newDeletes(table, dataFile1); + assertDeletes(dataFile1, deleteFile1); + table.newRowDelta().addDeletes(deleteFile1).commit(); + + DeleteFile deleteFile2 = newDeletes(table, dataFile2); + assertDeletes(dataFile2, deleteFile2); + table.newRowDelta().addDeletes(deleteFile2).commit(); + + DeleteFile deleteFile3 = newDeletes(table, dataFile3); + assertDeletes(dataFile3, deleteFile3); + table.newRowDelta().addDeletes(deleteFile3).commit(); + + SparkActions actions = SparkActions.get(); + + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + table.refresh(); + + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (FileScanTask fileTask : tasks) { + DataFile dataFile = fileTask.file(); + DeleteFile deleteFile = Iterables.getOnlyElement(fileTask.deletes()); + if (dataFile.location().equals(dataFile1.location())) { + assertThat(deleteFile.referencedDataFile()).isEqualTo(deleteFile1.referencedDataFile()); + assertEqual(deleteFile, deleteFile1); + } else if (dataFile.location().equals(dataFile2.location())) { + assertThat(deleteFile.referencedDataFile()).isEqualTo(deleteFile2.referencedDataFile()); + assertEqual(deleteFile, deleteFile2); + } else { + assertThat(deleteFile.referencedDataFile()).isEqualTo(deleteFile3.referencedDataFile()); + assertEqual(deleteFile, deleteFile3); + } + } + } + } + + @TestTemplate + public void testRewriteManifestsEmptyTable() throws IOException { + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + assertThat(table.currentSnapshot()).as("Table must be empty").isNull(); + + SparkActions actions = SparkActions.get(); + + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .stagingLocation(java.nio.file.Files.createTempDirectory(temp, "junit").toString()) + .execute(); + + assertThat(table.currentSnapshot()).as("Table must stay empty").isNull(); + } + + @TestTemplate + public void testRewriteSmallManifestsNonPartitionedTable() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + List records1 = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), new ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")); + writeRecords(records1); + + List records2 = + Lists.newArrayList( + new ThreeColumnRecord(2, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(2, "DDDDDDDDDD", "DDDD")); + writeRecords(records2); + + table.refresh(); + + List manifests = table.currentSnapshot().allManifests(table.io()); + assertThat(manifests).as("Should have 2 manifests before rewrite").hasSize(2); + + SparkActions actions = SparkActions.get(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + assertThat(result.rewrittenManifests()).as("Action should rewrite 2 manifests").hasSize(2); + assertThat(result.addedManifests()).as("Action should add 1 manifests").hasSize(1); + assertManifestsLocation(result.addedManifests()); + + table.refresh(); + + List newManifests = table.currentSnapshot().allManifests(table.io()); + assertThat(newManifests).as("Should have 1 manifests after rewrite").hasSize(1); + + assertThat(newManifests.get(0).existingFilesCount()).isEqualTo(4); + assertThat(newManifests.get(0).hasAddedFiles()).isFalse(); + assertThat(newManifests.get(0).hasDeletedFiles()).isFalse(); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(records1); + expectedRecords.addAll(records2); + + Dataset resultDF = spark.read().format("iceberg").load(tableLocation); + List actualRecords = + resultDF.sort("c1", "c2").as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); + } + + @TestTemplate + public void testRewriteManifestsWithCommitStateUnknownException() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + List records1 = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), new ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")); + writeRecords(records1); + + List records2 = + Lists.newArrayList( + new ThreeColumnRecord(2, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(2, "DDDDDDDDDD", "DDDD")); + writeRecords(records2); + + table.refresh(); + + List manifests = table.currentSnapshot().allManifests(table.io()); + assertThat(manifests).as("Should have 2 manifests before rewrite").hasSize(2); + + SparkActions actions = SparkActions.get(); + + // create a spy which would throw a CommitStateUnknownException after successful commit. + org.apache.iceberg.RewriteManifests newRewriteManifests = table.rewriteManifests(); + org.apache.iceberg.RewriteManifests spyNewRewriteManifests = spy(newRewriteManifests); + doAnswer( + invocation -> { + newRewriteManifests.commit(); + throw new CommitStateUnknownException(new RuntimeException("Datacenter on Fire")); + }) + .when(spyNewRewriteManifests) + .commit(); + + Table spyTable = spy(table); + when(spyTable.rewriteManifests()).thenReturn(spyNewRewriteManifests); + + assertThatThrownBy( + () -> actions.rewriteManifests(spyTable).rewriteIf(manifest -> true).execute()) + .cause() + .isInstanceOf(RuntimeException.class) + .hasMessage("Datacenter on Fire"); + + table.refresh(); + + // table should reflect the changes, since the commit was successful + List newManifests = table.currentSnapshot().allManifests(table.io()); + assertThat(newManifests).as("Should have 1 manifests after rewrite").hasSize(1); + + assertThat(newManifests.get(0).existingFilesCount()).isEqualTo(4); + assertThat(newManifests.get(0).hasAddedFiles()).isFalse(); + assertThat(newManifests.get(0).hasDeletedFiles()).isFalse(); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(records1); + expectedRecords.addAll(records2); + + Dataset resultDF = spark.read().format("iceberg").load(tableLocation); + List actualRecords = + resultDF.sort("c1", "c2").as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); + } + + @TestTemplate + public void testRewriteSmallManifestsPartitionedTable() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 2).build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + List records1 = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), new ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")); + writeRecords(records1); + + List records2 = + Lists.newArrayList( + new ThreeColumnRecord(2, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(2, "DDDDDDDDDD", "DDDD")); + writeRecords(records2); + + List records3 = + Lists.newArrayList( + new ThreeColumnRecord(3, "EEEEEEEEEE", "EEEE"), + new ThreeColumnRecord(3, "FFFFFFFFFF", "FFFF")); + writeRecords(records3); + + List records4 = + Lists.newArrayList( + new ThreeColumnRecord(4, "GGGGGGGGGG", "GGGG"), + new ThreeColumnRecord(4, "HHHHHHHHHG", "HHHH")); + writeRecords(records4); + + table.refresh(); + + List manifests = table.currentSnapshot().allManifests(table.io()); + assertThat(manifests).as("Should have 4 manifests before rewrite").hasSize(4); + + SparkActions actions = SparkActions.get(); + + // we will expect to have 2 manifests with 4 entries in each after rewrite + long manifestEntrySizeBytes = computeManifestEntrySizeBytes(manifests); + long targetManifestSizeBytes = (long) (1.05 * 4 * manifestEntrySizeBytes); + + table + .updateProperties() + .set(TableProperties.MANIFEST_TARGET_SIZE_BYTES, String.valueOf(targetManifestSizeBytes)) + .commit(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + assertThat(result.rewrittenManifests()).as("Action should rewrite 4 manifests").hasSize(4); + assertThat(result.addedManifests()).as("Action should add 2 manifests").hasSize(2); + assertManifestsLocation(result.addedManifests()); + + table.refresh(); + + List newManifests = table.currentSnapshot().allManifests(table.io()); + assertThat(newManifests).as("Should have 2 manifests after rewrite").hasSize(2); + + assertThat(newManifests.get(0).existingFilesCount()).isEqualTo(4); + assertThat(newManifests.get(0).hasAddedFiles()).isFalse(); + assertThat(newManifests.get(0).hasDeletedFiles()).isFalse(); + + assertThat(newManifests.get(1).existingFilesCount()).isEqualTo(4); + assertThat(newManifests.get(1).hasAddedFiles()).isFalse(); + assertThat(newManifests.get(1).hasDeletedFiles()).isFalse(); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(records1); + expectedRecords.addAll(records2); + expectedRecords.addAll(records3); + expectedRecords.addAll(records4); + + Dataset resultDF = spark.read().format("iceberg").load(tableLocation); + List actualRecords = + resultDF.sort("c1", "c2").as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); + } + + @TestTemplate + public void testRewriteImportedManifests() throws IOException { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c3").build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + List records = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), new ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")); + File parquetTableDir = temp.resolve("parquet_table").toFile(); + String parquetTableLocation = parquetTableDir.toURI().toString(); + + try { + Dataset inputDF = spark.createDataFrame(records, ThreeColumnRecord.class); + inputDF + .select("c1", "c2", "c3") + .write() + .format("parquet") + .mode("overwrite") + .option("path", parquetTableLocation) + .partitionBy("c3") + .saveAsTable("parquet_table"); + + File stagingDir = temp.resolve("staging-dir").toFile(); + SparkTableUtil.importSparkTable( + spark, new TableIdentifier("parquet_table"), table, stagingDir.toString()); + + // add some more data to create more than one manifest for the rewrite + inputDF.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(tableLocation); + table.refresh(); + + Snapshot snapshot = table.currentSnapshot(); + + SparkActions actions = SparkActions.get(); + + String rewriteStagingLocation = + java.nio.file.Files.createTempDirectory(temp, "junit").toString(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .stagingLocation(rewriteStagingLocation) + .execute(); + + assertThat(result.rewrittenManifests()) + .as("Action should rewrite all manifests") + .isEqualTo(snapshot.allManifests(table.io())); + assertThat(result.addedManifests()).as("Action should add 1 manifest").hasSize(1); + assertManifestsLocation(result.addedManifests(), rewriteStagingLocation); + + } finally { + spark.sql("DROP TABLE parquet_table"); + } + } + + @TestTemplate + public void testRewriteLargeManifestsPartitionedTable() throws IOException { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c3").build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + List dataFiles = Lists.newArrayList(); + for (int fileOrdinal = 0; fileOrdinal < 1000; fileOrdinal++) { + dataFiles.add(newDataFile(table, "c3=" + fileOrdinal)); + } + ManifestFile appendManifest = writeManifest(table, dataFiles); + table.newFastAppend().appendManifest(appendManifest).commit(); + + List manifests = table.currentSnapshot().allManifests(table.io()); + assertThat(manifests).as("Should have 1 manifests before rewrite").hasSize(1); + + // set the target manifest size to a small value to force splitting records into multiple files + table + .updateProperties() + .set( + TableProperties.MANIFEST_TARGET_SIZE_BYTES, + String.valueOf(manifests.get(0).length() / 2)) + .commit(); + + SparkActions actions = SparkActions.get(); + + String stagingLocation = java.nio.file.Files.createTempDirectory(temp, "junit").toString(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .stagingLocation(stagingLocation) + .execute(); + + assertThat(result.rewrittenManifests()).hasSize(1); + assertThat(result.addedManifests()).hasSizeGreaterThanOrEqualTo(2); + assertManifestsLocation(result.addedManifests(), stagingLocation); + + table.refresh(); + + List newManifests = table.currentSnapshot().allManifests(table.io()); + assertThat(newManifests).hasSizeGreaterThanOrEqualTo(2); + } + + @TestTemplate + public void testRewriteManifestsPartitionedTableWithInvalidSortingColumns() throws IOException { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").bucket("c3", 10).build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + SparkActions actions = org.apache.iceberg.spark.actions.SparkActions.get(); + + List nonexistentFields = ImmutableList.of("c1", "c2"); + assertThatThrownBy( + () -> + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .sortBy(nonexistentFields) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot set manifest sorting because specified field(s) [c2] were not found in " + + "current partition spec [\n" + + " 1000: c1: identity(1)\n" + + " 1001: c3_bucket: bucket[10](3)\n" + + "]. Spec ID 0"); + + // c3_bucket is the correct internal partition name to use, c3 is the untransformed column name, + // sortBy() expects the hidden partition column names + List hasIncorrectPartitionFieldNames = ImmutableList.of("c1", "c3"); + assertThatThrownBy( + () -> + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .sortBy(hasIncorrectPartitionFieldNames) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot set manifest sorting because specified field(s) [c3] were not found in " + + "current partition spec [\n" + + " 1000: c1: identity(1)\n" + + " 1001: c3_bucket: bucket[10](3)\n" + + "]. Spec ID 0"); + } + + @TestTemplate + public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOException { + Random random = new Random(4141912); + + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 3).bucket("c3", 10).build(); + Table table = TABLES.create(SCHEMA, spec, tableLocation); + + // write a large number of random records so the rewrite will split into multiple manifests + List dataFiles = Lists.newArrayList(); + for (int i = 0; i < 1000; i++) { + dataFiles.add( + newDataFile( + table, + TestHelpers.Row.of(i, String.valueOf(random.nextInt() * 100), random.nextInt(10)))); + } + ManifestFile appendManifest = writeManifest(table, dataFiles); + table.newFastAppend().appendManifest(appendManifest).commit(); + + // force manifest splitting + table + .updateProperties() + .set( + TableProperties.MANIFEST_TARGET_SIZE_BYTES, String.valueOf(appendManifest.length() / 2)) + .commit(); + + List clusterKeys = ImmutableList.of("c3_bucket", "c2_trunc", "c1"); + SparkActions actions = SparkActions.get(); + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .sortBy(clusterKeys) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + table.refresh(); + + // Read the manifests metadata table. The partition_summaries column contains + // an array of structs with lower_bound and upper_bound as strings + Dataset manifestsDf = spark.read().format("iceberg").load(tableLocation + "#manifests"); + + List bounds = Lists.newArrayList(); + for (Row row : manifestsDf.select("partition_summaries").collectAsList()) { + // partition_summaries is an array of structs; + // index 2 corresponds to c3_bucket since it is the third ordinal field in the table + List summaries = row.getList(0); + Row c3Summary = summaries.get(2); + // lower_bound and upper_bound are at positions 2 and 3 of the summary struct + String lower = c3Summary.getString(2); + String upper = c3Summary.getString(3); + bounds.add(Integer.valueOf(lower)); + bounds.add(Integer.valueOf(upper)); + } + + // Ensure that the list of bounds is sorted; if custom sorting is working, + // the lower/upper bounds should form a non‑decreasing sequence. AKA [0, 4, 4, 9] + assertThat(bounds) + .as("Manifest boundaries should be sorted") + .isSortedAccordingTo(Integer::compareTo); + + // Make sure we have at least two manifests, otherwise the test misses the point + assertThat(manifestsDf.count()).as("There should be at least 2 manifests").isGreaterThan(1L); + } + + @TestTemplate + public void testRewriteManifestsWithPredicate() throws IOException { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 2).build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + List records1 = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), new ThreeColumnRecord(1, "BBBBBBBBBB", "BBBB")); + writeRecords(records1); + + writeRecords(records1); + + List records2 = + Lists.newArrayList( + new ThreeColumnRecord(2, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(2, "DDDDDDDDDD", "DDDD")); + writeRecords(records2); + + table.refresh(); + + List manifests = table.currentSnapshot().allManifests(table.io()); + assertThat(manifests).as("Should have 3 manifests before rewrite").hasSize(3); + + SparkActions actions = SparkActions.get(); + + String stagingLocation = java.nio.file.Files.createTempDirectory(temp, "junit").toString(); + + // rewrite only the first manifest + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf( + manifest -> + (manifest.path().equals(manifests.get(0).path()) + || (manifest.path().equals(manifests.get(1).path())))) + .stagingLocation(stagingLocation) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + assertThat(result.rewrittenManifests()).as("Action should rewrite 2 manifest").hasSize(2); + assertThat(result.addedManifests()).as("Action should add 1 manifests").hasSize(1); + assertManifestsLocation(result.addedManifests(), stagingLocation); + + table.refresh(); + + List newManifests = table.currentSnapshot().allManifests(table.io()); + assertThat(newManifests) + .as("Should have 2 manifests after rewrite") + .hasSize(2) + .as("First manifest must be rewritten") + .doesNotContain(manifests.get(0)) + .as("Second manifest must be rewritten") + .doesNotContain(manifests.get(1)) + .as("Third manifest must not be rewritten") + .contains(manifests.get(2)); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.add(records1.get(0)); + expectedRecords.add(records1.get(0)); + expectedRecords.add(records1.get(1)); + expectedRecords.add(records1.get(1)); + expectedRecords.addAll(records2); + + Dataset resultDF = spark.read().format("iceberg").load(tableLocation); + List actualRecords = + resultDF.sort("c1", "c2").as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); + } + + @TestTemplate + public void testRewriteSmallManifestsNonPartitionedV2Table() { + assumeThat(formatVersion).isGreaterThan(1); + + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map properties = ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"); + Table table = TABLES.create(SCHEMA, spec, properties, tableLocation); + + List records1 = Lists.newArrayList(new ThreeColumnRecord(1, null, "AAAA")); + writeRecords(records1); + + table.refresh(); + + Snapshot snapshot1 = table.currentSnapshot(); + DataFile file1 = Iterables.getOnlyElement(snapshot1.addedDataFiles(table.io())); + + List records2 = Lists.newArrayList(new ThreeColumnRecord(2, "CCCC", "CCCC")); + writeRecords(records2); + + table.refresh(); + + Snapshot snapshot2 = table.currentSnapshot(); + DataFile file2 = Iterables.getOnlyElement(snapshot2.addedDataFiles(table.io())); + + List manifests = table.currentSnapshot().allManifests(table.io()); + assertThat(manifests).as("Should have 2 manifests before rewrite").hasSize(2); + + SparkActions actions = SparkActions.get(); + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + assertThat(result.rewrittenManifests()).as("Action should rewrite 2 manifests").hasSize(2); + assertThat(result.addedManifests()).as("Action should add 1 manifests").hasSize(1); + assertManifestsLocation(result.addedManifests()); + + table.refresh(); + + List newManifests = table.currentSnapshot().allManifests(table.io()); + assertThat(newManifests).as("Should have 1 manifests after rewrite").hasSize(1); + + ManifestFile newManifest = Iterables.getOnlyElement(newManifests); + assertThat(newManifest.existingFilesCount()).isEqualTo(2); + assertThat(newManifest.hasAddedFiles()).isFalse(); + assertThat(newManifest.hasDeletedFiles()).isFalse(); + + validateDataManifest( + table, + newManifest, + dataSeqs(1L, 2L), + fileSeqs(1L, 2L), + snapshotIds(snapshot1.snapshotId(), snapshot2.snapshotId()), + files(file1, file2)); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(records1); + expectedRecords.addAll(records2); + + Dataset resultDF = spark.read().format("iceberg").load(tableLocation); + List actualRecords = + resultDF.sort("c1", "c2").as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + + assertThat(actualRecords).as("Rows must match").isEqualTo(expectedRecords); + } + + @TestTemplate + public void testRewriteLargeManifestsEvolvedUnpartitionedV1Table() throws IOException { + assumeThat(formatVersion).isEqualTo(1); + + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c3").build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + table.updateSpec().removeField("c3").commit(); + + assertThat(table.spec().fields()).hasSize(1).allMatch(field -> field.transform().isVoid()); + + List dataFiles = Lists.newArrayList(); + for (int fileOrdinal = 0; fileOrdinal < 1000; fileOrdinal++) { + dataFiles.add(newDataFile(table, TestHelpers.Row.of(new Object[] {null}))); + } + ManifestFile appendManifest = writeManifest(table, dataFiles); + table.newFastAppend().appendManifest(appendManifest).commit(); + + List originalManifests = table.currentSnapshot().allManifests(table.io()); + ManifestFile originalManifest = Iterables.getOnlyElement(originalManifests); + + // set the target manifest size to a small value to force splitting records into multiple files + table + .updateProperties() + .set( + TableProperties.MANIFEST_TARGET_SIZE_BYTES, + String.valueOf(originalManifest.length() / 2)) + .commit(); + + SparkActions actions = SparkActions.get(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + assertThat(result.rewrittenManifests()).hasSize(1); + assertThat(result.addedManifests()).hasSizeGreaterThanOrEqualTo(2); + assertManifestsLocation(result.addedManifests()); + + List manifests = table.currentSnapshot().allManifests(table.io()); + assertThat(manifests).hasSizeGreaterThanOrEqualTo(2); + } + + @TestTemplate + public void testRewriteSmallDeleteManifestsNonPartitionedTable() throws IOException { + assumeThat(formatVersion).isEqualTo(2); + + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + // commit data records + List records = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "BBBB"), + new ThreeColumnRecord(3, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(4, "DDDDDDDDDD", "DDDD")); + writeRecords(records); + + // commit a position delete file to remove records where c1 = 1 OR c1 = 2 + List> posDeletes = generatePosDeletes("c1 = 1 OR c1 = 2"); + Pair posDeleteWriteResult = writePosDeletes(table, posDeletes); + table + .newRowDelta() + .addDeletes(posDeleteWriteResult.first()) + .validateDataFilesExist(posDeleteWriteResult.second()) + .commit(); + + // commit an equality delete file to remove all records where c1 = 3 + DeleteFile eqDeleteFile = writeEqDeletes(table, "c1", 3); + table.newRowDelta().addDeletes(eqDeleteFile).commit(); + + // the current snapshot should contain 1 data manifest and 2 delete manifests + List originalManifests = table.currentSnapshot().allManifests(table.io()); + assertThat(originalManifests).hasSize(3); + + SparkActions actions = SparkActions.get(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + // the original delete manifests must be combined + assertThat(result.rewrittenManifests()) + .hasSize(2) + .allMatch(m -> m.content() == ManifestContent.DELETES); + assertThat(result.addedManifests()) + .hasSize(1) + .allMatch(m -> m.content() == ManifestContent.DELETES); + assertManifestsLocation(result.addedManifests()); + + // the new delete manifest must only contain files with status EXISTING + ManifestFile deleteManifest = + Iterables.getOnlyElement(table.currentSnapshot().deleteManifests(table.io())); + assertThat(deleteManifest.existingFilesCount()).isEqualTo(2); + assertThat(deleteManifest.hasAddedFiles()).isFalse(); + assertThat(deleteManifest.hasDeletedFiles()).isFalse(); + + // the preserved data manifest must only contain files with status ADDED + ManifestFile dataManifest = + Iterables.getOnlyElement(table.currentSnapshot().dataManifests(table.io())); + assertThat(dataManifest.hasExistingFiles()).isFalse(); + assertThat(dataManifest.hasAddedFiles()).isTrue(); + assertThat(dataManifest.hasDeletedFiles()).isFalse(); + + // the table must produce expected records after the rewrite + List expectedRecords = + Lists.newArrayList(new ThreeColumnRecord(4, "DDDDDDDDDD", "DDDD")); + assertThat(actualRecords()).isEqualTo(expectedRecords); + } + + @TestTemplate + public void testRewriteSmallDeleteManifestsPartitionedTable() throws IOException { + assumeThat(formatVersion).isEqualTo(2); + + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c3").build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + options.put(TableProperties.MANIFEST_MERGE_ENABLED, "false"); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + // commit data records + List records = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "BBBB"), + new ThreeColumnRecord(3, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(4, "DDDDDDDDDD", "DDDD"), + new ThreeColumnRecord(5, "EEEEEEEEEE", "EEEE")); + writeRecords(records); + + // commit the first position delete file to remove records where c1 = 1 + List> posDeletes1 = generatePosDeletes("c1 = 1"); + Pair posDeleteWriteResult1 = + writePosDeletes(table, TestHelpers.Row.of("AAAA"), posDeletes1); + table + .newRowDelta() + .addDeletes(posDeleteWriteResult1.first()) + .validateDataFilesExist(posDeleteWriteResult1.second()) + .commit(); + + // commit the second position delete file to remove records where c1 = 2 + List> posDeletes2 = generatePosDeletes("c1 = 2"); + Pair positionDeleteWriteResult2 = + writePosDeletes(table, TestHelpers.Row.of("BBBB"), posDeletes2); + table + .newRowDelta() + .addDeletes(positionDeleteWriteResult2.first()) + .validateDataFilesExist(positionDeleteWriteResult2.second()) + .commit(); + + // commit the first equality delete file to remove records where c1 = 3 + DeleteFile eqDeleteFile1 = writeEqDeletes(table, TestHelpers.Row.of("CCCC"), "c1", 3); + table.newRowDelta().addDeletes(eqDeleteFile1).commit(); + + // commit the second equality delete file to remove records where c1 = 4 + DeleteFile eqDeleteFile2 = writeEqDeletes(table, TestHelpers.Row.of("DDDD"), "c1", 4); + table.newRowDelta().addDeletes(eqDeleteFile2).commit(); + + // the table must have 1 data manifest and 4 delete manifests + List originalManifests = table.currentSnapshot().allManifests(table.io()); + assertThat(originalManifests).hasSize(5); + + // set the target manifest size to have 2 manifests with 2 entries in each after the rewrite + List originalDeleteManifests = + table.currentSnapshot().deleteManifests(table.io()); + long manifestEntrySizeBytes = computeManifestEntrySizeBytes(originalDeleteManifests); + long targetManifestSizeBytes = (long) (1.05 * 2 * manifestEntrySizeBytes); + + table + .updateProperties() + .set(TableProperties.MANIFEST_TARGET_SIZE_BYTES, String.valueOf(targetManifestSizeBytes)) + .commit(); + + SparkActions actions = SparkActions.get(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> manifest.content() == ManifestContent.DELETES) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + // the original 4 delete manifests must be replaced with 2 new delete manifests + assertThat(result.rewrittenManifests()) + .hasSize(4) + .allMatch(m -> m.content() == ManifestContent.DELETES); + assertThat(result.addedManifests()) + .hasSize(2) + .allMatch(m -> m.content() == ManifestContent.DELETES); + assertManifestsLocation(result.addedManifests()); + + List deleteManifests = table.currentSnapshot().deleteManifests(table.io()); + assertThat(deleteManifests).hasSize(2); + + // the first new delete manifest must only contain files with status EXISTING + ManifestFile deleteManifest1 = deleteManifests.get(0); + assertThat(deleteManifest1.existingFilesCount()).isEqualTo(2); + assertThat(deleteManifest1.hasAddedFiles()).isFalse(); + assertThat(deleteManifest1.hasDeletedFiles()).isFalse(); + + // the second new delete manifest must only contain files with status EXISTING + ManifestFile deleteManifest2 = deleteManifests.get(1); + assertThat(deleteManifest2.existingFilesCount()).isEqualTo(2); + assertThat(deleteManifest2.hasAddedFiles()).isFalse(); + assertThat(deleteManifest2.hasDeletedFiles()).isFalse(); + + // the table must produce expected records after the rewrite + List expectedRecords = + Lists.newArrayList(new ThreeColumnRecord(5, "EEEEEEEEEE", "EEEE")); + assertThat(actualRecords()).isEqualTo(expectedRecords); + } + + @TestTemplate + public void testRewriteLargeDeleteManifestsPartitionedTable() throws IOException { + assumeThat(formatVersion).isEqualTo(2); + + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c3").build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + // generate enough delete files to have a reasonably sized manifest + List deleteFiles = Lists.newArrayList(); + for (int fileOrdinal = 0; fileOrdinal < 1000; fileOrdinal++) { + DeleteFile deleteFile = newDeleteFile(table, "c3=" + fileOrdinal); + deleteFiles.add(deleteFile); + } + + // commit delete files + RowDelta rowDelta = table.newRowDelta(); + for (DeleteFile deleteFile : deleteFiles) { + rowDelta.addDeletes(deleteFile); + } + rowDelta.commit(); + + // the current snapshot should contain only 1 delete manifest + List originalDeleteManifests = + table.currentSnapshot().deleteManifests(table.io()); + ManifestFile originalDeleteManifest = Iterables.getOnlyElement(originalDeleteManifests); + + // set the target manifest size to a small value to force splitting records into multiple files + table + .updateProperties() + .set( + TableProperties.MANIFEST_TARGET_SIZE_BYTES, + String.valueOf(originalDeleteManifest.length() / 2)) + .commit(); + + SparkActions actions = SparkActions.get(); + + String stagingLocation = java.nio.file.Files.createTempDirectory(temp, "junit").toString(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .stagingLocation(stagingLocation) + .execute(); + + // the action must rewrite the original delete manifest and add at least 2 new ones + assertThat(result.rewrittenManifests()) + .hasSize(1) + .allMatch(m -> m.content() == ManifestContent.DELETES); + assertThat(result.addedManifests()) + .hasSizeGreaterThanOrEqualTo(2) + .allMatch(m -> m.content() == ManifestContent.DELETES); + assertManifestsLocation(result.addedManifests(), stagingLocation); + + // the current snapshot must return the correct number of delete manifests + List deleteManifests = table.currentSnapshot().deleteManifests(table.io()); + assertThat(deleteManifests).hasSizeGreaterThanOrEqualTo(2); + } + + @TestTemplate + public void testRewriteManifestsAfterUpgradeToV3() throws IOException { + assumeThat(formatVersion).isEqualTo(2); + + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + Map options = ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + DataFile dataFile1 = newDataFile(table, "c1=1"); + DeleteFile deleteFile1 = newDeletes(table, dataFile1); + table.newRowDelta().addRows(dataFile1).addDeletes(deleteFile1).commit(); + + DataFile dataFile2 = newDataFile(table, "c1=1"); + DeleteFile deleteFile2 = newDeletes(table, dataFile2); + table.newRowDelta().addRows(dataFile2).addDeletes(deleteFile2).commit(); + + // upgrade the table to enable DVs + table.updateProperties().set(TableProperties.FORMAT_VERSION, "3").commit(); + + DataFile dataFile3 = newDataFile(table, "c1=1"); + DeleteFile dv3 = newDV(table, dataFile3); + table.newRowDelta().addRows(dataFile3).addDeletes(dv3).commit(); + + SparkActions actions = SparkActions.get(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + assertThat(result.rewrittenManifests()).as("Action should rewrite 6 manifests").hasSize(6); + assertThat(result.addedManifests()).as("Action should add 2 manifests").hasSize(2); + assertManifestsLocation(result.addedManifests()); + + table.refresh(); + + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (FileScanTask fileTask : tasks) { + DataFile dataFile = fileTask.file(); + DeleteFile deleteFile = Iterables.getOnlyElement(fileTask.deletes()); + if (dataFile.location().equals(dataFile1.location())) { + assertThat(deleteFile.referencedDataFile()).isEqualTo(deleteFile1.referencedDataFile()); + assertEqual(deleteFile, deleteFile1); + } else if (dataFile.location().equals(dataFile2.location())) { + assertThat(deleteFile.referencedDataFile()).isEqualTo(deleteFile2.referencedDataFile()); + assertEqual(deleteFile, deleteFile2); + } else { + assertThat(deleteFile.referencedDataFile()).isEqualTo(dv3.referencedDataFile()); + assertEqual(deleteFile, dv3); + } + } + } + } + + private List actualRecords() { + return spark + .read() + .format("iceberg") + .load(tableLocation) + .as(Encoders.bean(ThreeColumnRecord.class)) + .sort("c1", "c2", "c3") + .collectAsList(); + } + + private void writeRecords(List records) { + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); + writeDF(df); + } + + private void writeDF(Dataset df) { + df.select("c1", "c2", "c3") + .write() + .format("iceberg") + .option(SparkWriteOptions.DISTRIBUTION_MODE, TableProperties.WRITE_DISTRIBUTION_MODE_NONE) + .mode("append") + .save(tableLocation); + } + + private long computeManifestEntrySizeBytes(List manifests) { + long totalSize = 0L; + int numEntries = 0; + + for (ManifestFile manifest : manifests) { + totalSize += manifest.length(); + numEntries += + manifest.addedFilesCount() + manifest.existingFilesCount() + manifest.deletedFilesCount(); + } + + return totalSize / numEntries; + } + + private void assertManifestsLocation(Iterable manifests) { + assertManifestsLocation(manifests, null); + } + + private void assertManifestsLocation(Iterable manifests, String stagingLocation) { + if (shouldStageManifests && stagingLocation != null) { + assertThat(manifests).allMatch(manifest -> manifest.path().startsWith(stagingLocation)); + } else { + assertThat(manifests).allMatch(manifest -> manifest.path().startsWith(tableLocation)); + } + } + + private ManifestFile writeManifest(Table table, List files) throws IOException { + File manifestFile = File.createTempFile("generated-manifest", ".avro", temp.toFile()); + assertThat(manifestFile.delete()).isTrue(); + OutputFile outputFile = table.io().newOutputFile(manifestFile.getCanonicalPath()); + + ManifestWriter writer = + ManifestFiles.write(formatVersion, table.spec(), outputFile, null); + + try { + for (DataFile file : files) { + writer.add(file); + } + } finally { + writer.close(); + } + + return writer.toManifestFile(); + } + + private DataFile newDataFile(Table table, String partitionPath) { + return newDataFileBuilder(table).withPartitionPath(partitionPath).build(); + } + + private DataFile newDataFile(Table table, StructLike partition) { + return newDataFileBuilder(table).withPartition(partition).build(); + } + + private DataFiles.Builder newDataFileBuilder(Table table) { + return DataFiles.builder(table.spec()) + .withPath("/path/to/data-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(10) + .withRecordCount(1); + } + + private DeleteFile newDeletes(Table table, DataFile dataFile) { + return formatVersion >= 3 ? newDV(table, dataFile) : newDeleteFileWithRef(table, dataFile); + } + + private DeleteFile newDeleteFileWithRef(Table table, DataFile dataFile) { + return FileGenerationUtil.generatePositionDeleteFileWithRef(table, dataFile); + } + + private DeleteFile newDV(Table table, DataFile dataFile) { + return FileGenerationUtil.generateDV(table, dataFile); + } + + private DeleteFile newDeleteFile(Table table, String partitionPath) { + return formatVersion >= 3 + ? FileMetadata.deleteFileBuilder(table.spec()) + .ofPositionDeletes() + .withPath("/path/to/pos-deletes-" + UUID.randomUUID() + ".puffin") + .withFileSizeInBytes(5) + .withPartitionPath(partitionPath) + .withRecordCount(1) + .withContentOffset(ThreadLocalRandom.current().nextInt()) + .withContentSizeInBytes(ThreadLocalRandom.current().nextInt()) + .build() + : FileMetadata.deleteFileBuilder(table.spec()) + .ofPositionDeletes() + .withPath("/path/to/pos-deletes-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(5) + .withPartitionPath(partitionPath) + .withRecordCount(1) + .build(); + } + + private List> generatePosDeletes(String predicate) { + List rows = + spark + .read() + .format("iceberg") + .load(tableLocation) + .selectExpr("_file", "_pos") + .where(predicate) + .collectAsList(); + + List> deletes = Lists.newArrayList(); + + for (Row row : rows) { + deletes.add(Pair.of(row.getString(0), row.getLong(1))); + } + + return deletes; + } + + private Pair writePosDeletes( + Table table, List> deletes) throws IOException { + return writePosDeletes(table, null, deletes); + } + + private Pair writePosDeletes( + Table table, StructLike partition, List> deletes) + throws IOException { + OutputFile outputFile = Files.localOutput(File.createTempFile("junit", null, temp.toFile())); + return FileHelpers.writeDeleteFile(table, outputFile, partition, deletes, formatVersion); + } + + private DeleteFile writeEqDeletes(Table table, String key, Object... values) throws IOException { + return writeEqDeletes(table, null, key, values); + } + + private DeleteFile writeEqDeletes(Table table, StructLike partition, String key, Object... values) + throws IOException { + List deletes = Lists.newArrayList(); + Schema deleteSchema = table.schema().select(key); + Record delete = GenericRecord.create(deleteSchema); + + for (Object value : values) { + deletes.add(delete.copy(key, value)); + } + + OutputFile outputFile = Files.localOutput(File.createTempFile("junit", null, temp.toFile())); + return FileHelpers.writeDeleteFile(table, outputFile, partition, deletes, deleteSchema); + } + + private void assertDeletes(DataFile dataFile, DeleteFile deleteFile) { + assertThat(deleteFile.referencedDataFile()).isEqualTo(dataFile.location()); + if (formatVersion >= 3) { + assertThat(deleteFile.contentOffset()).isNotNull(); + assertThat(deleteFile.contentSizeInBytes()).isNotNull(); + } else { + assertThat(deleteFile.contentOffset()).isNull(); + assertThat(deleteFile.contentSizeInBytes()).isNull(); + } + } + + private void assertEqual(DeleteFile deleteFile1, DeleteFile deleteFile2) { + assertThat(deleteFile1.location()).isEqualTo(deleteFile2.location()); + assertThat(deleteFile1.content()).isEqualTo(deleteFile2.content()); + assertThat(deleteFile1.specId()).isEqualTo(deleteFile2.specId()); + assertThat(deleteFile1.partition()).isEqualTo(deleteFile2.partition()); + assertThat(deleteFile1.format()).isEqualTo(deleteFile2.format()); + assertThat(deleteFile1.referencedDataFile()).isEqualTo(deleteFile2.referencedDataFile()); + assertThat(deleteFile1.contentOffset()).isEqualTo(deleteFile2.contentOffset()); + assertThat(deleteFile1.contentSizeInBytes()).isEqualTo(deleteFile2.contentSizeInBytes()); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java new file mode 100644 index 000000000000..829ac761c876 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewritePositionDeleteFilesAction.java @@ -0,0 +1,1307 @@ +/* + * 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.actions; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.expr; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.fail; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.BiFunction; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.BinPackRewriteFilePlanner; +import org.apache.iceberg.actions.RewriteDataFiles; +import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupRewriteResult; +import org.apache.iceberg.actions.RewritePositionDeleteFiles.Result; +import org.apache.iceberg.actions.SizeBasedFileRewritePlanner; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +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.collect.Lists; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.spark.source.FourColumnRecord; +import org.apache.iceberg.spark.source.ThreeColumnRecord; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.StructLikeMap; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewritePositionDeleteFilesAction extends CatalogTestBase { + + private static final String TABLE_NAME = "test_table"; + private static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + + private static final Map CATALOG_PROPS = + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "cache-enabled", "false"); + + private static final int SCALE = 4000; + private static final int DELETES_SCALE = 1000; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, fileFormat = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + CATALOG_PROPS, + FileFormat.PARQUET + } + }; + } + + @Parameter(index = 3) + private FileFormat format; + + @AfterEach + public void cleanup() { + validationCatalog.dropTable(TableIdentifier.of("default", TABLE_NAME)); + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testEmptyTable() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + Table table = + validationCatalog.createTable( + TableIdentifier.of("default", TABLE_NAME), SCHEMA, spec, tableProperties()); + + Result result = SparkActions.get(spark).rewritePositionDeletes(table).execute(); + assertThat(result.rewrittenDeleteFilesCount()).as("No rewritten delete files").isZero(); + assertThat(result.addedDeleteFilesCount()).as("No added delete files").isZero(); + } + + @TestTemplate + public void testFileGranularity() throws Exception { + checkDeleteGranularity(DeleteGranularity.FILE); + } + + @TestTemplate + public void testPartitionGranularity() throws Exception { + checkDeleteGranularity(DeleteGranularity.PARTITION); + } + + private void checkDeleteGranularity(DeleteGranularity deleteGranularity) throws Exception { + Table table = createTableUnpartitioned(2, SCALE); + + table + .updateProperties() + .set(TableProperties.DELETE_GRANULARITY, deleteGranularity.toString()) + .commit(); + + List dataFiles = TestHelpers.dataFiles(table); + assertThat(dataFiles).hasSize(2); + + writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(2); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + + int expectedDeleteFilesCount = deleteGranularity == DeleteGranularity.FILE ? 2 : 1; + assertThat(result.addedDeleteFilesCount()).isEqualTo(expectedDeleteFilesCount); + } + + @TestTemplate + public void testUnpartitioned() throws Exception { + Table table = createTableUnpartitioned(2, SCALE); + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); + assertThat(dataFiles).hasSize(2); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(2); + + List expectedRecords = records(table); + List expectedDeletes = deleteRecords(table); + assertThat(expectedRecords).hasSize(2000); + assertThat(expectedDeletes).hasSize(2000); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).as("New delete files").hasSize(1); + assertLocallySorted(newDeleteFiles); + assertNotContains(deleteFiles, newDeleteFiles); + checkResult(result, deleteFiles, newDeleteFiles, 1); + checkSequenceNumbers(table, deleteFiles, newDeleteFiles); + + List actualRecords = records(table); + List actualDeletes = deleteRecords(table); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + } + + @TestTemplate + public void testRewriteAll() throws Exception { + Table table = createTablePartitioned(4, 2, SCALE); + + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); + assertThat(dataFiles).hasSize(4); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(8); + + List expectedRecords = records(table); + List expectedDeletes = deleteRecords(table); + assertThat(expectedRecords).hasSize(12000); + assertThat(expectedDeletes).hasSize(4000); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, + Long.toString(Long.MAX_VALUE - 1)) + .execute(); + + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).hasSize(4); + assertNotContains(deleteFiles, newDeleteFiles); + assertLocallySorted(newDeleteFiles); + checkResult(result, deleteFiles, newDeleteFiles, 4); + checkSequenceNumbers(table, deleteFiles, newDeleteFiles); + + List actualRecords = records(table); + List actualDeletes = deleteRecords(table); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + } + + @TestTemplate + public void testRewriteFilter() throws Exception { + Table table = createTablePartitioned(4, 2, SCALE); + table.refresh(); + + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); + assertThat(dataFiles).hasSize(4); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(8); + + table.refresh(); + List expectedRecords = records(table); + List expectedDeletes = deleteRecords(table); + assertThat(expectedRecords).hasSize(12000); + assertThat(expectedDeletes).hasSize(4000); + + Expression filter = + Expressions.and( + Expressions.greaterThan("c3", "0"), // should have no effect + // "C1" should work because Spark defaults case sensitivity to false. + Expressions.or(Expressions.equal("C1", 1), Expressions.equal("C1", 2))); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .filter(filter) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, + Long.toString(Long.MAX_VALUE - 1)) + .execute(); + + List newDeleteFiles = except(deleteFiles(table), deleteFiles); + assertThat(newDeleteFiles).as("Delete files").hasSize(2); + + List expectedRewrittenFiles = + filterFiles(table, deleteFiles, ImmutableList.of(1), ImmutableList.of(2)); + assertLocallySorted(newDeleteFiles); + checkResult(result, expectedRewrittenFiles, newDeleteFiles, 2); + + List actualRecords = records(table); + List actualDeletes = deleteRecords(table); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + + withSQLConf( + ImmutableMap.of(SQLConf.CASE_SENSITIVE().key(), "true"), + () -> { + assertThatThrownBy( + () -> + SparkActions.get(spark) + .rewritePositionDeletes(table) + .filter(filter) + .execute()) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot find field 'C1' in struct"); + }); + } + + @TestTemplate + public void testRewriteToSmallerTarget() throws Exception { + Table table = createTablePartitioned(4, 2, SCALE); + + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); + assertThat(dataFiles).hasSize(4); + + List expectedRecords = records(table); + List expectedDeletes = deleteRecords(table); + assertThat(expectedRecords).hasSize(12000); + assertThat(expectedDeletes).hasSize(4000); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(8); + + long avgSize = size(deleteFiles) / deleteFiles.size(); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option(SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, String.valueOf(avgSize / 2)) + .execute(); + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).as("New delete files").hasSize(8); + assertNotContains(deleteFiles, newDeleteFiles); + assertLocallySorted(newDeleteFiles); + checkResult(result, deleteFiles, newDeleteFiles, 4); + checkSequenceNumbers(table, deleteFiles, newDeleteFiles); + + List actualRecords = records(table); + List actualDeletes = deleteRecords(table); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + } + + @TestTemplate + public void testRemoveDanglingDeletes() throws Exception { + Table table = createTablePartitioned(4, 2, SCALE); + + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles( + table, + 2, + DELETES_SCALE, + dataFiles, + true /* Disable commit-time ManifestFilterManager removal of dangling deletes */); + + assertThat(dataFiles).hasSize(4); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(8); + + List expectedRecords = records(table); + List expectedDeletes = deleteRecords(table); + assertThat(expectedRecords).hasSize(12000); + assertThat(expectedDeletes).hasSize(4000); + + SparkActions.get(spark) + .rewriteDataFiles(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).as("New delete files").isEmpty(); + assertNotContains(deleteFiles, newDeleteFiles); + assertLocallySorted(newDeleteFiles); + checkResult(result, deleteFiles, newDeleteFiles, 4); + checkSequenceNumbers(table, deleteFiles, newDeleteFiles); + + List actualRecords = records(table); + List actualDeletes = deleteRecords(table); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertThat(actualDeletes).as("New position deletes").isEmpty(); + } + + @TestTemplate + public void testRemoveDanglingDVsAfterCompaction() { + sql( + "create table %s (s string, id string) PARTITIONED BY (bucket(8, id)) " + + "tblproperties ('format-version'='3'," + + "'write.update.mode'='merge-on-read'," + + "'write.delete.mode'='merge-on-read'," + + "'write.merge.mode'='merge-on-read')", + tableName); + sql("insert into %s select * from (values ('foo', '1'), ('bar', '1')) order by 1", tableName); + sql("insert into %s select * from (values ('foo', '1'), ('bat', '1')) order by 1", tableName); + sql("insert into %s select * from (values ('bar', '1'), ('bat', '1')) order by 1", tableName); + + List objects = sql("select * from %s.files", tableName); + assertThat(objects).hasSize(3); + + sql("delete from %s where s = 'foo'", tableName); + assertThat(sql("select * from %s.files", tableName)).hasSize(5); + + sql("delete from %s where s = 'bar'", tableName); + assertThat(sql("select * from %s.files", tableName)).hasSize(6); + + assertThat(sql("select * from %s.data_files", tableName)).hasSize(3); + assertThat(sql("select * from %s.delete_files", tableName)).hasSize(3); + + Set deleteFilesBefore = + TestHelpers.deleteFiles(validationCatalog.loadTable(tableIdent)); + assertThat(deleteFilesBefore).hasSize(3); + + RewriteDataFiles.Result result = + SparkActions.get(spark) + .rewriteDataFiles(validationCatalog.loadTable(tableIdent)) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") + .execute(); + + Set deleteFilesAfter = + TestHelpers.deleteFiles(validationCatalog.loadTable(tableIdent)); + assertThat(deleteFilesAfter).isEmpty(); + assertThat(result.addedDataFilesCount()).isEqualTo(1); + assertThat(result.rewrittenDataFilesCount()).isEqualTo(3); + assertThat(sql("select * from %s.delete_files", tableName)).hasSameSizeAs(deleteFilesAfter); + assertThat(result.removedDeleteFilesCount()).isEqualTo(deleteFilesBefore.size()); + } + + @TestTemplate + public void testValidDVsAreNotRemovedDuringDanglingDeletesRemoval() { + sql( + "create table %s (s string, id string) PARTITIONED BY (bucket(8, id)) " + + "tblproperties ('format-version'='3'," + + "'write.update.mode'='merge-on-read'," + + "'write.delete.mode'='merge-on-read'," + + "'write.merge.mode'='merge-on-read')", + tableName); + sql("insert into %s select * from (values ('foo', '1'), ('bar', '1')) order by 1", tableName); + sql("insert into %s select * from (values ('foo', '1'), ('bat', '1')) order by 1", tableName); + sql("insert into %s select * from (values ('bar', '1'), ('bat', '1')) order by 1", tableName); + + List objects = sql("select * from %s.files", tableName); + assertThat(objects).hasSize(3); + + sql("delete from %s where s = 'foo'", tableName); + assertThat(sql("select * from %s.files", tableName)).hasSize(5); + + assertThat(sql("select * from %s.data_files", tableName)).hasSize(3); + assertThat(sql("select * from %s.delete_files", tableName)).hasSize(2); + + Set deleteFilesBefore = + TestHelpers.deleteFiles(validationCatalog.loadTable(tableIdent)); + assertThat(deleteFilesBefore).hasSize(2); + + // data files are not compacted and removing dangling deletes should not remove valid DVs + RewriteDataFiles.Result result = + SparkActions.get(spark) + .rewriteDataFiles(validationCatalog.loadTable(tableIdent)) + .option(RewriteDataFiles.REMOVE_DANGLING_DELETES, "true") + .option(BinPackRewriteFilePlanner.MIN_FILE_SIZE_BYTES, "0") + .option(BinPackRewriteFilePlanner.DELETE_RATIO_THRESHOLD, "1.0") + .execute(); + + Set deleteFilesAfter = + TestHelpers.deleteFiles(validationCatalog.loadTable(tableIdent)); + assertThat(deleteFilesAfter).isEqualTo(deleteFilesBefore); + assertThat(result.addedDataFilesCount()).isEqualTo(0); + assertThat(result.rewrittenDataFilesCount()).isEqualTo(0); + assertThat(sql("select * from %s.delete_files", tableName)).hasSameSizeAs(deleteFilesAfter); + assertThat(result.removedDeleteFilesCount()).isEqualTo(0); + } + + @TestTemplate + public void testSomePartitionsDanglingDeletes() throws Exception { + Table table = createTablePartitioned(4, 2, SCALE); + + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); + assertThat(dataFiles).hasSize(4); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(8); + + List expectedRecords = records(table); + List expectedDeletes = deleteRecords(table); + assertThat(expectedRecords).hasSize(12000); + assertThat(expectedDeletes).hasSize(4000); + + // Rewrite half the data files + Expression filter = Expressions.or(Expressions.equal("c1", 0), Expressions.equal("c1", 1)); + SparkActions.get(spark) + .rewriteDataFiles(table) + .filter(filter) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).as("New delete files").hasSize(2); + assertNotContains(deleteFiles, newDeleteFiles); + assertLocallySorted(newDeleteFiles); + checkResult(result, deleteFiles, newDeleteFiles, 4); + checkSequenceNumbers(table, deleteFiles, newDeleteFiles); + + // As only half the files have been rewritten, + // we expect to retain position deletes only for those not rewritten + expectedDeletes = + expectedDeletes.stream() + .filter( + r -> { + Object[] partition = (Object[]) r[3]; + return partition[0] == (Integer) 2 || partition[0] == (Integer) 3; + }) + .collect(Collectors.toList()); + + List actualRecords = records(table); + List actualDeletes = deleteRecords(table); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + } + + @TestTemplate + public void testRewriteFilterRemoveDangling() throws Exception { + Table table = createTablePartitioned(4, 2, SCALE); + table.refresh(); + + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles, true); + assertThat(dataFiles).hasSize(4); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(8); + + table.refresh(); + List expectedRecords = records(table); + List expectedDeletes = deleteRecords(table); + assertThat(expectedRecords).hasSize(12000); // 16000 data - 4000 delete rows + assertThat(expectedDeletes).hasSize(4000); + + SparkActions.get(spark) + .rewriteDataFiles(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + + Expression filter = Expressions.or(Expressions.equal("c1", 0), Expressions.equal("c1", 1)); + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .filter(filter) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, + Long.toString(Long.MAX_VALUE - 1)) + .execute(); + + List newDeleteFiles = except(deleteFiles(table), deleteFiles); + assertThat(newDeleteFiles).as("New delete files").isEmpty(); + + List expectedRewrittenFiles = + filterFiles(table, deleteFiles, ImmutableList.of(0), ImmutableList.of(1)); + checkResult(result, expectedRewrittenFiles, newDeleteFiles, 2); + + List actualRecords = records(table); + List allDeletes = deleteRecords(table); + // Only non-compacted deletes remain + List expectedDeletesFiltered = + filterDeletes(expectedDeletes, ImmutableList.of(2), ImmutableList.of(3)); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertEquals("Position deletes must match", expectedDeletesFiltered, allDeletes); + } + + @TestTemplate + public void testPartitionEvolutionAdd() throws Exception { + Table table = createTableUnpartitioned(2, SCALE); + List unpartitionedDataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 2, DELETES_SCALE, unpartitionedDataFiles); + assertThat(unpartitionedDataFiles).hasSize(2); + + List unpartitionedDeleteFiles = deleteFiles(table); + assertThat(unpartitionedDeleteFiles).hasSize(2); + + List expectedUnpartitionedDeletes = deleteRecords(table); + List expectedUnpartitionedRecords = records(table); + assertThat(expectedUnpartitionedRecords).hasSize(2000); + assertThat(expectedUnpartitionedDeletes).hasSize(2000); + + table.updateSpec().addField("c1").commit(); + writeRecords(table, 2, SCALE, 2); + List partitionedDataFiles = + except(TestHelpers.dataFiles(table), unpartitionedDataFiles); + writePosDeletesForFiles(table, 2, DELETES_SCALE, partitionedDataFiles); + assertThat(partitionedDataFiles).hasSize(2); + + List partitionedDeleteFiles = except(deleteFiles(table), unpartitionedDeleteFiles); + assertThat(partitionedDeleteFiles).hasSize(4); + + List expectedDeletes = deleteRecords(table); + List expectedRecords = records(table); + assertThat(expectedDeletes).hasSize(4000); + assertThat(expectedRecords).hasSize(8000); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + + List rewrittenDeleteFiles = + Stream.concat(unpartitionedDeleteFiles.stream(), partitionedDeleteFiles.stream()) + .collect(Collectors.toList()); + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).as("New delete files").hasSize(3); + assertNotContains(rewrittenDeleteFiles, newDeleteFiles); + assertLocallySorted(newDeleteFiles); + checkResult(result, rewrittenDeleteFiles, newDeleteFiles, 3); + checkSequenceNumbers(table, rewrittenDeleteFiles, newDeleteFiles); + + List actualRecords = records(table); + List actualDeletes = deleteRecords(table); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + } + + @TestTemplate + public void testPartitionEvolutionRemove() throws Exception { + Table table = createTablePartitioned(2, 2, SCALE); + List dataFilesUnpartitioned = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFilesUnpartitioned); + assertThat(dataFilesUnpartitioned).hasSize(2); + + List deleteFilesUnpartitioned = deleteFiles(table); + assertThat(deleteFilesUnpartitioned).hasSize(4); + + table.updateSpec().removeField("c1").commit(); + + writeRecords(table, 2, SCALE); + List dataFilesPartitioned = + except(TestHelpers.dataFiles(table), dataFilesUnpartitioned); + writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFilesPartitioned); + assertThat(dataFilesPartitioned).hasSize(2); + + List deleteFilesPartitioned = except(deleteFiles(table), deleteFilesUnpartitioned); + assertThat(deleteFilesPartitioned).hasSize(2); + + List expectedRecords = records(table); + List expectedDeletes = deleteRecords(table); + assertThat(expectedDeletes).hasSize(4000); + assertThat(expectedRecords).hasSize(8000); + + List expectedRewritten = deleteFiles(table); + assertThat(expectedRewritten).hasSize(6); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).as("New delete files").hasSize(3); + assertNotContains(expectedRewritten, newDeleteFiles); + assertLocallySorted(newDeleteFiles); + checkResult(result, expectedRewritten, newDeleteFiles, 3); + checkSequenceNumbers(table, expectedRewritten, newDeleteFiles); + + List actualRecords = records(table); + List actualDeletes = deleteRecords(table); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + } + + @TestTemplate + public void testSchemaEvolution() throws Exception { + Table table = createTablePartitioned(2, 2, SCALE); + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); + assertThat(dataFiles).hasSize(2); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(4); + + table.updateSchema().addColumn("c4", Types.StringType.get()).commit(); + writeNewSchemaRecords(table, 2, SCALE, 2, 2); + + int newColId = table.schema().findField("c4").fieldId(); + List newSchemaDataFiles = + TestHelpers.dataFiles(table).stream() + .filter(f -> f.upperBounds().containsKey(newColId)) + .collect(Collectors.toList()); + writePosDeletesForFiles(table, 2, DELETES_SCALE, newSchemaDataFiles); + + List newSchemaDeleteFiles = except(deleteFiles(table), deleteFiles); + assertThat(newSchemaDeleteFiles).hasSize(4); + + table.refresh(); + List expectedDeletes = deleteRecords(table); + List expectedRecords = records(table); + assertThat(expectedDeletes).hasSize(4000); // 4 files * 1000 per file + assertThat(expectedRecords).hasSize(12000); // 4 * 4000 - 4000 + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + + List rewrittenDeleteFiles = + Stream.concat(deleteFiles.stream(), newSchemaDeleteFiles.stream()) + .collect(Collectors.toList()); + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).as("New delete files").hasSize(4); + assertNotContains(rewrittenDeleteFiles, newDeleteFiles); + assertLocallySorted(newDeleteFiles); + checkResult(result, rewrittenDeleteFiles, newDeleteFiles, 4); + checkSequenceNumbers(table, rewrittenDeleteFiles, newDeleteFiles); + + List actualRecords = records(table); + assertEquals("Rows must match", expectedRecords, actualRecords); + } + + @TestTemplate + public void testSnapshotProperty() throws Exception { + Table table = createTableUnpartitioned(2, SCALE); + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); + assertThat(dataFiles).hasSize(2); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(2); + + Result ignored = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .snapshotProperty("key", "value") + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + assertThat(table.currentSnapshot().summary()) + .containsAllEntriesOf(ImmutableMap.of("key", "value")); + + // make sure internal produced properties are not lost + String[] commitMetricsKeys = + new String[] { + SnapshotSummary.ADDED_DELETE_FILES_PROP, + SnapshotSummary.ADDED_POS_DELETES_PROP, + SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, + SnapshotSummary.REMOVED_DELETE_FILES_PROP, + SnapshotSummary.REMOVED_POS_DELETES_PROP, + SnapshotSummary.TOTAL_DATA_FILES_PROP, + SnapshotSummary.TOTAL_DELETE_FILES_PROP, + }; + assertThat(table.currentSnapshot().summary()).containsKeys(commitMetricsKeys); + } + + @TestTemplate + public void testRewriteManyColumns() throws Exception { + List fields = + Lists.newArrayList(Types.NestedField.required(0, "id", Types.LongType.get())); + List additionalCols = + IntStream.range(1, 1010) + .mapToObj(i -> Types.NestedField.optional(i, "c" + i, Types.StringType.get())) + .collect(Collectors.toList()); + fields.addAll(additionalCols); + Schema schema = new Schema(fields); + PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("id", 2).build(); + Table table = + validationCatalog.createTable( + TableIdentifier.of("default", TABLE_NAME), schema, spec, tableProperties()); + + Dataset df = + spark + .range(4) + .withColumns( + IntStream.range(1, 1010) + .boxed() + .collect(Collectors.toMap(i -> "c" + i, i -> expr("CAST(id as STRING)")))); + StructType sparkSchema = spark.table(name(table)).schema(); + spark + .createDataFrame(df.rdd(), sparkSchema) + .coalesce(1) + .write() + .format("iceberg") + .mode("append") + .save(name(table)); + + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 1, 1, dataFiles); + assertThat(dataFiles).hasSize(2); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(2); + + List expectedRecords = records(table); + List expectedDeletes = deleteRecords(table); + assertThat(expectedRecords).hasSize(2); + assertThat(expectedDeletes).hasSize(2); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + SizeBasedFileRewritePlanner.TARGET_FILE_SIZE_BYTES, + Long.toString(Long.MAX_VALUE - 1)) + .execute(); + + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).hasSize(2); + assertNotContains(deleteFiles, newDeleteFiles); + assertLocallySorted(newDeleteFiles); + checkResult(result, deleteFiles, newDeleteFiles, 2); + checkSequenceNumbers(table, deleteFiles, newDeleteFiles); + + List actualRecords = records(table); + List actualDeletes = deleteRecords(table); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + } + + @TestTemplate + public void testRewriteV2PositionDeletesToV3DVs() throws IOException { + Table table = createTableUnpartitioned(2, SCALE); + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 2, DELETES_SCALE, dataFiles); + assertThat(dataFiles).hasSize(2); + assertThat(deleteFiles(table)).hasSize(2).allMatch(file -> file.format() == FileFormat.PARQUET); + + List expectedRecords = records(table); + List expectedDeletes = deleteRecords(table); + assertThat(expectedRecords).hasSize(2000); + assertThat(expectedDeletes).hasSize(2000); + assertThat(dvRecords(table)).isEmpty(); + + // upgrade the table to V3 + table.updateProperties().set(TableProperties.FORMAT_VERSION, "3").commit(); + + // v2 position deletes should now be rewritten to DVs + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + assertThat(result.rewrittenDeleteFilesCount()).isEqualTo(2); + assertThat(result.addedDeleteFilesCount()).isEqualTo(2); + assertThat(deleteFiles(table)).hasSize(2).allMatch(file -> file.format() == FileFormat.PUFFIN); + assertThat(dvRecords(table)).hasSize(2); + + // rewriting DVs via rewritePositionDeletes shouldn't be possible anymore + assertThat(SparkActions.get(spark).rewritePositionDeletes(table).execute().rewriteResults()) + .isEmpty(); + } + + private List dvRecords(Table table) { + return spark + .read() + .format("iceberg") + .option(SparkReadOptions.SPLIT_SIZE, 1024 * 1024 * 64) + .option(SparkReadOptions.FILE_OPEN_COST, 0) + .load(name(table) + ".position_deletes") + .select("file_path", "delete_file_path") + .where(col("delete_file_path").endsWith(".puffin")) + .coalesce(1) + .distinct() + .collectAsList(); + } + + private Table createTablePartitioned(int partitions, int files, int numRecords) { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + Table table = + validationCatalog.createTable( + TableIdentifier.of("default", TABLE_NAME), SCHEMA, spec, tableProperties()); + + writeRecords(table, files, numRecords, partitions); + return table; + } + + private Table createTableUnpartitioned(int files, int numRecords) { + Table table = + validationCatalog.createTable( + TableIdentifier.of("default", TABLE_NAME), + SCHEMA, + PartitionSpec.unpartitioned(), + tableProperties()); + + writeRecords(table, files, numRecords); + return table; + } + + private Map tableProperties() { + return tableProperties(2); + } + + private Map tableProperties(int formatVersion) { + return ImmutableMap.of( + TableProperties.DEFAULT_WRITE_METRICS_MODE, + "full", + TableProperties.FORMAT_VERSION, + String.valueOf(formatVersion), + TableProperties.DEFAULT_FILE_FORMAT, + format.toString(), + TableProperties.DELETE_GRANULARITY, + DeleteGranularity.PARTITION.toString()); + } + + private void writeRecords(Table table, int files, int numRecords) { + writeRecords(table, files, numRecords, 1); + } + + private void writeRecords(Table table, int files, int numRecords, int numPartitions) { + writeRecordsWithPartitions( + table, + files, + numRecords, + IntStream.range(0, numPartitions).mapToObj(ImmutableList::of).collect(Collectors.toList())); + } + + private void writeRecordsWithPartitions( + Table table, int files, int numRecords, List> partitions) { + int partitionTypeSize = table.spec().partitionType().fields().size(); + assertThat(partitionTypeSize) + .as("This method currently supports only two columns as partition columns") + .isLessThanOrEqualTo(2); + + BiFunction, ThreeColumnRecord> recordFunction = + (i, partValues) -> { + switch (partitionTypeSize) { + case (0): + return new ThreeColumnRecord(i, String.valueOf(i), String.valueOf(i)); + case (1): + return new ThreeColumnRecord(partValues.get(0), String.valueOf(i), String.valueOf(i)); + case (2): + return new ThreeColumnRecord( + partValues.get(0), String.valueOf(partValues.get(1)), String.valueOf(i)); + default: + throw new ValidationException( + "This method currently supports only two columns as partition columns"); + } + }; + List records = + partitions.stream() + .flatMap( + partition -> + IntStream.range(0, numRecords) + .mapToObj(i -> recordFunction.apply(i, partition))) + .collect(Collectors.toList()); + spark + .createDataFrame(records, ThreeColumnRecord.class) + .repartition(files) + .write() + .format("iceberg") + .mode("append") + .save(name(table)); + table.refresh(); + } + + private void writeNewSchemaRecords( + Table table, int files, int numRecords, int startingPartition, int partitions) { + List records = + IntStream.range(startingPartition, startingPartition + partitions) + .boxed() + .flatMap( + partition -> + IntStream.range(0, numRecords) + .mapToObj( + i -> + new FourColumnRecord( + partition, + String.valueOf(i), + String.valueOf(i), + String.valueOf(i)))) + .collect(Collectors.toList()); + spark + .createDataFrame(records, FourColumnRecord.class) + .repartition(files) + .write() + .format("iceberg") + .mode("append") + .save(name(table)); + } + + private List records(Table table) { + return rowsToJava( + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SPLIT_SIZE, 1024 * 1024 * 64) + .option(SparkReadOptions.FILE_OPEN_COST, 0) + .load(name(table)) + .coalesce(1) + .sort("c1", "c2", "c3") + .collectAsList()); + } + + private List deleteRecords(Table table) { + String[] additionalFields; + // do not select delete_file_path for comparison + // as delete files have been rewritten + if (table.spec().isUnpartitioned()) { + additionalFields = new String[] {"pos", "row"}; + } else { + additionalFields = new String[] {"pos", "row", "partition", "spec_id"}; + } + return rowsToJava( + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SPLIT_SIZE, 1024 * 1024 * 64) + .option(SparkReadOptions.FILE_OPEN_COST, 0) + .load(name(table) + ".position_deletes") + .select("file_path", additionalFields) + .coalesce(1) + .sort("file_path", "pos") + .collectAsList()); + } + + private void writePosDeletesForFiles( + Table table, int deleteFilesPerPartition, int deletesPerDataFile, List files) + throws IOException { + writePosDeletesForFiles(table, deleteFilesPerPartition, deletesPerDataFile, files, false); + } + + private void writePosDeletesForFiles( + Table table, + int deleteFilesPerPartition, + int deletesPerDataFile, + List files, + boolean transactional) + throws IOException { + + Map> filesByPartition = + files.stream().collect(Collectors.groupingBy(ContentFile::partition)); + List deleteFiles = + Lists.newArrayListWithCapacity(deleteFilesPerPartition * filesByPartition.size()); + String suffix = String.format(".%s", FileFormat.PARQUET.name().toLowerCase()); + + for (Map.Entry> filesByPartitionEntry : + filesByPartition.entrySet()) { + + StructLike partition = filesByPartitionEntry.getKey(); + List partitionFiles = filesByPartitionEntry.getValue(); + + int deletesForPartition = partitionFiles.size() * deletesPerDataFile; + assertThat(deletesForPartition % deleteFilesPerPartition) + .as( + "Number of delete files per partition should be " + + "evenly divisible by requested deletes per data file times number of data files in this partition") + .isZero(); + + int deleteFileSize = deletesForPartition / deleteFilesPerPartition; + int counter = 0; + List> deletes = Lists.newArrayList(); + for (DataFile partitionFile : partitionFiles) { + for (int deletePos = 0; deletePos < deletesPerDataFile; deletePos++) { + deletes.add(Pair.of(partitionFile.location(), (long) deletePos)); + counter++; + if (counter == deleteFileSize) { + // Dump to file and reset variables + OutputFile output = + Files.localOutput(File.createTempFile("junit", suffix, temp.toFile())); + deleteFiles.add(FileHelpers.writeDeleteFile(table, output, partition, deletes).first()); + counter = 0; + deletes.clear(); + } + } + } + } + + if (transactional) { + RowDelta rowDelta = table.newRowDelta(); + deleteFiles.forEach(rowDelta::addDeletes); + rowDelta.commit(); + } else { + deleteFiles.forEach( + deleteFile -> { + RowDelta rowDelta = table.newRowDelta(); + rowDelta.addDeletes(deleteFile); + rowDelta.commit(); + }); + } + } + + private List deleteFiles(Table table) { + Table deletesTable = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); + CloseableIterable tasks = deletesTable.newBatchScan().planFiles(); + return Lists.newArrayList( + CloseableIterable.transform(tasks, t -> ((PositionDeletesScanTask) t).file())); + } + + private > List except(List first, List second) { + Set secondPaths = + second.stream().map(ContentFile::location).collect(Collectors.toSet()); + return first.stream() + .filter(f -> !secondPaths.contains(f.location())) + .collect(Collectors.toList()); + } + + private void assertNotContains(List original, List rewritten) { + Set originalPaths = + original.stream().map(ContentFile::location).collect(Collectors.toSet()); + Set rewrittenPaths = + rewritten.stream().map(ContentFile::location).collect(Collectors.toSet()); + rewrittenPaths.retainAll(originalPaths); + assertThat(rewrittenPaths).isEmpty(); + } + + private void assertLocallySorted(List deleteFiles) { + for (DeleteFile deleteFile : deleteFiles) { + Dataset deletes = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.FILE_OPEN_COST, 0) + .option(SparkReadOptions.SPLIT_SIZE, 1024 * 1024 * 64) + .load("default." + TABLE_NAME + ".position_deletes"); + deletes.filter(deletes.col("delete_file_path").equalTo(deleteFile.location())); + List rows = deletes.collectAsList(); + assertThat(rows).as("Empty delete file found").isNotEmpty(); + int lastPos = 0; + String lastPath = ""; + for (Row row : rows) { + String path = row.getAs("file_path"); + long pos = row.getAs("pos"); + if (path.compareTo(lastPath) < 0) { + fail(String.format("File_path not sorted, Found %s after %s", path, lastPath)); + } else if (path.equals(lastPath)) { + assertThat(pos).as("Pos not sorted").isGreaterThanOrEqualTo(lastPos); + } + } + } + } + + private String name(Table table) { + String[] splits = table.name().split("\\."); + assertThat(splits).hasSize(3); + return String.format("%s.%s", splits[1], splits[2]); + } + + private long size(List deleteFiles) { + return deleteFiles.stream().mapToLong(DeleteFile::fileSizeInBytes).sum(); + } + + private List filterDeletes(List deletes, List... partitionValues) { + Stream matches = + deletes.stream() + .filter( + r -> { + Object[] partition = (Object[]) r[3]; + return Arrays.stream(partitionValues) + .map(partitionValue -> match(partition, partitionValue)) + .reduce((a, b) -> a || b) + .get(); + }); + return sorted(matches).collect(Collectors.toList()); + } + + private boolean match(Object[] partition, List expectedPartition) { + return IntStream.range(0, expectedPartition.size()) + .mapToObj(j -> partition[j] == expectedPartition.get(j)) + .reduce((a, b) -> a && b) + .get(); + } + + private Stream sorted(Stream deletes) { + return deletes.sorted( + (a, b) -> { + String aFilePath = (String) a[0]; + String bFilePath = (String) b[0]; + int filePathCompare = aFilePath.compareTo(bFilePath); + if (filePathCompare != 0) { + return filePathCompare; + } else { + long aPos = (long) a[1]; + long bPos = (long) b[1]; + return Long.compare(aPos, bPos); + } + }); + } + + private List filterFiles( + Table table, List files, List... partitionValues) { + List partitionTypes = + table.specs().values().stream() + .map(PartitionSpec::partitionType) + .collect(Collectors.toList()); + List partitionDatas = + Arrays.stream(partitionValues) + .map( + partitionValue -> { + Types.StructType thisType = + partitionTypes.stream() + .filter(f -> f.fields().size() == partitionValue.size()) + .findFirst() + .get(); + PartitionData partition = new PartitionData(thisType); + for (int i = 0; i < partitionValue.size(); i++) { + partition.set(i, partitionValue.get(i)); + } + return partition; + }) + .collect(Collectors.toList()); + + return files.stream() + .filter(f -> partitionDatas.stream().anyMatch(data -> f.partition().equals(data))) + .collect(Collectors.toList()); + } + + private void checkResult( + Result result, + List rewrittenDeletes, + List newDeletes, + int expectedGroups) { + assertThat(result.rewrittenDeleteFilesCount()) + .as("Rewritten delete file count") + .isEqualTo(rewrittenDeletes.size()); + + assertThat(result.addedDeleteFilesCount()) + .as("New delete file count") + .isEqualTo(newDeletes.size()); + + assertThat(result.rewrittenBytesCount()) + .as("Rewritten delete byte count") + .isEqualTo(size(rewrittenDeletes)); + + assertThat(result.addedBytesCount()).as("New delete byte count").isEqualTo(size(newDeletes)); + + assertThat(result.rewriteResults()).as("Rewrite group count").hasSize(expectedGroups); + + assertThat( + result.rewriteResults().stream() + .mapToInt(FileGroupRewriteResult::rewrittenDeleteFilesCount) + .sum()) + .as("Rewritten delete file count in all groups") + .isEqualTo(rewrittenDeletes.size()); + + assertThat( + result.rewriteResults().stream() + .mapToInt(FileGroupRewriteResult::addedDeleteFilesCount) + .sum()) + .as("Added delete file count in all groups") + .isEqualTo(newDeletes.size()); + + assertThat( + result.rewriteResults().stream() + .mapToLong(FileGroupRewriteResult::rewrittenBytesCount) + .sum()) + .as("Rewritten delete bytes in all groups") + .isEqualTo(size(rewrittenDeletes)); + + assertThat( + result.rewriteResults().stream() + .mapToLong(FileGroupRewriteResult::addedBytesCount) + .sum()) + .as("Added delete bytes in all groups") + .isEqualTo(size(newDeletes)); + } + + private void checkSequenceNumbers( + Table table, List rewrittenDeletes, List addedDeletes) { + StructLikeMap> rewrittenFilesPerPartition = + groupPerPartition(table, rewrittenDeletes); + StructLikeMap> addedFilesPerPartition = groupPerPartition(table, addedDeletes); + for (StructLike partition : rewrittenFilesPerPartition.keySet()) { + Long maxRewrittenSeq = + rewrittenFilesPerPartition.get(partition).stream() + .mapToLong(ContentFile::dataSequenceNumber) + .max() + .getAsLong(); + List addedPartitionFiles = addedFilesPerPartition.get(partition); + if (addedPartitionFiles != null) { + addedPartitionFiles.forEach( + d -> + assertThat(d.dataSequenceNumber()) + .as("Sequence number should be max of rewritten set") + .isEqualTo(maxRewrittenSeq)); + } + } + } + + private StructLikeMap> groupPerPartition( + Table table, List deleteFiles) { + StructLikeMap> result = + StructLikeMap.create(Partitioning.partitionType(table)); + for (DeleteFile deleteFile : deleteFiles) { + StructLike partition = deleteFile.partition(); + List partitionFiles = result.get(partition); + if (partitionFiles == null) { + partitionFiles = Lists.newArrayList(); + } + partitionFiles.add(deleteFile); + result.put(partition, partitionFiles); + } + return result; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java new file mode 100644 index 000000000000..0bcaf0af6581 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteTablePathsAction.java @@ -0,0 +1,1600 @@ +/* + * 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.actions; + +import static org.apache.iceberg.spark.actions.RewriteTablePathSparkAction.NOT_APPLICABLE; +import static org.apache.iceberg.types.Types.NestedField.optional; +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.net.URI; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StaticTableOperations; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.actions.ActionsProvider; +import org.apache.iceberg.actions.ExpireSnapshots; +import org.apache.iceberg.actions.RewriteManifests; +import org.apache.iceberg.actions.RewriteTablePath; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +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.SparkCatalog; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.spark.source.ThreeColumnRecord; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.SparkEnv; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.BlockInfoManager; +import org.apache.spark.storage.BlockManager; +import org.apache.spark.storage.BroadcastBlockId; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import scala.Tuple2; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewriteTablePathsAction extends TestBase { + + @TempDir private Path staging; + @TempDir private Path tableDir; + @TempDir private Path newTableDir; + @TempDir private Path targetTableDir; + + @Parameters(name = "formatVersion = {0}") + protected static List formatVersions() { + return TestHelpers.V2_AND_ABOVE; + } + + @Parameter private int formatVersion; + + protected ActionsProvider actions() { + return SparkActions.get(); + } + + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + protected static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + + protected String tableLocation = null; + private Table table = null; + + private final String ns = "testns"; + private final String backupNs = "backupns"; + + @BeforeEach + public void setupTableLocation() { + this.tableLocation = tableDir.toFile().toURI().toString(); + this.table = createATableWith2Snapshots(tableLocation); + createNameSpaces(); + } + + @AfterEach + public void cleanupTableSetup() { + dropNameSpaces(); + } + + private Table createATableWith2Snapshots(String location) { + return createTableWithSnapshots(location, 2); + } + + private Table createTableWithSnapshots(String location, int snapshotNumber) { + return createTableWithSnapshots(location, snapshotNumber, Maps.newHashMap()); + } + + protected Table createTableWithSnapshots( + String location, int snapshotNumber, Map properties) { + return createTableWithSnapshots(location, snapshotNumber, properties, "append"); + } + + private Table createTableWithSnapshots( + String location, int snapshotNumber, Map properties, String mode) { + Table newTable = + TABLES.create( + SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.builder() + .put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)) + .putAll(properties) + .build(), + location); + + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + + for (int i = 0; i < snapshotNumber; i++) { + df.select("c1", "c2", "c3").write().format("iceberg").mode(mode).save(location); + } + + return newTable; + } + + private void createNameSpaces() { + sql("CREATE DATABASE IF NOT EXISTS %s", ns); + sql("CREATE DATABASE IF NOT EXISTS %s", backupNs); + } + + private void dropNameSpaces() { + sql("DROP DATABASE IF EXISTS %s CASCADE", ns); + sql("DROP DATABASE IF EXISTS %s CASCADE", backupNs); + } + + @TestTemplate + public void testRewritePath() throws Exception { + String targetTableLocation = targetTableLocation(); + + // check the data file location before the rebuild + List validDataFiles = + spark + .read() + .format("iceberg") + .load(tableLocation + "#files") + .select("file_path") + .as(Encoders.STRING()) + .collectAsList(); + assertThat(validDataFiles).hasSize(2); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(tableLocation, targetTableLocation) + .endVersion("v3.metadata.json") + .execute(); + + assertThat(result.latestVersion()).isEqualTo("v3.metadata.json"); + + checkFileNum(3, 2, 2, 9, result); + + // copy the metadata files and data files + copyTableFiles(result); + + // verify the data file path after the rebuild + List validDataFilesAfterRebuilt = + spark + .read() + .format("iceberg") + .load(targetTableLocation + "#files") + .select("file_path") + .as(Encoders.STRING()) + .collectAsList(); + assertThat(validDataFilesAfterRebuilt) + .hasSize(2) + .allMatch(item -> item.startsWith(targetTableLocation)); + + // verify data rows + List actual = rows(targetTableLocation); + List expected = rows(tableLocation); + assertEquals("Rows should match after copy", expected, actual); + } + + @TestTemplate + public void testSameLocations() { + assertThatThrownBy( + () -> + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(tableLocation, tableLocation) + .endVersion("v1.metadata.json") + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Source prefix cannot be the same as target prefix"); + } + + @TestTemplate + public void testStartVersion() throws Exception { + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(tableLocation, targetTableLocation()) + .startVersion("v2.metadata.json") + .execute(); + + checkFileNum(1, 1, 1, 4, result); + + List> paths = readPathPairList(result.fileListLocation()); + + String currentSnapshotId = String.valueOf(table.currentSnapshot().snapshotId()); + assertThat(paths.stream().filter(c -> c._2().contains(currentSnapshotId))) + .as("Should have the current snapshot file") + .hasSize(1); + + String parentSnapshotId = String.valueOf(table.currentSnapshot().parentId()); + assertThat(paths.stream().filter(c -> c._2().contains(parentSnapshotId))) + .as("Should NOT have the parent snapshot file") + .isEmpty(); + } + + @TestTemplate + public void testIncrementalRewrite() throws Exception { + String location = newTableLocation(); + Table sourceTable = + TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap(), location); + List recordsA = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + Dataset dfA = spark.createDataFrame(recordsA, ThreeColumnRecord.class).coalesce(1); + + // Write first increment to source table + dfA.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(location); + assertThat(spark.read().format("iceberg").load(location).collectAsList()).hasSize(1); + + // Replicate first increment to target table + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(sourceTable.location(), targetTableLocation()) + .execute(); + copyTableFiles(result); + assertThat(spark.read().format("iceberg").load(targetTableLocation()).collectAsList()) + .hasSize(1); + + // Write second increment to source table + List recordsB = + Lists.newArrayList(new ThreeColumnRecord(2, "BBBBBBBBB", "BBB")); + Dataset dfB = spark.createDataFrame(recordsB, ThreeColumnRecord.class).coalesce(1); + dfB.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(location); + assertThat(spark.read().format("iceberg").load(location).collectAsList()).hasSize(2); + + // Replicate second increment to target table + sourceTable.refresh(); + Table targetTable = TABLES.load(targetTableLocation()); + String targetTableMetadata = currentMetadata(targetTable).metadataFileLocation(); + String startVersion = fileName(targetTableMetadata); + RewriteTablePath.Result incrementalRewriteResult = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(sourceTable.location(), targetTableLocation()) + .startVersion(startVersion) + .execute(); + copyTableFiles(incrementalRewriteResult); + List actual = rowsSorted(targetTableLocation(), "c1"); + List expected = rowsSorted(location, "c1"); + assertEquals("Rows should match after copy", expected, actual); + } + + @TestTemplate + public void testTableWith3Snapshots(@TempDir Path location1, @TempDir Path location2) + throws Exception { + String location = newTableLocation(); + Table tableWith3Snaps = createTableWithSnapshots(location, 3); + RewriteTablePath.Result result = + actions() + .rewriteTablePath(tableWith3Snaps) + .rewriteLocationPrefix(location, toAbsolute(location1)) + .startVersion("v2.metadata.json") + .execute(); + + checkFileNum(2, 2, 2, 8, result); + + // start from the first version + RewriteTablePath.Result result1 = + actions() + .rewriteTablePath(tableWith3Snaps) + .rewriteLocationPrefix(location, toAbsolute(location2)) + .startVersion("v1.metadata.json") + .execute(); + + checkFileNum(3, 3, 3, 12, result1); + } + + @TestTemplate + public void testFullTableRewritePath() throws Exception { + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(tableLocation, targetTableLocation()) + .execute(); + + checkFileNum(3, 2, 2, 9, result); + } + + @TestTemplate + public void testManifestRewriteAndIncrementalCopy() throws Exception { + RewriteTablePath.Result initialResult = + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(table.location(), targetTableLocation()) + .stagingLocation(stagingLocation()) + .execute(); + checkFileNum(3, 2, 2, 9, initialResult); + + // rewrite manifest without change data files + RewriteManifests.Result rewriteManifestResult = actions().rewriteManifests(table).execute(); + int addedManifest = Iterables.size(rewriteManifestResult.addedManifests()); + + // only move version v4, which is the version generated by rewrite manifest + RewriteTablePath.Result postReweiteResult = + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(table.location(), targetTableLocation()) + .stagingLocation(stagingLocation()) + .startVersion("v3.metadata.json") + .execute(); + + // no data files need to move + checkFileNum(1, 1, addedManifest, 3, postReweiteResult); + } + + @TestTemplate + public void testDeleteDataFile() throws Exception { + List validDataFiles = + spark + .read() + .format("iceberg") + .load(table.location() + "#files") + .select("file_path") + .as(Encoders.STRING()) + .collectAsList(); + + table.newDelete().deleteFile(validDataFiles.stream().findFirst().get()).commit(); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(table.location(), targetTableLocation()) + .stagingLocation(stagingLocation()) + .execute(); + + checkFileNum(4, 3, 3, 12, result); + + // copy the metadata files and data files + copyTableFiles(result); + + // verify data rows + Dataset resultDF = spark.read().format("iceberg").load(targetTableLocation()); + assertThat(resultDF.as(Encoders.bean(ThreeColumnRecord.class)).collectAsList()) + .as("There are only one row left since we deleted a data file") + .hasSize(1); + } + + @TestTemplate + public void testPositionDeletesParquet() throws Exception { + runPositionDeletesTest("parquet"); + } + + @TestTemplate + public void testPositionDeletesAvro() throws Exception { + runPositionDeletesTest("avro"); + } + + @TestTemplate + public void testPositionDeletesOrc() throws Exception { + runPositionDeletesTest("orc"); + } + + private void runPositionDeletesTest(String fileFormat) throws Exception { + Table tableWithPosDeletes = + createTableWithSnapshots( + tableDir.toFile().toURI().toString().concat("tableWithPosDeletes").concat(fileFormat), + 2, + Map.of(TableProperties.DELETE_DEFAULT_FILE_FORMAT, fileFormat)); + + List> deletes = + Lists.newArrayList( + Pair.of( + tableWithPosDeletes + .currentSnapshot() + .addedDataFiles(tableWithPosDeletes.io()) + .iterator() + .next() + .location(), + 0L)); + + File file = + new File( + removePrefix( + tableWithPosDeletes.location() + "/data/deeply/nested/deletes." + fileFormat)); + DeleteFile positionDeletes = + FileHelpers.writeDeleteFile( + tableWithPosDeletes, + tableWithPosDeletes.io().newOutputFile(file.toURI().toString()), + deletes, + formatVersion) + .first(); + + tableWithPosDeletes.newRowDelta().addDeletes(positionDeletes).commit(); + + assertThat(spark.read().format("iceberg").load(tableWithPosDeletes.location()).collectAsList()) + .hasSize(1); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(tableWithPosDeletes) + .stagingLocation(stagingLocation()) + .rewriteLocationPrefix(tableWithPosDeletes.location(), targetTableLocation()) + .execute(); + + // We have one more snapshot, an additional manifest list, and a new (delete) manifest, + // and an additional position delete + checkFileNum(4, 3, 3, 13, result); + + // copy the metadata files and data files + copyTableFiles(result); + + // Positional delete affects a single row, so only one row must remain + assertThat(spark.read().format("iceberg").load(targetTableLocation()).collectAsList()) + .hasSize(1); + } + + @TestTemplate + public void testPositionDeleteWithRow() throws Exception { + String dataFileLocation = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().location(); + List> deletes = Lists.newArrayList(); + OutputFile deleteFile = + table + .io() + .newOutputFile( + new File(removePrefix(table.location() + "/data/deeply/nested/deletes.parquet")) + .toURI() + .toString()); + deletes.add(positionDelete(SCHEMA, dataFileLocation, 0L, 1, "AAAAAAAAAA", "AAAA")); + DeleteFile positionDeletes = + FileHelpers.writePosDeleteFile(table, deleteFile, null, deletes, formatVersion); + table.newRowDelta().addDeletes(positionDeletes).commit(); + + assertThat(spark.read().format("iceberg").load(table.location()).collectAsList()).hasSize(1); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .stagingLocation(stagingLocation()) + .rewriteLocationPrefix(table.location(), targetTableLocation()) + .execute(); + + // We have one more snapshot, an additional manifest list, and a new (delete) manifest, + // and an additional position delete + checkFileNum(4, 3, 3, 13, result); + + // copy the metadata files and data files + copyTableFiles(result); + + // check copied position delete row - only v2 stores row data with position deletes + // v3+ uses Deletion Vectors (DV) which only store position information + if (formatVersion == 2) { + Object[] deletedRow = (Object[]) rows(targetTableLocation() + "#position_deletes").get(0)[2]; + assertEquals( + "Position deletes should be equal", new Object[] {1, "AAAAAAAAAA", "AAAA"}, deletedRow); + } + + // Positional delete affects a single row, so only one row must remain + assertThat(spark.read().format("iceberg").load(targetTableLocation()).collectAsList()) + .hasSize(1); + } + + @TestTemplate + public void testPositionDeletesAcrossFiles() throws Exception { + assumeThat(formatVersion) + .as("Can't write multiple deletes into a single v3 delete file") + .isEqualTo(2); + Stream allFiles = + StreamSupport.stream(table.snapshots().spliterator(), false) + .flatMap(s -> StreamSupport.stream(s.addedDataFiles(table.io()).spliterator(), false)); + List> deletes = + allFiles.map(f -> Pair.of((CharSequence) f.location(), 0L)).collect(Collectors.toList()); + + // a single position delete with two entries + assertThat(deletes).hasSize(2); + + File file = new File(removePrefix(table.location() + "/data/deeply/nested/file.parquet")); + DeleteFile positionDeletes = + FileHelpers.writeDeleteFile( + table, table.io().newOutputFile(file.toURI().toString()), deletes, formatVersion) + .first(); + + table.newRowDelta().addDeletes(positionDeletes).commit(); + + assertThat(spark.read().format("iceberg").load(table.location()).collectAsList()).isEmpty(); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .stagingLocation(stagingLocation()) + .rewriteLocationPrefix(table.location(), targetTableLocation()) + .execute(); + + // We have one more snapshot, an additional manifest list, and a new (delete) manifest, + // and an additional position delete + checkFileNum(4, 3, 3, 13, result); + + // copy the metadata files and data files + copyTableFiles(result); + + assertThat(spark.read().format("iceberg").load(targetTableLocation()).collectAsList()) + .isEmpty(); + } + + @TestTemplate + public void testEqualityDeletes() throws Exception { + Table sourceTable = createTableWithSnapshots(newTableLocation(), 1); + + // Add more varied data + List records = + Lists.newArrayList( + new ThreeColumnRecord(2, "AAAAAAAAAA", "AAAA"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "BBBB"), + new ThreeColumnRecord(4, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(5, "DDDDDDDDDD", "DDDD")); + spark + .createDataFrame(records, ThreeColumnRecord.class) + .coalesce(1) + .select("c1", "c2", "c3") + .write() + .format("iceberg") + .mode("append") + .save(newTableLocation()); + + Schema deleteRowSchema = sourceTable.schema().select("c2"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + List dataDeletes = + Lists.newArrayList( + dataDelete.copy("c2", "AAAAAAAAAA"), dataDelete.copy("c2", "CCCCCCCCCC")); + File file = new File(removePrefix(sourceTable.location()) + "/data/deeply/nested/file.parquet"); + DeleteFile equalityDeletes = + FileHelpers.writeDeleteFile( + sourceTable, + sourceTable.io().newOutputFile(file.toURI().toString()), + TestHelpers.Row.of(0), + dataDeletes, + deleteRowSchema); + sourceTable.newRowDelta().addDeletes(equalityDeletes).commit(); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .stagingLocation(stagingLocation()) + .rewriteLocationPrefix(newTableLocation(), targetTableLocation()) + .execute(); + + // We have four metadata files: for the table creation, for the initial snapshot, for the + // second append here, and for commit with equality deletes. Thus, we have three manifest lists. + // We have a data file for each snapshot (two with data, one with equality deletes) + checkFileNum(4, 3, 3, 13, result); + + // copy the metadata files and data files + copyTableFiles(result); + + // Equality deletes affect three rows, so just two rows must remain + assertThat(spark.read().format("iceberg").load(targetTableLocation()).collectAsList()) + .hasSize(2); + } + + @TestTemplate + public void testFullTableRewritePathWithDeletedVersionFiles() throws Exception { + String location = newTableLocation(); + Table sourceTable = createTableWithSnapshots(location, 2); + // expire the first snapshot + Table staticTable = newStaticTable(location + "metadata/v2.metadata.json", table.io()); + int expiredManifestListCount = 1; + ExpireSnapshots.Result expireResult = + actions() + .expireSnapshots(sourceTable) + .expireSnapshotId(staticTable.currentSnapshot().snapshotId()) + .execute(); + assertThat(expireResult.deletedManifestListsCount()).isEqualTo(expiredManifestListCount); + + // create 100 more snapshots + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + for (int i = 0; i < 100; i++) { + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(location); + } + sourceTable.refresh(); + + // each iteration generate 1 version file, 1 manifest list, 1 manifest and 1 data file + int totalIteration = 102; + // v1/v2/v3.metadata.json has been deleted in v104.metadata.json, and there is no way to find + // the first snapshot + // from the version file history + int missingVersionFile = 1; + // since first snapshot cannot be found, first data files will also be skipped + int missingDataFile = 1; + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .stagingLocation(stagingLocation()) + .rewriteLocationPrefix(location, targetTableLocation()) + .execute(); + + checkFileNum( + totalIteration - missingVersionFile, + totalIteration - expiredManifestListCount, + totalIteration, + totalIteration * 4 - missingVersionFile - expiredManifestListCount - missingDataFile, + result); + } + + @TestTemplate + public void testRewritePathWithoutSnapshot() throws Exception { + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(tableLocation, newTableLocation()) + .endVersion("v1.metadata.json") + .execute(); + + // the only rebuilt file is v1.metadata.json since it contains no snapshot + checkFileNum(1, 0, 0, 1, result); + } + + @TestTemplate + public void testExpireSnapshotBeforeRewrite() throws Exception { + // expire one snapshot + actions().expireSnapshots(table).expireSnapshotId(table.currentSnapshot().parentId()).execute(); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .stagingLocation(stagingLocation()) + .rewriteLocationPrefix(table.location(), targetTableLocation()) + .execute(); + + checkFileNum(4, 1, 2, 9, result); + } + + @TestTemplate + public void testRewritePathWithNonLiveEntry() throws Exception { + String location = newTableLocation(); + // first overwrite generate 1 manifest and 1 data file + // each subsequent overwrite on unpartitioned table generate 2 manifests and 1 data file + Table tableWith3Snaps = createTableWithSnapshots(location, 3, Maps.newHashMap(), "overwrite"); + + Snapshot oldest = SnapshotUtil.oldestAncestor(tableWith3Snaps); + String oldestDataFilePath = + Iterables.getOnlyElement( + tableWith3Snaps.snapshot(oldest.snapshotId()).addedDataFiles(tableWith3Snaps.io())) + .location(); + String deletedDataFilePathInTargetLocation = + String.format("%sdata/%s", targetTableLocation(), fileName(oldestDataFilePath)); + + // expire the oldest snapshot and remove oldest DataFile + ExpireSnapshots.Result expireResult = + actions().expireSnapshots(tableWith3Snaps).expireSnapshotId(oldest.snapshotId()).execute(); + assertThat(expireResult) + .as("Should deleted 1 data files in root snapshot") + .extracting( + ExpireSnapshots.Result::deletedManifestListsCount, + ExpireSnapshots.Result::deletedManifestsCount, + ExpireSnapshots.Result::deletedDataFilesCount) + .contains(1L, 1L, 1L); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(tableWith3Snaps) + .stagingLocation(stagingLocation()) + .rewriteLocationPrefix(tableWith3Snaps.location(), targetTableLocation()) + .execute(); + + // 5 version files include 1 table creation 3 overwrite and 1 snapshot expiration + // 3 overwrites generate 3 manifest list and 5 manifests with 3 data files + // snapshot expiration removed 1 of each + checkFileNum(5, 2, 4, 13, result); + + // copy the metadata files and data files + copyTableFiles(result); + + // expect deleted data file is excluded from rewrite and copy + List copiedDataFiles = + spark + .read() + .format("iceberg") + .load(targetTableLocation() + "#all_files") + .select("file_path") + .as(Encoders.STRING()) + .collectAsList(); + assertThat(copiedDataFiles).hasSize(2).doesNotContain(deletedDataFilePathInTargetLocation); + + // expect manifest entries still contain deleted entry + List copiedEntries = + spark + .read() + .format("iceberg") + .load(targetTableLocation() + "#all_entries") + .filter("status == 2") + .select("data_file.file_path") + .as(Encoders.STRING()) + .collectAsList(); + assertThat(copiedEntries).contains(deletedDataFilePathInTargetLocation); + } + + @TestTemplate + public void testStartSnapshotWithoutValidSnapshot() throws Exception { + // expire one snapshot + actions().expireSnapshots(table).expireSnapshotId(table.currentSnapshot().parentId()).execute(); + + assertThat(table.snapshots()).hasSize(1); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(table.location(), targetTableLocation()) + .stagingLocation(stagingLocation()) + .startVersion("v2.metadata.json") + .execute(); + + // 2 metadata.json, 1 manifest list file, 1 manifest files + checkFileNum(2, 1, 1, 5, result); + } + + @TestTemplate + public void testMoveTheVersionExpireSnapshot() throws Exception { + // expire one snapshot + actions().expireSnapshots(table).expireSnapshotId(table.currentSnapshot().parentId()).execute(); + + // only move version v4, which is the version generated by snapshot expiration + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(table.location(), targetTableLocation()) + .stagingLocation(stagingLocation()) + .startVersion("v3.metadata.json") + .execute(); + + // only v4.metadata.json needs to move + checkFileNum(1, 0, 0, 1, result); + } + + @TestTemplate + public void testMoveVersionWithInvalidSnapshots() { + // expire one snapshot + actions().expireSnapshots(table).expireSnapshotId(table.currentSnapshot().parentId()).execute(); + + assertThatThrownBy( + () -> + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(table.location(), newTableLocation()) + .stagingLocation(stagingLocation()) + .endVersion("v3.metadata.json") + .execute()) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining( + "Unable to build the manifest files dataframe. The end version in use may contain invalid snapshots. " + + "Please choose an earlier version without invalid snapshots."); + } + + @TestTemplate + public void testRollBack() throws Exception { + long secondSnapshotId = table.currentSnapshot().snapshotId(); + + // roll back to the first snapshot(v2) + table.manageSnapshots().setCurrentSnapshot(table.currentSnapshot().parentId()).commit(); + + // add a new snapshot + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(table.location()); + + table.refresh(); + + // roll back to the second snapshot(v3) + table.manageSnapshots().setCurrentSnapshot(secondSnapshotId).commit(); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(table.location(), newTableLocation()) + .stagingLocation(stagingLocation()) + .execute(); + checkFileNum(6, 3, 3, 15, result); + } + + @TestTemplate + public void testWriteAuditPublish() throws Exception { + // enable WAP + table.updateProperties().set(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED, "true").commit(); + spark.conf().set("spark.wap.id", "1"); + + // add a new snapshot without changing the current snapshot of the table + List records = + Lists.newArrayList(new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA")); + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + df.select("c1", "c2", "c3").write().format("iceberg").mode("append").save(table.location()); + + table.refresh(); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(table.location(), newTableLocation()) + .stagingLocation(stagingLocation()) + .execute(); + + // There are 3 snapshots in total, although the current snapshot is the second one. + checkFileNum(5, 3, 3, 14, result); + } + + @TestTemplate + public void testSchemaChange() throws Exception { + // change the schema + table.updateSchema().addColumn("c4", Types.StringType.get()).commit(); + + // copy table + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(table.location(), newTableLocation()) + .stagingLocation(stagingLocation()) + .execute(); + + // check the result + checkFileNum(4, 2, 2, 10, result); + } + + @TestTemplate + public void testSnapshotIdInheritanceEnabled() throws Exception { + String sourceTableLocation = newTableLocation(); + Map properties = Maps.newHashMap(); + properties.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true"); + + Table sourceTable = createTableWithSnapshots(sourceTableLocation, 2, properties); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .stagingLocation(stagingLocation()) + .rewriteLocationPrefix(sourceTableLocation, targetTableLocation()) + .execute(); + + checkFileNum(3, 2, 2, 9, result); + } + + @TestTemplate + public void testMetadataCompression() throws Exception { + String sourceTableLocation = newTableLocation(); + Map properties = Maps.newHashMap(); + properties.put(TableProperties.METADATA_COMPRESSION, "gzip"); + Table sourceTable = createTableWithSnapshots(sourceTableLocation, 2, properties); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(sourceTableLocation, targetTableLocation()) + .endVersion("v2.gz.metadata.json") + .execute(); + + checkFileNum(2, 1, 1, 5, result); + + result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(sourceTableLocation, targetTableLocation()) + .startVersion("v1.gz.metadata.json") + .execute(); + + checkFileNum(2, 2, 2, 8, result); + } + + @TestTemplate + public void testInvalidArgs() { + RewriteTablePath actions = actions().rewriteTablePath(table); + + assertThatThrownBy(() -> actions.rewriteLocationPrefix("", null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Source prefix('') cannot be empty"); + + assertThatThrownBy(() -> actions.rewriteLocationPrefix(null, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Source prefix('null') cannot be empty"); + + assertThatThrownBy(() -> actions.stagingLocation("")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Staging location('') cannot be empty"); + + assertThatThrownBy(() -> actions.stagingLocation(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Staging location('null') cannot be empty"); + + assertThatThrownBy(() -> actions.startVersion(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Start version('null') cannot be empty"); + + assertThatThrownBy(() -> actions.endVersion(" ")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("End version(' ') cannot be empty"); + + assertThatThrownBy(() -> actions.endVersion(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("End version('null') cannot be empty"); + } + + @TestTemplate + public void testTableWithManyPartitionStatisticFile() throws IOException { + String sourceTableLocation = newTableLocation(); + Map properties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + String tableName = String.format("v%stblwithPartStats", formatVersion); + Table sourceTable = + createMetastoreTable(sourceTableLocation, properties, "default", tableName, 0, "c1"); + + int iterations = 10; + for (int i = 0; i < iterations; i++) { + sql("insert into hive.default.%s values (%s, 'AAAAAAAAAA', 'AAAA')", tableName, i); + sourceTable.refresh(); + actions().computePartitionStats(sourceTable).execute(); + } + + sourceTable.refresh(); + assertThat(sourceTable.partitionStatisticsFiles()).hasSize(iterations); + + String targetTableLocation = targetTableLocation(); + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(sourceTableLocation, targetTableLocation) + .execute(); + checkFileNum( + iterations * 2 + 1, iterations, iterations, 0, iterations, iterations * 6 + 1, result); + + findAndAssertFileInFileList( + result, "partition-stats", sourceTableLocation, targetTableLocation); + } + + @TestTemplate + public void testTableWithManyStatisticFiles() throws IOException { + String sourceTableLocation = newTableLocation(); + Map properties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + String tableName = String.format("v%stblwithmanystats", formatVersion); + Table sourceTable = + createMetastoreTable(sourceTableLocation, properties, "default", tableName, 0); + + int iterations = 10; + for (int i = 0; i < iterations; i++) { + sql("insert into hive.default.%s values (%s, 'AAAAAAAAAA', 'AAAA')", tableName, i); + sourceTable.refresh(); + actions().computeTableStats(sourceTable).execute(); + } + + sourceTable.refresh(); + assertThat(sourceTable.statisticsFiles()).hasSize(iterations); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(sourceTableLocation, targetTableLocation()) + .execute(); + + checkFileNum( + iterations * 2 + 1, iterations, iterations, iterations, iterations * 6 + 1, result); + } + + @TestTemplate + public void testStatisticsFileSourcePath() throws IOException { + String sourceTableLocation = newTableLocation(); + Map properties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + String tableName = String.format("v%stblwithstats", formatVersion); + Table sourceTable = + createMetastoreTable(sourceTableLocation, properties, "default", tableName, 1); + + // Compute table statistics to generate a .stats file + actions().computeTableStats(sourceTable).execute(); + + assertThat(sourceTable.statisticsFiles()) + .as("Should include 1 statistics file after compute stats") + .hasSize(1); + + String targetTableLocation = targetTableLocation(); + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(sourceTableLocation, targetTableLocation) + .execute(); + + checkFileNum(3, 1, 1, 1, 7, result); + + findAndAssertFileInFileList(result, ".stats", sourceTableLocation, targetTableLocation); + } + + @TestTemplate + public void testMetadataCompressionWithMetastoreTable() throws Exception { + Map properties = Maps.newHashMap(); + properties.put(TableProperties.METADATA_COMPRESSION, "gzip"); + Table sourceTable = + createMetastoreTable( + newTableLocation(), + properties, + "default", + String.format("v%sMetadataCompression", formatVersion), + 2); + + TableMetadata currentMetadata = currentMetadata(sourceTable); + + // set the second version as the endVersion + String endVersion = fileName(currentMetadata.previousFiles().get(1).file()); + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetTableLocation()) + .endVersion(endVersion) + .execute(); + + checkFileNum(2, 1, 1, 5, result); + + // set the first version as the lastCopiedVersion + String firstVersion = fileName(currentMetadata.previousFiles().get(0).file()); + result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetTableLocation()) + .startVersion(firstVersion) + .execute(); + + checkFileNum(2, 2, 2, 8, result); + } + + // Metastore table tests + @TestTemplate + public void testMetadataLocationChange() throws Exception { + String tableName = String.format("v%stblWithLocation", formatVersion); + Table sourceTable = + createMetastoreTable(newTableLocation(), Maps.newHashMap(), "default", tableName, 1); + String metadataFilePath = currentMetadata(sourceTable).metadataFileLocation(); + + String newMetadataDir = "new-metadata-dir"; + sourceTable + .updateProperties() + .set(TableProperties.WRITE_METADATA_LOCATION, newTableLocation() + newMetadataDir) + .commit(); + + sql("insert into hive.default.%s values (1, 'AAAAAAAAAA', 'AAAA')", tableName); + sourceTable.refresh(); + + // copy table + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetTableLocation()) + .execute(); + + checkFileNum(4, 2, 2, 10, result); + + // pick up a version from the old metadata dir as the end version + RewriteTablePath.Result result1 = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetTableLocation()) + .endVersion(fileName(metadataFilePath)) + .execute(); + + checkFileNum(2, 1, 1, 5, result1); + + // pick up a version from the old metadata dir as the last copied version + RewriteTablePath.Result result2 = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetTableLocation()) + .startVersion(fileName(metadataFilePath)) + .execute(); + + checkFileNum(2, 1, 1, 5, result2); + } + + @TestTemplate + public void testDeleteFrom() throws Exception { + Map properties = + ImmutableMap.of( + TableProperties.FORMAT_VERSION, + String.valueOf(formatVersion), + "write.delete.mode", + "merge-on-read"); + String tableName = String.format("v%stbl", formatVersion); + Table sourceTable = + createMetastoreTable(newTableLocation(), properties, "default", tableName, 0); + // ingest data + List records = + Lists.newArrayList( + new ThreeColumnRecord(1, "AAAAAAAAAA", "AAAA"), + new ThreeColumnRecord(2, "AAAAAAAAAA", "AAAA"), + new ThreeColumnRecord(3, "AAAAAAAAAA", "AAAA")); + + Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class).coalesce(1); + + df.select("c1", "c2", "c3") + .write() + .format("iceberg") + .mode("append") + .saveAsTable("hive.default." + tableName); + sourceTable.refresh(); + + // generate position delete files + spark.sql(String.format("delete from hive.default.%s where c1 = 1", tableName)); + sourceTable.refresh(); + + List originalData = + rowsToJava( + spark + .read() + .format("iceberg") + .load("hive.default." + tableName) + .sort("c1", "c2", "c3") + .collectAsList()); + // two rows + assertThat(originalData).hasSize(2); + + // copy table and check the results + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .rewriteLocationPrefix(newTableLocation(), targetTableLocation()) + .execute(); + + checkFileNum(3, 2, 2, 9, result); + // one data and one metadata file + copyTableFiles(result); + + // register table + String metadataLocation = currentMetadata(sourceTable).metadataFileLocation(); + String versionFile = fileName(metadataLocation); + String targetTableName = String.format("copiedV%sTable", formatVersion); + TableIdentifier tableIdentifier = TableIdentifier.of("default", targetTableName); + catalog.registerTable(tableIdentifier, targetTableLocation() + "/metadata/" + versionFile); + + List copiedData = + rowsToJava( + spark + .read() + .format("iceberg") + .load("hive.default." + targetTableName) + .sort("c1", "c2", "c3") + .collectAsList()); + + assertEquals("Rows must match", originalData, copiedData); + } + + @TestTemplate + public void testKryoDeserializeBroadcastValues() { + sparkContext.getConf().set("spark.serializer", "org.apache.spark.serializer.KryoSerializer"); + RewriteTablePathSparkAction action = + (RewriteTablePathSparkAction) actions().rewriteTablePath(table); + Broadcast
    tableBroadcast = action.tableBroadcast(); + // force deserializing broadcast values + removeBroadcastValuesFromLocalBlockManager(tableBroadcast.id()); + assertThat(tableBroadcast.getValue().uuid()).isEqualTo(table.uuid()); + } + + @TestTemplate + public void testNestedDirectoryStructurePreservation() throws Exception { + assumeThat(formatVersion) + .as("Can't add multiple DVs for the same data file in v3") + .isEqualTo(2); + String sourceTableLocation = newTableLocation(); + Table sourceTable = createTableWithSnapshots(sourceTableLocation, 1); + + // Create position delete files with same names in different nested directories + // This simulates the scenario tested in + // TestRewriteTablePathUtil.testStagingPathPreservesDirectoryStructure + List> deletes1 = + Lists.newArrayList( + Pair.of( + sourceTable + .currentSnapshot() + .addedDataFiles(sourceTable.io()) + .iterator() + .next() + .location(), + 0L)); + + List> deletes2 = + Lists.newArrayList( + Pair.of( + sourceTable + .currentSnapshot() + .addedDataFiles(sourceTable.io()) + .iterator() + .next() + .location(), + 0L)); + + // Create delete files with same name in different nested paths (hash1/ and hash2/) + File file1 = + new File(removePrefix(sourceTable.location() + "/data/hash1/delete_0_0_0.parquet")); + File file2 = + new File(removePrefix(sourceTable.location() + "/data/hash2/delete_0_0_0.parquet")); + + DeleteFile positionDeletes1 = + FileHelpers.writeDeleteFile( + sourceTable, + sourceTable.io().newOutputFile(file1.toURI().toString()), + deletes1, + formatVersion) + .first(); + + DeleteFile positionDeletes2 = + FileHelpers.writeDeleteFile( + sourceTable, + sourceTable.io().newOutputFile(file2.toURI().toString()), + deletes2, + formatVersion) + .first(); + + sourceTable.newRowDelta().addDeletes(positionDeletes1).commit(); + sourceTable.newRowDelta().addDeletes(positionDeletes2).commit(); + + // Perform rewrite with staging location to test directory structure preservation + RewriteTablePath.Result result = + actions() + .rewriteTablePath(sourceTable) + .stagingLocation(stagingLocation()) + .rewriteLocationPrefix(sourceTableLocation, targetTableLocation()) + .execute(); + + // Copy the files and verify structure is preserved + copyTableFiles(result); + + // Read the file paths from the rewritten result to verify directory structure + List> filePaths = readPathPairList(result.fileListLocation()); + + // Find the delete files in the result + List> deleteFilePaths = + filePaths.stream() + .filter(pair -> pair._2().contains("delete_0_0_0.parquet")) + .collect(Collectors.toList()); + + // Should have 2 delete files with different paths + assertThat(deleteFilePaths).hasSize(2); + + // Verify that the directory structure is preserved in target paths + assertThat(deleteFilePaths) + .anyMatch(pair -> pair._2().contains("/hash1/delete_0_0_0.parquet")) + .anyMatch(pair -> pair._2().contains("/hash2/delete_0_0_0.parquet")); + + // Verify that the files have different target paths (no conflicts) + String targetPath1 = deleteFilePaths.get(0)._2(); + String targetPath2 = deleteFilePaths.get(1)._2(); + assertThat(targetPath1).isNotEqualTo(targetPath2); + + // Verify both target paths start with the target table location + assertThat(targetPath1).startsWith(targetTableLocation()); + assertThat(targetPath2).startsWith(targetTableLocation()); + } + + @TestTemplate + public void testRewritePathWithoutCreateFileList() throws Exception { + String targetTableLocation = targetTableLocation(); + + RewriteTablePath.Result result = + actions() + .rewriteTablePath(table) + .rewriteLocationPrefix(tableLocation, targetTableLocation) + .createFileList(false) // Disable file list creation + .execute(); + + assertThat(result.latestVersion()).isEqualTo("v3.metadata.json"); + + assertThat(result.fileListLocation()) + .as("File list location should not be set when createFileList is false") + .isEqualTo(NOT_APPLICABLE); + } + + protected void checkFileNum( + int versionFileCount, + int manifestListCount, + int manifestFileCount, + int totalCount, + RewriteTablePath.Result result) { + checkFileNum(versionFileCount, manifestListCount, manifestFileCount, 0, 0, totalCount, result); + } + + protected void checkFileNum( + int versionFileCount, + int manifestListCount, + int manifestFileCount, + int statisticsFileCount, + int totalCount, + RewriteTablePath.Result result) { + checkFileNum( + versionFileCount, + manifestListCount, + manifestFileCount, + statisticsFileCount, + 0, + totalCount, + result); + } + + protected void checkFileNum( + int versionFileCount, + int manifestListCount, + int manifestFileCount, + int statisticsFileCount, + int partitionFileCount, + int totalCount, + RewriteTablePath.Result result) { + List filesToMove = + spark + .read() + .format("text") + .load(result.fileListLocation()) + .as(Encoders.STRING()) + .collectAsList(); + Predicate isManifest = + f -> + (f.contains("optimized-m-") && f.endsWith(".avro")) + || f.endsWith("-m0.avro") + || f.endsWith("-m1.avro"); + Predicate isManifestList = f -> f.contains("snap-") && f.endsWith(".avro"); + Predicate isMetadataJSON = f -> f.endsWith(".metadata.json"); + + assertThat(filesToMove.stream().filter(isMetadataJSON)) + .as("Wrong rebuilt version file count") + .hasSize(versionFileCount); + assertThat(filesToMove.stream().filter(isManifestList)) + .as("Wrong rebuilt Manifest list file count") + .hasSize(manifestListCount); + assertThat(filesToMove.stream().filter(isManifest)) + .as("Wrong rebuilt Manifest file file count") + .hasSize(manifestFileCount); + assertThat(filesToMove.stream().filter(f -> f.endsWith(".stats"))) + .as("Wrong rebuilt Statistic file count") + .hasSize(statisticsFileCount); + assertThat(filesToMove.stream().filter(f -> f.contains("partition-stats"))) + .as("Wrong rebuilt Partition Statistic file count") + .hasSize(partitionFileCount); + assertThat(filesToMove).as("Wrong total file count").hasSize(totalCount); + } + + protected String newTableLocation() throws IOException { + return toAbsolute(newTableDir); + } + + protected String targetTableLocation() throws IOException { + return toAbsolute(targetTableDir); + } + + protected String stagingLocation() throws IOException { + return toAbsolute(staging); + } + + protected String toAbsolute(Path relative) { + return relative.toFile().toURI().toString(); + } + + private void copyTableFiles(RewriteTablePath.Result result) throws Exception { + List> filesToMove = readPathPairList(result.fileListLocation()); + + for (Tuple2 pathPair : filesToMove) { + FileUtils.copyFile(new File(URI.create(pathPair._1())), new File(URI.create(pathPair._2()))); + } + } + + private String removePrefix(String path) { + return path.substring(path.lastIndexOf(":") + 1); + } + + protected Table newStaticTable(String metadataFileLocation, FileIO io) { + StaticTableOperations ops = new StaticTableOperations(metadataFileLocation, io); + return new BaseTable(ops, metadataFileLocation); + } + + private List> readPathPairList(String path) { + Encoder> encoder = Encoders.tuple(Encoders.STRING(), Encoders.STRING()); + return spark + .read() + .format("csv") + .schema(encoder.schema()) + .load(path) + .as(encoder) + .collectAsList(); + } + + private Table createMetastoreTable( + String location, + Map properties, + String namespace, + String tableName, + int snapshotNumber) { + return createMetastoreTable(location, properties, namespace, tableName, snapshotNumber, null); + } + + private Table createMetastoreTable( + String location, + Map properties, + String namespace, + String tableName, + int snapshotNumber, + String partitionColumn) { + spark.conf().set("spark.sql.catalog.hive", SparkCatalog.class.getName()); + spark.conf().set("spark.sql.catalog.hive.type", "hive"); + spark.conf().set("spark.sql.catalog.hive.default-namespace", "default"); + spark.conf().set("spark.sql.catalog.hive.cache-enabled", "false"); + + // Generate and execute CREATE TABLE SQL + String createTableSQL = + generateCreateTableSQL(location, properties, namespace, tableName, partitionColumn); + sql(createTableSQL); + + for (int i = 0; i < snapshotNumber; i++) { + sql("insert into hive.%s.%s values (%s, 'AAAAAAAAAA', 'AAAA')", namespace, tableName, i); + } + return catalog.loadTable(TableIdentifier.of(namespace, tableName)); + } + + /** + * Generates SQL statement for creating an Iceberg table + * + * @param location location the storage location path for the table, can be empty + * @param properties key-value pairs of table properties for setting table metadata + * @param namespace the namespace (database name) + * @param tableName the name of the table to be created + * @param partitionColumn the partition column name, must be one of c1, c2, or c3; can be null or + * empty string for non-partitioned table + * @return CREATE TABLE SQL statement string + */ + private String generateCreateTableSQL( + String location, + Map properties, + String namespace, + String tableName, + String partitionColumn) { + StringBuilder propertiesStr = new StringBuilder(); + properties.forEach((k, v) -> propertiesStr.append("'" + k + "'='" + v + "',")); + String tblProperties = + propertiesStr.substring(0, propertiesStr.length() > 0 ? propertiesStr.length() - 1 : 0); + + sql("DROP TABLE IF EXISTS hive.%s.%s", namespace, tableName); + + StringBuilder createTableSql = new StringBuilder(); + createTableSql + .append("CREATE TABLE hive.") + .append(namespace) + .append(".") + .append(tableName) + .append(" (c1 bigint, c2 string, c3 string)"); + + if (partitionColumn != null && !partitionColumn.isEmpty()) { + createTableSql.append(" USING iceberg PARTITIONED BY (").append(partitionColumn).append(")"); + } else { + createTableSql.append(" USING iceberg"); + } + + if (!location.isEmpty()) { + createTableSql.append(" LOCATION '").append(location).append("'"); + } + + if (!tblProperties.isEmpty()) { + createTableSql.append(" TBLPROPERTIES (").append(tblProperties).append(")"); + } + + return createTableSql.toString(); + } + + private static String fileName(String path) { + String filename = path; + int lastIndex = path.lastIndexOf(File.separator); + if (lastIndex != -1) { + filename = path.substring(lastIndex + 1); + } + return filename; + } + + private TableMetadata currentMetadata(Table tbl) { + return ((HasTableOperations) tbl).operations().current(); + } + + private List rows(String location) { + return rowsToJava(spark.read().format("iceberg").load(location).collectAsList()); + } + + private List rowsSorted(String location, String sortCol) { + return rowsToJava(spark.read().format("iceberg").load(location).sort(sortCol).collectAsList()); + } + + private PositionDelete positionDelete( + Schema tableSchema, CharSequence path, Long position, Object... values) { + PositionDelete posDelete = PositionDelete.create(); + GenericRecord nested = GenericRecord.create(tableSchema); + for (int i = 0; i < values.length; i++) { + nested.set(i, values[i]); + } + posDelete.set(path, position, nested); + return posDelete; + } + + private void removeBroadcastValuesFromLocalBlockManager(long id) { + BlockId blockId = new BroadcastBlockId(id, ""); + SparkEnv env = SparkEnv.get(); + env.broadcastManager().cachedValues().clear(); + BlockManager blockManager = env.blockManager(); + BlockInfoManager blockInfoManager = blockManager.blockInfoManager(); + blockInfoManager.lockForWriting(blockId, true); + blockInfoManager.removeBlock(blockId); + blockManager.memoryStore().remove(blockId); + } + + private void findAndAssertFileInFileList( + RewriteTablePath.Result result, + String fileIdentifier, + String sourceTableLocation, + String targetTableLocation) { + + List> filesToMove = readPathPairList(result.fileListLocation()); + + // Find the file path pair that contains the specified file identifier + Tuple2 filePathPair = + filesToMove.stream() + .filter(pair -> pair._1().contains(fileIdentifier)) + .findFirst() + .orElse(null); + + // Assert that the file was found in the list + assertThat(filePathPair).as("Should find " + fileIdentifier + " file in file list").isNotNull(); + + // Validate source path: should point to source table location, contain metadata, and not + // staging + assertThat(filePathPair._1()) + .as(fileIdentifier + " source should point to source table location and NOT staging") + .startsWith(sourceTableLocation) + .contains("/metadata/") + .doesNotContain("staging"); + + // Validate target path: should point to target table location + assertThat(filePathPair._2()) + .as(fileIdentifier + " target should point to target table location") + .startsWith(targetTableLocation); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java new file mode 100644 index 000000000000..d9c42a07b853 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java @@ -0,0 +1,68 @@ +/* + * 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.actions; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.spark.CatalogTestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSnapshotTableAction extends CatalogTestBase { + private static final String SOURCE_NAME = "spark_catalog.default.source"; + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s PURGE", SOURCE_NAME); + } + + @TestTemplate + public void testSnapshotWithParallelTasks() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + + AtomicInteger snapshotThreadsIndex = new AtomicInteger(0); + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .executeWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("table-snapshot-" + snapshotThreadsIndex.getAndIncrement()); + thread.setDaemon(true); + return thread; + })) + .execute(); + assertThat(snapshotThreadsIndex.get()).isEqualTo(2); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriteRunners.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriteRunners.java new file mode 100644 index 000000000000..072b8e8a7eeb --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriteRunners.java @@ -0,0 +1,139 @@ +/* + * 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.actions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +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.TestBase; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.types.Types.StringType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +public class TestSparkFileRewriteRunners extends TestBase { + + private static final TableIdentifier TABLE_IDENT = TableIdentifier.of("default", "tbl"); + private static final Schema SCHEMA = + new Schema( + NestedField.required(1, "id", IntegerType.get()), + NestedField.required(2, "dep", StringType.get())); + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("dep").build(); + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + + @AfterEach + public void removeTable() { + catalog.dropTable(TABLE_IDENT); + } + + @Test + public void testInvalidConstructorUsagesSortData() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + + assertThatThrownBy(() -> new SparkSortFileRewriteRunner(spark, table)) + .hasMessageContaining("Cannot sort data without a valid sort order") + .hasMessageContaining("is unsorted and no sort order is provided"); + + assertThatThrownBy(() -> new SparkSortFileRewriteRunner(spark, table, null)) + .hasMessageContaining("Cannot sort data without a valid sort order") + .hasMessageContaining("the provided sort order is null or empty"); + + assertThatThrownBy(() -> new SparkSortFileRewriteRunner(spark, table, SortOrder.unsorted())) + .hasMessageContaining("Cannot sort data without a valid sort order") + .hasMessageContaining("the provided sort order is null or empty"); + } + + @Test + public void testInvalidConstructorUsagesZOrderData() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA, SPEC); + + assertThatThrownBy(() -> new SparkZOrderFileRewriteRunner(spark, table, null)) + .hasMessageContaining("Cannot ZOrder when no columns are specified"); + + assertThatThrownBy(() -> new SparkZOrderFileRewriteRunner(spark, table, ImmutableList.of())) + .hasMessageContaining("Cannot ZOrder when no columns are specified"); + + assertThatThrownBy( + () -> new SparkZOrderFileRewriteRunner(spark, table, ImmutableList.of("dep"))) + .hasMessageContaining("Cannot ZOrder") + .hasMessageContaining("all columns provided were identity partition columns"); + + assertThatThrownBy( + () -> new SparkZOrderFileRewriteRunner(spark, table, ImmutableList.of("DeP"))) + .hasMessageContaining("Cannot ZOrder") + .hasMessageContaining("all columns provided were identity partition columns"); + } + + @Test + public void testSortDataValidOptions() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + SparkSortFileRewriteRunner rewriter = new SparkSortFileRewriteRunner(spark, table, SORT_ORDER); + + assertThat(rewriter.validOptions()) + .as("Rewriter must report all supported options") + .containsExactlyInAnyOrder(SparkShufflingFileRewriteRunner.SHUFFLE_PARTITIONS_PER_FILE); + } + + @Test + public void testZOrderDataValidOptions() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + ImmutableList zOrderCols = ImmutableList.of("id"); + SparkZOrderFileRewriteRunner rewriter = + new SparkZOrderFileRewriteRunner(spark, table, zOrderCols); + + assertThat(rewriter.validOptions()) + .as("Rewriter must report all supported options") + .containsExactlyInAnyOrder( + SparkZOrderFileRewriteRunner.SHUFFLE_PARTITIONS_PER_FILE, + SparkZOrderFileRewriteRunner.MAX_OUTPUT_SIZE, + SparkZOrderFileRewriteRunner.VAR_LENGTH_CONTRIBUTION); + } + + @Test + public void testInvalidValuesForZOrderDataOptions() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + ImmutableList zOrderCols = ImmutableList.of("id"); + SparkZOrderFileRewriteRunner rewriter = + new SparkZOrderFileRewriteRunner(spark, table, zOrderCols); + + Map invalidMaxOutputOptions = + ImmutableMap.of(SparkZOrderFileRewriteRunner.MAX_OUTPUT_SIZE, "0"); + assertThatThrownBy(() -> rewriter.init(invalidMaxOutputOptions)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot have the interleaved ZOrder value use less than 1 byte") + .hasMessageContaining("'max-output-size' was set to 0"); + + Map invalidVarLengthContributionOptions = + ImmutableMap.of(SparkZOrderFileRewriteRunner.VAR_LENGTH_CONTRIBUTION, "0"); + assertThatThrownBy(() -> rewriter.init(invalidVarLengthContributionOptions)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot use less than 1 byte for variable length types with ZOrder") + .hasMessageContaining("'var-length-contribution' was set to 0"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkShufflingDataRewritePlanner.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkShufflingDataRewritePlanner.java new file mode 100644 index 000000000000..9fcf9f5ec51a --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkShufflingDataRewritePlanner.java @@ -0,0 +1,93 @@ +/* + * 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.actions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Map; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewriteDataFiles; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.types.Types.IntegerType; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.types.Types.StringType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +public class TestSparkShufflingDataRewritePlanner extends TestBase { + + private static final TableIdentifier TABLE_IDENT = TableIdentifier.of("default", "tbl"); + private static final Schema SCHEMA = + new Schema( + NestedField.required(1, "id", IntegerType.get()), + NestedField.required(2, "dep", StringType.get())); + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("dep").build(); + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + + @AfterEach + public void removeTable() { + catalog.dropTable(TABLE_IDENT); + } + + @Test + void testSparkShufflingDataRewritePlannerValidOptions() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + SparkShufflingDataRewritePlanner planner = + new SparkShufflingDataRewritePlanner( + table, Expressions.alwaysTrue(), null, false /* caseSensitive */); + + assertThat(planner.validOptions()) + .as("Planner must report all supported options") + .isEqualTo( + ImmutableSet.of( + SparkShufflingDataRewritePlanner.COMPRESSION_FACTOR, + SparkShufflingDataRewritePlanner.TARGET_FILE_SIZE_BYTES, + SparkShufflingDataRewritePlanner.MIN_FILE_SIZE_BYTES, + SparkShufflingDataRewritePlanner.MAX_FILE_SIZE_BYTES, + SparkShufflingDataRewritePlanner.MIN_INPUT_FILES, + SparkShufflingDataRewritePlanner.REWRITE_ALL, + SparkShufflingDataRewritePlanner.MAX_FILE_GROUP_SIZE_BYTES, + SparkShufflingDataRewritePlanner.DELETE_FILE_THRESHOLD, + SparkShufflingDataRewritePlanner.DELETE_RATIO_THRESHOLD, + RewriteDataFiles.REWRITE_JOB_ORDER, + SparkShufflingDataRewritePlanner.MAX_FILES_TO_REWRITE)); + } + + @Test + void testInvalidValuesSparkShufflingDataRewritePlannerOptions() { + Table table = catalog.createTable(TABLE_IDENT, SCHEMA); + SparkShufflingDataRewritePlanner planner = + new SparkShufflingDataRewritePlanner( + table, Expressions.alwaysTrue(), null, false /* caseSensitive */); + + Map invalidCompressionFactorOptions = + ImmutableMap.of(SparkShufflingDataRewritePlanner.COMPRESSION_FACTOR, "0"); + assertThatThrownBy(() -> planner.init(invalidCompressionFactorOptions)) + .hasMessageContaining("'compression-factor' is set to 0.0 but must be > 0"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java new file mode 100644 index 000000000000..8ce60f6275e5 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/AvroDataTestBase.java @@ -0,0 +1,728 @@ +/* + * 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.data; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.LongType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.util.DateTimeUtil; +import org.assertj.core.api.Condition; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.FieldSource; +import org.junit.jupiter.params.provider.MethodSource; + +public abstract class AvroDataTestBase { + + private static final long FIRST_ROW_ID = 2_000L; + protected static final Map ID_TO_CONSTANT = + Map.of( + MetadataColumns.ROW_ID.fieldId(), + FIRST_ROW_ID, + MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId(), + 34L); + + protected abstract void writeAndValidate(Schema schema) throws IOException; + + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + throw new UnsupportedEncodingException( + "Cannot run test, writeAndValidate(Schema, Schema) is not implemented"); + } + + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema, List records) + throws IOException { + throw new UnsupportedEncodingException( + "Cannot run test, writeAndValidate(Schema, Schema, List) is not implemented"); + } + + protected boolean supportsDefaultValues() { + return false; + } + + protected boolean supportsNestedTypes() { + return true; + } + + protected boolean supportsRowLineage() { + return false; + } + + protected static final StructType SUPPORTED_PRIMITIVES = + StructType.of( + required(100, "id", LongType.get()), + optional(101, "data", Types.StringType.get()), + required(102, "b", Types.BooleanType.get()), + optional(103, "i", Types.IntegerType.get()), + required(104, "l", LongType.get()), + optional(105, "f", Types.FloatType.get()), + required(106, "d", Types.DoubleType.get()), + optional(107, "date", Types.DateType.get()), + required(108, "ts", Types.TimestampType.withZone()), + required(109, "ts_without_zone", Types.TimestampType.withoutZone()), + required(110, "s", Types.StringType.get()), + required(111, "uuid", Types.UUIDType.get()), + required(112, "fixed", Types.FixedType.ofLength(7)), + optional(113, "bytes", Types.BinaryType.get()), + required(114, "dec_9_0", Types.DecimalType.of(9, 0)), // int encoded + required(115, "dec_11_2", Types.DecimalType.of(11, 2)), // long encoded + required(116, "dec_20_5", Types.DecimalType.of(20, 5)), // requires padding + required(117, "dec_38_10", Types.DecimalType.of(38, 10)), // Spark's maximum precision + optional(118, "unk", Types.UnknownType.get())); + + @TempDir protected Path temp; + + private static final Type[] SIMPLE_TYPES = + new Type[] { + Types.UnknownType.get(), + Types.BooleanType.get(), + Types.IntegerType.get(), + LongType.get(), + Types.FloatType.get(), + Types.DoubleType.get(), + Types.DateType.get(), + Types.TimeType.get(), + Types.TimestampType.withZone(), + Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.StringType.get(), + Types.FixedType.ofLength(7), + Types.BinaryType.get(), + Types.DecimalType.of(9, 0), + Types.DecimalType.of(11, 2), + Types.DecimalType.of(38, 10), + Types.VariantType.get(), + Types.GeometryType.crs84(), + Types.GeographyType.crs84(), + }; + + protected boolean supportsTime() { + return false; + } + + protected boolean supportsTimestampNanos() { + return false; + } + + protected boolean supportsVariant() { + return false; + } + + protected boolean supportsGeospatial() { + return false; + } + + @ParameterizedTest + @FieldSource("SIMPLE_TYPES") + public void testTypeSchema(Type type) throws IOException { + assumeThat(supportsTime() || TypeUtil.find(type, t -> t.typeId() == Type.TypeID.TIME) == null) + .as("Spark does not support time fields") + .isTrue(); + assumeThat( + supportsTimestampNanos() + || TypeUtil.find(type, t -> t.typeId() == Type.TypeID.TIMESTAMP_NANO) == null) + .as("timestamp_ns is not yet implemented") + .isTrue(); + assumeThat( + supportsVariant() + || TypeUtil.find(type, t -> t.typeId() == Type.TypeID.VARIANT) == null) + .as("variant is not yet implemented") + .isTrue(); + if (!supportsGeospatial()) { + assumeThat(TypeUtil.find(type, t -> t.typeId() == Type.TypeID.GEOMETRY) == null) + .as("geometry is not yet implemented") + .isTrue(); + assumeThat(TypeUtil.find(type, t -> t.typeId() == Type.TypeID.GEOGRAPHY) == null) + .as("geography is not yet implemented") + .isTrue(); + } + + writeAndValidate( + new Schema( + required(1, "id", LongType.get()), + optional(2, "test_type", type), + required(3, "trailing_data", Types.StringType.get()))); + } + + @Test + public void testSimpleStruct() throws IOException { + writeAndValidate(TypeUtil.assignIncreasingFreshIds(new Schema(SUPPORTED_PRIMITIVES.fields()))); + } + + @Test + public void testStructWithRequiredFields() throws IOException { + List supportedPrimitives = + SUPPORTED_PRIMITIVES.fields().stream() + .filter(f -> f.type().typeId() != Type.TypeID.UNKNOWN) + .collect(Collectors.toList()); + writeAndValidate( + TypeUtil.assignIncreasingFreshIds( + new Schema(Lists.transform(supportedPrimitives, Types.NestedField::asRequired)))); + } + + @Test + public void testStructWithOptionalFields() throws IOException { + writeAndValidate( + TypeUtil.assignIncreasingFreshIds( + new Schema( + Lists.transform(SUPPORTED_PRIMITIVES.fields(), Types.NestedField::asOptional)))); + } + + @Test + public void testNestedStruct() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + writeAndValidate( + TypeUtil.assignIncreasingFreshIds(new Schema(required(1, "struct", SUPPORTED_PRIMITIVES)))); + } + + @Test + public void testArray() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + new Schema( + required(0, "id", LongType.get()), + optional(1, "data", ListType.ofOptional(2, Types.StringType.get()))); + + writeAndValidate(schema); + } + + @Test + public void testArrayOfStructs() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + TypeUtil.assignIncreasingFreshIds( + new Schema( + required(0, "id", LongType.get()), + optional(1, "data", ListType.ofOptional(2, SUPPORTED_PRIMITIVES)))); + + writeAndValidate(schema); + } + + @Test + public void testMap() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + new Schema( + required(0, "id", LongType.get()), + optional( + 1, + "data", + MapType.ofOptional(2, 3, Types.StringType.get(), Types.StringType.get()))); + + writeAndValidate(schema); + } + + @Test + public void testNumericMapKey() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + new Schema( + required(0, "id", LongType.get()), + optional( + 1, "data", MapType.ofOptional(2, 3, Types.LongType.get(), Types.StringType.get()))); + + writeAndValidate(schema); + } + + @Test + public void testComplexMapKey() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + new Schema( + required(0, "id", LongType.get()), + optional( + 1, + "data", + MapType.ofOptional( + 2, + 3, + Types.StructType.of( + required(4, "i", Types.IntegerType.get()), + optional(5, "s", Types.StringType.get())), + Types.StringType.get()))); + + writeAndValidate(schema); + } + + @Test + public void testMapOfStructs() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + TypeUtil.assignIncreasingFreshIds( + new Schema( + required(0, "id", LongType.get()), + optional( + 1, + "data", + MapType.ofOptional(2, 3, Types.StringType.get(), SUPPORTED_PRIMITIVES)))); + + writeAndValidate(schema); + } + + @Test + public void testMixedTypes() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + StructType structType = + StructType.of( + required(0, "id", LongType.get()), + optional( + 1, + "list_of_maps", + ListType.ofOptional( + 2, MapType.ofOptional(3, 4, Types.StringType.get(), SUPPORTED_PRIMITIVES))), + optional( + 5, + "map_of_lists", + MapType.ofOptional( + 6, 7, Types.StringType.get(), ListType.ofOptional(8, SUPPORTED_PRIMITIVES))), + required( + 9, + "list_of_lists", + ListType.ofOptional(10, ListType.ofOptional(11, SUPPORTED_PRIMITIVES))), + required( + 12, + "map_of_maps", + MapType.ofOptional( + 13, + 14, + Types.StringType.get(), + MapType.ofOptional(15, 16, Types.StringType.get(), SUPPORTED_PRIMITIVES))), + required( + 17, + "list_of_struct_of_nested_types", + ListType.ofOptional( + 19, + StructType.of( + Types.NestedField.required( + 20, + "m1", + MapType.ofOptional( + 21, 22, Types.StringType.get(), SUPPORTED_PRIMITIVES)), + Types.NestedField.optional( + 23, "l1", ListType.ofRequired(24, SUPPORTED_PRIMITIVES)), + Types.NestedField.required( + 25, "l2", ListType.ofRequired(26, SUPPORTED_PRIMITIVES)), + Types.NestedField.optional( + 27, + "m2", + MapType.ofOptional( + 28, 29, Types.StringType.get(), SUPPORTED_PRIMITIVES)))))); + + Schema schema = + new Schema( + TypeUtil.assignFreshIds(structType, new AtomicInteger(0)::incrementAndGet) + .asStructType() + .fields()); + + writeAndValidate(schema); + } + + @Test + public void testMissingRequiredWithoutDefault() { + assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = new Schema(required(1, "id", Types.LongType.get())); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.required("missing_str") + .withId(6) + .ofType(Types.StringType.get()) + .withDoc("Missing required field with no default") + .build()); + + assertThatThrownBy(() -> writeAndValidate(writeSchema, expectedSchema)) + .has( + new Condition<>( + t -> + IllegalArgumentException.class.isInstance(t) + || IllegalArgumentException.class.isInstance(t.getCause()), + "Expecting a throwable or cause that is an instance of IllegalArgumentException")) + .hasMessageContaining("Missing required field: missing_str"); + } + + @Test + public void testDefaultValues() throws IOException { + assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault(Literal.of("wrong!")) + .withDoc("Should not produce default value") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault(Literal.of("wrong!")) + .build(), + Types.NestedField.required("missing_str") + .withId(6) + .ofType(Types.StringType.get()) + .withInitialDefault(Literal.of("orange")) + .build(), + Types.NestedField.optional("missing_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(Literal.of(34)) + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testNullDefaultValue() throws IOException { + assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault(Literal.of("wrong!")) + .withDoc("Should not produce default value") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault(Literal.of("wrong!")) + .build(), + Types.NestedField.optional("missing_date") + .withId(3) + .ofType(Types.DateType.get()) + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testNestedDefaultValue() throws IOException { + assumeThat(supportsDefaultValues()).isTrue(); + assumeThat(supportsNestedTypes()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault(Literal.of("wrong!")) + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested") + .withId(3) + .ofType(Types.StructType.of(required(4, "inner", Types.StringType.get()))) + .withDoc("Used to test nested field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault(Literal.of("wrong!")) + .build(), + Types.NestedField.optional("nested") + .withId(3) + .ofType( + Types.StructType.of( + required(4, "inner", Types.StringType.get()), + Types.NestedField.optional("missing_inner_float") + .withId(5) + .ofType(Types.FloatType.get()) + .withInitialDefault(Literal.of(-0.0F)) + .build())) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testMapNestedDefaultValue() throws IOException { + assumeThat(supportsDefaultValues()).isTrue(); + assumeThat(supportsNestedTypes()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault(Literal.of("wrong!")) + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested_map") + .withId(3) + .ofType( + Types.MapType.ofOptional( + 4, + 5, + Types.StringType.get(), + Types.StructType.of(required(6, "value_str", Types.StringType.get())))) + .withDoc("Used to test nested map value field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault(Literal.of("wrong!")) + .build(), + Types.NestedField.optional("nested_map") + .withId(3) + .ofType( + Types.MapType.ofOptional( + 4, + 5, + Types.StringType.get(), + Types.StructType.of( + required(6, "value_str", Types.StringType.get()), + Types.NestedField.optional("value_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(Literal.of(34)) + .build()))) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + @Test + public void testListNestedDefaultValue() throws IOException { + assumeThat(supportsDefaultValues()).isTrue(); + assumeThat(supportsNestedTypes()).isTrue(); + + Schema writeSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault(Literal.of("wrong!")) + .withDoc("Should not produce default value") + .build(), + Types.NestedField.optional("nested_list") + .withId(3) + .ofType( + Types.ListType.ofOptional( + 4, Types.StructType.of(required(5, "element_str", Types.StringType.get())))) + .withDoc("Used to test nested field defaults") + .build()); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withInitialDefault(Literal.of("wrong!")) + .build(), + Types.NestedField.optional("nested_list") + .withId(3) + .ofType( + Types.ListType.ofOptional( + 4, + Types.StructType.of( + required(5, "element_str", Types.StringType.get()), + Types.NestedField.optional("element_int") + .withId(7) + .ofType(Types.IntegerType.get()) + .withInitialDefault(Literal.of(34)) + .build()))) + .withDoc("Used to test nested field defaults") + .build()); + + writeAndValidate(writeSchema, expectedSchema); + } + + private static Stream primitiveTypesAndDefaults() { + return Stream.of( + Arguments.of(Types.BooleanType.get(), Literal.of(false)), + Arguments.of(Types.IntegerType.get(), Literal.of(34)), + Arguments.of(Types.LongType.get(), Literal.of(4900000000L)), + Arguments.of(Types.FloatType.get(), Literal.of(12.21F)), + Arguments.of(Types.DoubleType.get(), Literal.of(-0.0D)), + Arguments.of(Types.DateType.get(), Literal.of(DateTimeUtil.isoDateToDays("2024-12-17"))), + // Arguments.of(Types.TimeType.get(), DateTimeUtil.isoTimeToMicros("23:59:59.999999")), + Arguments.of( + Types.TimestampType.withZone(), + Literal.of(DateTimeUtil.isoTimestamptzToMicros("2024-12-17T23:59:59.999999+00:00"))), + Arguments.of( + Types.TimestampType.withoutZone(), + Literal.of(DateTimeUtil.isoTimestampToMicros("2024-12-17T23:59:59.999999"))), + Arguments.of(Types.StringType.get(), Literal.of("iceberg")), + Arguments.of(Types.UUIDType.get(), Literal.of(UUID.randomUUID())), + Arguments.of( + Types.FixedType.ofLength(4), + Literal.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d}))), + Arguments.of(Types.BinaryType.get(), Literal.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b}))), + Arguments.of(Types.DecimalType.of(9, 2), Literal.of(new BigDecimal("12.34")))); + } + + @ParameterizedTest + @MethodSource("primitiveTypesAndDefaults") + public void testPrimitiveTypeDefaultValues(Type.PrimitiveType type, Literal defaultValue) + throws IOException { + assumeThat(supportsDefaultValues()).isTrue(); + + Schema writeSchema = new Schema(required(1, "id", Types.LongType.get())); + + Schema readSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.optional("col_with_default") + .withId(2) + .ofType(type) + .withInitialDefault(defaultValue) + .build()); + + writeAndValidate(writeSchema, readSchema); + } + + @Test + public void testRowLineage() throws Exception { + assumeThat(supportsRowLineage()).as("Row lineage support is not implemented").isTrue(); + + Schema schema = + new Schema( + required(1, "id", LongType.get()), + required(2, "data", Types.StringType.get()), + MetadataColumns.ROW_ID, + MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER); + + GenericRecord record = GenericRecord.create(schema); + + writeAndValidate( + schema, + schema, + List.of( + record.copy(Map.of("id", 1L, "data", "a")), + record.copy(Map.of("id", 2L, "data", "b")), + record.copy( + Map.of( + "id", + 3L, + "data", + "c", + "_row_id", + 1_000L, + "_last_updated_sequence_number", + 33L)), + record.copy(Map.of("id", 4L, "data", "d", "_row_id", 1_001L)), + record.copy(Map.of("id", 5L, "data", "e")))); + } + + @Test + public void testUnknownNestedLevel() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + new Schema( + required(1, "id", LongType.get()), + optional( + 2, + "nested", + Types.StructType.of( + required(20, "int", Types.IntegerType.get()), + optional(21, "unk", Types.UnknownType.get())))); + + writeAndValidate(schema); + } + + @Test + public void testUnknownListType() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + new Schema( + required(0, "id", LongType.get()), + optional(1, "data", ListType.ofOptional(2, Types.UnknownType.get()))); + + writeAndValidate(schema); + } + + @Test + public void testUnknownMapType() throws IOException { + assumeThat(supportsNestedTypes()).isTrue(); + + Schema schema = + new Schema( + required(0, "id", LongType.get()), + optional( + 1, + "data", + MapType.ofOptional(2, 3, Types.StringType.get(), Types.UnknownType.get()))); + + writeAndValidate(schema); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java new file mode 100644 index 000000000000..fecb8275c240 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java @@ -0,0 +1,441 @@ +/* + * 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.data; + +import static org.apache.iceberg.spark.SparkSchemaUtil.convert; +import static org.assertj.core.api.Assertions.assertThat; +import static scala.collection.JavaConverters.mapAsJavaMapConverter; +import static scala.collection.JavaConverters.seqAsJavaListConverter; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.Collection; +import java.util.Date; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.data.GenericDataUtil; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.variants.Variant; +import org.apache.iceberg.variants.VariantMetadata; +import org.apache.iceberg.variants.VariantTestUtil; +import org.apache.iceberg.variants.VariantValue; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.apache.spark.unsafe.types.UTF8String; +import org.apache.spark.unsafe.types.VariantVal; +import scala.collection.Seq; + +public class GenericsHelpers { + private GenericsHelpers() {} + + private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC); + private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); + + public static void assertEqualsSafe(Types.StructType struct, Record expected, Row actual) { + Types.StructType expectedType = expected.struct(); + List fields = struct.fields(); + for (int readPos = 0; readPos < fields.size(); readPos += 1) { + Type fieldType = fields.get(readPos).type(); + Types.NestedField field = fields.get(readPos); + Types.NestedField expectedField = expectedType.field(field.fieldId()); + + Object actualValue = actual.get(readPos); + + Object expectedValue; + if (expectedField != null) { + expectedValue = expected.getField(expectedField.name()); + } else { + expectedValue = GenericDataUtil.internalToGeneric(field.type(), field.initialDefault()); + } + + assertEqualsSafe(fieldType, expectedValue, actualValue); + } + } + + public static void assertEqualsBatch( + Types.StructType struct, + Iterator expectedRecords, + ColumnarBatch batch, + Map idToConstant, + Integer batchFirstRowPos) { + for (int rowPos = 0; rowPos < batch.numRows(); rowPos++) { + InternalRow row = batch.getRow(rowPos); + Record expectedRecord = expectedRecords.next(); + assertEqualsUnsafe(struct, expectedRecord, row, idToConstant, batchFirstRowPos + rowPos); + } + } + + private static void assertEqualsSafe( + Types.ListType list, Collection expected, List actual) { + Type elementType = list.elementType(); + List expectedElements = Lists.newArrayList(expected); + for (int i = 0; i < expectedElements.size(); i += 1) { + Object expectedValue = expectedElements.get(i); + Object actualValue = actual.get(i); + + assertEqualsSafe(elementType, expectedValue, actualValue); + } + } + + private static void assertEqualsSafe(Types.MapType map, Map expected, Map actual) { + Type keyType = map.keyType(); + Type valueType = map.valueType(); + assertThat(actual.keySet()) + .as("Should have the same number of keys") + .hasSameSizeAs(expected.keySet()); + + for (Object expectedKey : expected.keySet()) { + Object matchingKey = null; + for (Object actualKey : actual.keySet()) { + try { + assertEqualsSafe(keyType, expectedKey, actualKey); + matchingKey = actualKey; + break; + } catch (AssertionError e) { + // failed + } + } + + assertThat(matchingKey).as("Should have a matching key").isNotNull(); + assertEqualsSafe(valueType, expected.get(expectedKey), actual.get(matchingKey)); + } + } + + @SuppressWarnings("unchecked") + private static void assertEqualsSafe(Type type, Object expected, Object actual) { + if (expected == null && actual == null) { + return; + } + + switch (type.typeId()) { + case BOOLEAN: + case INTEGER: + case LONG: + case FLOAT: + case DOUBLE: + assertThat(actual).as("Primitive value should be equal to expected").isEqualTo(expected); + break; + case DATE: + assertThat(expected).as("Should expect a LocalDate").isInstanceOf(LocalDate.class); + assertThat(actual) + .isInstanceOf(Date.class) + .as("ISO-8601 date should be equal") + .asString() + .isEqualTo(String.valueOf(expected)); + break; + case TIMESTAMP: + Types.TimestampType timestampType = (Types.TimestampType) type; + if (timestampType.shouldAdjustToUTC()) { + // Timestamptz + assertThat(actual).as("Should be a Timestamp").isInstanceOf(Timestamp.class); + Timestamp ts = (Timestamp) actual; + // milliseconds from nanos has already been added by getTime + OffsetDateTime actualTs = + EPOCH.plusNanos((ts.getTime() * 1_000_000) + (ts.getNanos() % 1_000_000)); + + assertThat(expected) + .as("Should expect an OffsetDateTime") + .isInstanceOf(OffsetDateTime.class); + + assertThat(actualTs).as("Timestamp should be equal").isEqualTo(expected); + } else { + // Timestamp + assertThat(actual).as("Should be a LocalDateTime").isInstanceOf(LocalDateTime.class); + + assertThat(expected) + .as("Should expect an LocalDateTime") + .isInstanceOf(LocalDateTime.class); + + assertThat(actual).as("Timestamp should be equal").isEqualTo(expected); + } + break; + case STRING: + assertThat(actual) + .isInstanceOf(String.class) + .asString() + .isEqualTo(String.valueOf(expected)); + break; + case UUID: + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + assertThat(actual) + .isInstanceOf(String.class) + .asString() + .isEqualTo(String.valueOf(expected)); + break; + case FIXED: + assertThat(expected).as("Should expect a byte[]").isInstanceOf(byte[].class); + assertThat(actual).isInstanceOf(byte[].class).isEqualTo(expected); + break; + case BINARY: + assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); + assertThat(actual).isInstanceOf(byte[].class).isEqualTo(((ByteBuffer) expected).array()); + break; + case DECIMAL: + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(actual).isInstanceOf(BigDecimal.class).isEqualTo(expected); + break; + case STRUCT: + assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); + assertThat(actual).as("Should be a Row").isInstanceOf(Row.class); + assertEqualsSafe(type.asNestedType().asStructType(), (Record) expected, (Row) actual); + break; + case LIST: + assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); + assertThat(actual).as("Should be a Seq").isInstanceOf(Seq.class); + List asList = seqAsJavaListConverter((Seq) actual).asJava(); + assertEqualsSafe(type.asNestedType().asListType(), (Collection) expected, asList); + break; + case MAP: + assertThat(expected).as("Should expect a Collection").isInstanceOf(Map.class); + assertThat(actual).as("Should be a Map").isInstanceOf(scala.collection.Map.class); + Map asMap = + mapAsJavaMapConverter((scala.collection.Map) actual).asJava(); + assertEqualsSafe(type.asNestedType().asMapType(), (Map) expected, asMap); + break; + case VARIANT: + assertThat(expected).as("Should expect a Variant").isInstanceOf(Variant.class); + assertThat(actual).as("Should be a VariantVal").isInstanceOf(VariantVal.class); + assertEquals((Variant) expected, (VariantVal) actual); + break; + case TIME: + default: + throw new IllegalArgumentException("Not a supported type: " + type); + } + } + + public static void assertEqualsUnsafe( + Types.StructType struct, Record expected, InternalRow actual) { + assertEqualsUnsafe(struct, expected, actual, null, -1); + } + + public static void assertEqualsUnsafe( + Types.StructType struct, + Record expected, + InternalRow actual, + Map idToConstant, + int pos) { + Types.StructType expectedType = expected.struct(); + List fields = struct.fields(); + for (int readPos = 0; readPos < fields.size(); readPos += 1) { + Types.NestedField field = fields.get(readPos); + Types.NestedField expectedField = expectedType.field(field.fieldId()); + + Type fieldType = field.type(); + Object actualValue = + actual.isNullAt(readPos) ? null : actual.get(readPos, convert(fieldType)); + + Object expectedValue; + if (expectedField != null) { + int id = expectedField.fieldId(); + if (id == MetadataColumns.ROW_ID.fieldId()) { + expectedValue = expected.getField(expectedField.name()); + if (expectedValue == null && idToConstant != null) { + expectedValue = (Long) idToConstant.get(id) + pos; + } + + } else if (id == MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId()) { + expectedValue = expected.getField(expectedField.name()); + if (expectedValue == null && idToConstant != null) { + expectedValue = idToConstant.get(id); + } + + } else { + expectedValue = expected.getField(expectedField.name()); + } + } else { + // comparison expects Iceberg's generic representation + expectedValue = GenericDataUtil.internalToGeneric(field.type(), field.initialDefault()); + } + + assertEqualsUnsafe(fieldType, expectedValue, actualValue); + } + } + + private static void assertEqualsUnsafe( + Types.ListType list, Collection expected, ArrayData actual) { + Type elementType = list.elementType(); + List expectedElements = Lists.newArrayList(expected); + for (int i = 0; i < expectedElements.size(); i += 1) { + Object expectedValue = expectedElements.get(i); + Object actualValue = actual.get(i, convert(elementType)); + + assertEqualsUnsafe(elementType, expectedValue, actualValue); + } + } + + private static void assertEqualsUnsafe(Types.MapType map, Map expected, MapData actual) { + Type keyType = map.keyType(); + Type valueType = map.valueType(); + + List> expectedElements = Lists.newArrayList(expected.entrySet()); + ArrayData actualKeys = actual.keyArray(); + ArrayData actualValues = actual.valueArray(); + + for (int i = 0; i < expectedElements.size(); i += 1) { + Map.Entry expectedPair = expectedElements.get(i); + Object actualKey = actualKeys.get(i, convert(keyType)); + Object actualValue = actualValues.get(i, convert(keyType)); + + assertEqualsUnsafe(keyType, expectedPair.getKey(), actualKey); + assertEqualsUnsafe(valueType, expectedPair.getValue(), actualValue); + } + } + + static void assertEquals(Variant expected, VariantVal actual) { + VariantMetadata actualMetadata = + VariantMetadata.from(ByteBuffer.wrap(actual.getMetadata()).order(ByteOrder.LITTLE_ENDIAN)); + VariantTestUtil.assertEqual(expected.metadata(), actualMetadata); + VariantTestUtil.assertEqual( + expected.value(), + VariantValue.from( + actualMetadata, ByteBuffer.wrap(actual.getValue()).order(ByteOrder.LITTLE_ENDIAN))); + } + + private static void assertEqualsUnsafe(Type type, Object expected, Object actual) { + if (expected == null && actual == null) { + return; + } + + switch (type.typeId()) { + case LONG: + assertThat(actual).as("Should be a long").isInstanceOf(Long.class); + if (expected instanceof Integer) { + assertThat(actual).as("Values didn't match").isEqualTo(((Number) expected).longValue()); + } else { + assertThat(actual).as("Primitive value should be equal to expected").isEqualTo(expected); + } + break; + case DOUBLE: + assertThat(actual).as("Should be a double").isInstanceOf(Double.class); + if (expected instanceof Float) { + assertThat(Double.doubleToLongBits((double) actual)) + .as("Values didn't match") + .isEqualTo(Double.doubleToLongBits(((Number) expected).doubleValue())); + } else { + assertThat(actual).as("Primitive value should be equal to expected").isEqualTo(expected); + } + break; + case BOOLEAN: + case INTEGER: + case FLOAT: + assertThat(actual).as("Primitive value should be equal to expected").isEqualTo(expected); + break; + case DATE: + assertThat(expected).as("Should expect a LocalDate").isInstanceOf(LocalDate.class); + int expectedDays = (int) ChronoUnit.DAYS.between(EPOCH_DAY, (LocalDate) expected); + assertThat(actual) + .as("Primitive value should be equal to expected") + .isEqualTo(expectedDays); + break; + case TIMESTAMP: + Types.TimestampType timestampType = (Types.TimestampType) type; + if (timestampType.shouldAdjustToUTC()) { + assertThat(expected) + .as("Should expect an OffsetDateTime") + .isInstanceOf(OffsetDateTime.class); + long expectedMicros = ChronoUnit.MICROS.between(EPOCH, (OffsetDateTime) expected); + assertThat(actual) + .as("Primitive value should be equal to expected") + .isEqualTo(expectedMicros); + } else { + assertThat(expected) + .as("Should expect an LocalDateTime") + .isInstanceOf(LocalDateTime.class); + long expectedMicros = + ChronoUnit.MICROS.between(EPOCH, ((LocalDateTime) expected).atZone(ZoneId.of("UTC"))); + assertThat(actual) + .as("Primitive value should be equal to expected") + .isEqualTo(expectedMicros); + } + break; + case STRING: + assertThat(actual) + .isInstanceOf(UTF8String.class) + .asString() + .isEqualTo(String.valueOf(expected)); + break; + case UUID: + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + assertThat(actual) + .isInstanceOf(UTF8String.class) + .asString() + .isEqualTo(String.valueOf(expected)); + break; + case FIXED: + assertThat(expected).as("Should expect a byte[]").isInstanceOf(byte[].class); + assertThat(actual).isInstanceOf(byte[].class).isEqualTo(expected); + break; + case BINARY: + assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); + assertThat(actual).isInstanceOf(byte[].class).isEqualTo(((ByteBuffer) expected).array()); + break; + case DECIMAL: + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(actual).as("Should be a Decimal").isInstanceOf(Decimal.class); + assertThat(((Decimal) actual).toJavaBigDecimal()) + .as("BigDecimals should be equal") + .isEqualTo(expected); + break; + case STRUCT: + assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); + assertThat(actual).as("Should be an InternalRow").isInstanceOf(InternalRow.class); + assertEqualsUnsafe( + type.asNestedType().asStructType(), (Record) expected, (InternalRow) actual); + break; + case LIST: + assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); + assertThat(actual).as("Should be an ArrayData").isInstanceOf(ArrayData.class); + assertEqualsUnsafe( + type.asNestedType().asListType(), (Collection) expected, (ArrayData) actual); + break; + case MAP: + assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); + assertThat(actual).as("Should be an ArrayBasedMapData").isInstanceOf(MapData.class); + assertEqualsUnsafe(type.asNestedType().asMapType(), (Map) expected, (MapData) actual); + break; + case VARIANT: + assertThat(expected).as("Should expect a Variant").isInstanceOf(Variant.class); + assertThat(actual).as("Should be a VariantVal").isInstanceOf(VariantVal.class); + assertEquals((Variant) expected, (VariantVal) actual); + break; + case TIME: + default: + throw new IllegalArgumentException("Not a supported type: " + type); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java new file mode 100644 index 000000000000..ae30d5ec88db --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java @@ -0,0 +1,404 @@ +/* + * 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.data; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.function.Supplier; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.RandomVariants; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; +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.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.RandomUtil; +import org.apache.iceberg.variants.Variant; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData; +import org.apache.spark.sql.catalyst.util.GenericArrayData; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.unsafe.types.UTF8String; +import org.apache.spark.unsafe.types.VariantVal; + +public class RandomData { + + // Default percentage of number of values that are null for optional fields + public static final float DEFAULT_NULL_PERCENTAGE = 0.05f; + + private RandomData() {} + + public static List generateList(Schema schema, int numRecords, long seed) { + RandomDataGenerator generator = new RandomDataGenerator(schema, seed, DEFAULT_NULL_PERCENTAGE); + List records = Lists.newArrayListWithExpectedSize(numRecords); + for (int i = 0; i < numRecords; i += 1) { + records.add((Record) TypeUtil.visit(schema, generator)); + } + + return records; + } + + public static Iterable generateSpark(Schema schema, int numRecords, long seed) { + return () -> + new Iterator() { + private final SparkRandomDataGenerator generator = new SparkRandomDataGenerator(seed); + private int count = 0; + + @Override + public boolean hasNext() { + return count < numRecords; + } + + @Override + public InternalRow next() { + if (count >= numRecords) { + throw new NoSuchElementException(); + } + count += 1; + return (InternalRow) TypeUtil.visit(schema, generator); + } + }; + } + + public static Iterable generate(Schema schema, int numRecords, long seed) { + return newIterable( + () -> new RandomDataGenerator(schema, seed, DEFAULT_NULL_PERCENTAGE), schema, numRecords); + } + + public static Iterable generate( + Schema schema, int numRecords, long seed, float nullPercentage) { + return newIterable( + () -> new RandomDataGenerator(schema, seed, nullPercentage), schema, numRecords); + } + + public static Iterable generateFallbackData( + Schema schema, int numRecords, long seed, long numDictRecords) { + return newIterable( + () -> new FallbackDataGenerator(schema, seed, numDictRecords), schema, numRecords); + } + + public static Iterable generateDictionaryEncodableData( + Schema schema, int numRecords, long seed, float nullPercentage) { + return newIterable( + () -> new DictionaryEncodedDataGenerator(schema, seed, nullPercentage), schema, numRecords); + } + + private static Iterable newIterable( + Supplier newGenerator, Schema schema, int numRecords) { + return () -> + new Iterator() { + private int count = 0; + private final RandomDataGenerator generator = newGenerator.get(); + + @Override + public boolean hasNext() { + return count < numRecords; + } + + @Override + public Record next() { + if (count >= numRecords) { + throw new NoSuchElementException(); + } + count += 1; + return (Record) TypeUtil.visit(schema, generator); + } + }; + } + + private static class RandomDataGenerator extends TypeUtil.CustomOrderSchemaVisitor { + private final Map typeToSchema; + private final Random random; + // Percentage of number of values that are null for optional fields + private final float nullPercentage; + + private RandomDataGenerator(Schema schema, long seed, float nullPercentage) { + Preconditions.checkArgument( + 0.0f <= nullPercentage && nullPercentage <= 1.0f, + "Percentage needs to be in the range (0.0, 1.0)"); + this.nullPercentage = nullPercentage; + this.typeToSchema = AvroSchemaUtil.convertTypes(schema.asStruct(), "test"); + this.random = new Random(seed); + } + + @Override + public Record schema(Schema schema, Supplier structResult) { + return (Record) structResult.get(); + } + + @Override + public Record struct(Types.StructType struct, Iterable fieldResults) { + Record rec = new Record(typeToSchema.get(struct)); + + List values = Lists.newArrayList(fieldResults); + for (int i = 0; i < values.size(); i += 1) { + rec.put(i, values.get(i)); + } + + return rec; + } + + @Override + public Object field(Types.NestedField field, Supplier fieldResult) { + if (field.isOptional() && isNull()) { + return null; + } + return fieldResult.get(); + } + + private boolean isNull() { + return random.nextFloat() < nullPercentage; + } + + @Override + public Object list(Types.ListType list, Supplier elementResult) { + int numElements = random.nextInt(20); + + List result = Lists.newArrayListWithExpectedSize(numElements); + for (int i = 0; i < numElements; i += 1) { + if (list.isElementOptional() && isNull()) { + result.add(null); + } else { + result.add(elementResult.get()); + } + } + + return result; + } + + @Override + public Object map(Types.MapType map, Supplier keyResult, Supplier valueResult) { + int numEntries = random.nextInt(20); + + Map result = Maps.newLinkedHashMap(); + Set keySet = Sets.newHashSet(); + for (int i = 0; i < numEntries; i += 1) { + Object key = keyResult.get(); + // ensure no collisions + while (keySet.contains(key)) { + key = keyResult.get(); + } + + keySet.add(key); + + if (map.isValueOptional() && isNull()) { + result.put(key, null); + } else { + result.put(key, valueResult.get()); + } + } + + return result; + } + + public Object variant(Types.VariantType variant) { + return RandomVariants.randomVariant(random); + } + + @Override + public Object primitive(Type.PrimitiveType primitive) { + Object result = randomValue(primitive, random); + // For the primitives that Avro needs a different type than Spark, fix + // them here. + switch (primitive.typeId()) { + case FIXED: + return new GenericData.Fixed(typeToSchema.get(primitive), (byte[]) result); + case BINARY: + return ByteBuffer.wrap((byte[]) result); + case UUID: + return UUID.nameUUIDFromBytes((byte[]) result); + default: + return result; + } + } + + protected Object randomValue(Type.PrimitiveType primitive, Random rand) { + return RandomUtil.generatePrimitive(primitive, random); + } + } + + private static class SparkRandomDataGenerator extends TypeUtil.CustomOrderSchemaVisitor { + private final Random random; + private final float nullPercentage; + + private SparkRandomDataGenerator(long seed) { + this(seed, DEFAULT_NULL_PERCENTAGE); + } + + private SparkRandomDataGenerator(long seed, float nullPercentage) { + Preconditions.checkArgument( + 0.0f <= nullPercentage && nullPercentage <= 1.0f, + "Percentage needs to be in the range (0.0, 1.0)"); + this.random = new Random(seed); + this.nullPercentage = nullPercentage; + } + + @Override + public InternalRow schema(Schema schema, Supplier structResult) { + return (InternalRow) structResult.get(); + } + + @Override + public InternalRow struct(Types.StructType struct, Iterable fieldResults) { + List values = Lists.newArrayList(fieldResults); + GenericInternalRow row = new GenericInternalRow(values.size()); + for (int i = 0; i < values.size(); i += 1) { + row.update(i, values.get(i)); + } + + return row; + } + + @Override + public Object field(Types.NestedField field, Supplier fieldResult) { + // return null 5% of the time when the value is optional + if (field.isOptional() && isNull()) { + return null; + } + return fieldResult.get(); + } + + private boolean isNull() { + return random.nextFloat() < nullPercentage; + } + + @Override + public GenericArrayData list(Types.ListType list, Supplier elementResult) { + int numElements = random.nextInt(20); + Object[] arr = new Object[numElements]; + GenericArrayData result = new GenericArrayData(arr); + + for (int i = 0; i < numElements; i += 1) { + if (list.isElementOptional() && isNull()) { + arr[i] = null; + } else { + arr[i] = elementResult.get(); + } + } + + return result; + } + + @Override + public Object map(Types.MapType map, Supplier keyResult, Supplier valueResult) { + int numEntries = random.nextInt(20); + + Object[] keysArr = new Object[numEntries]; + Object[] valuesArr = new Object[numEntries]; + GenericArrayData keys = new GenericArrayData(keysArr); + GenericArrayData values = new GenericArrayData(valuesArr); + ArrayBasedMapData result = new ArrayBasedMapData(keys, values); + + Set keySet = Sets.newHashSet(); + for (int i = 0; i < numEntries; i += 1) { + Object key = keyResult.get(); + // ensure no collisions + while (keySet.contains(key)) { + key = keyResult.get(); + } + + keySet.add(key); + + keysArr[i] = key; + if (map.isValueOptional() && isNull()) { + valuesArr[i] = null; + } else { + valuesArr[i] = valueResult.get(); + } + } + + return result; + } + + @Override + public VariantVal variant(Types.VariantType type) { + Variant variant = RandomVariants.randomVariant(random); + + byte[] metadataBytes = new byte[variant.metadata().sizeInBytes()]; + ByteBuffer metadataBuffer = ByteBuffer.wrap(metadataBytes).order(ByteOrder.LITTLE_ENDIAN); + variant.metadata().writeTo(metadataBuffer, 0); + + byte[] valueBytes = new byte[variant.value().sizeInBytes()]; + ByteBuffer valueBuffer = ByteBuffer.wrap(valueBytes).order(ByteOrder.LITTLE_ENDIAN); + variant.value().writeTo(valueBuffer, 0); + + return new VariantVal(valueBytes, metadataBytes); + } + + @Override + public Object primitive(Type.PrimitiveType primitive) { + Object obj = RandomUtil.generatePrimitive(primitive, random); + switch (primitive.typeId()) { + case STRING: + return UTF8String.fromString((String) obj); + case DECIMAL: + return Decimal.apply((BigDecimal) obj); + case UUID: + return UTF8String.fromString(UUID.nameUUIDFromBytes((byte[]) obj).toString()); + default: + return obj; + } + } + } + + private static class DictionaryEncodedDataGenerator extends RandomDataGenerator { + private DictionaryEncodedDataGenerator(Schema schema, long seed, float nullPercentage) { + super(schema, seed, nullPercentage); + } + + @Override + protected Object randomValue(Type.PrimitiveType primitive, Random random) { + return RandomUtil.generateDictionaryEncodablePrimitive(primitive, random); + } + } + + private static class FallbackDataGenerator extends RandomDataGenerator { + private final long dictionaryEncodedRows; + private long rowCount = 0; + + private FallbackDataGenerator(Schema schema, long seed, long numDictionaryEncoded) { + super(schema, seed, DEFAULT_NULL_PERCENTAGE); + this.dictionaryEncodedRows = numDictionaryEncoded; + } + + @Override + protected Object randomValue(Type.PrimitiveType primitive, Random rand) { + this.rowCount += 1; + if (rowCount > dictionaryEncodedRows) { + return RandomUtil.generatePrimitive(primitive, rand); + } else { + return RandomUtil.generateDictionaryEncodablePrimitive(primitive, rand); + } + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java new file mode 100644 index 000000000000..91606d16a0a0 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -0,0 +1,957 @@ +/* + * 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.data; + +import static org.apache.iceberg.spark.SparkSchemaUtil.convert; +import static org.assertj.core.api.Assertions.assertThat; +import static scala.collection.JavaConverters.mapAsJavaMapConverter; +import static scala.collection.JavaConverters.seqAsJavaListConverter; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.Collection; +import java.util.Date; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; +import org.apache.avro.Schema.Field; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.deletes.DeleteCounter; +import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.util.DeleteFileSet; +import org.apache.iceberg.variants.Variant; +import org.apache.orc.storage.serde2.io.DateWritable; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericRow; +import org.apache.spark.sql.catalyst.expressions.SpecializedGetters; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.BinaryType; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.apache.spark.unsafe.types.UTF8String; +import org.apache.spark.unsafe.types.VariantVal; +import scala.collection.Seq; + +public class TestHelpers { + + private TestHelpers() {} + + public static void assertEqualsSafe(Types.StructType struct, List recs, List rows) { + Streams.forEachPair( + recs.stream(), rows.stream(), (rec, row) -> assertEqualsSafe(struct, rec, row)); + } + + public static void assertEqualsSafe(Types.StructType struct, Record rec, Row row) { + List fields = struct.fields(); + for (int readPos = 0; readPos < fields.size(); readPos += 1) { + Types.NestedField field = fields.get(readPos); + Field writeField = rec.getSchema().getField(field.name()); + + Type fieldType = field.type(); + Object actualValue = row.get(readPos); + + Object expectedValue; + if (writeField != null) { + int writePos = writeField.pos(); + expectedValue = rec.get(writePos); + } else { + expectedValue = field.initialDefault(); + } + + assertEqualsSafe(fieldType, expectedValue, actualValue); + } + } + + public static void assertEqualsBatchWithRows( + Types.StructType struct, Iterator expected, ColumnarBatch batch) { + for (int rowId = 0; rowId < batch.numRows(); rowId++) { + List fields = struct.fields(); + InternalRow row = batch.getRow(rowId); + Row expectedRow = expected.next(); + for (int i = 0; i < fields.size(); i += 1) { + Type fieldType = fields.get(i).type(); + Object expectedValue = expectedRow.get(i); + Object actualValue = row.isNullAt(i) ? null : row.get(i, convert(fieldType)); + assertEqualsUnsafe(fieldType, expectedValue, actualValue); + } + } + } + + private static void assertEqualsSafe(Types.ListType list, Collection expected, List actual) { + Type elementType = list.elementType(); + List expectedElements = Lists.newArrayList(expected); + for (int i = 0; i < expectedElements.size(); i += 1) { + Object expectedValue = expectedElements.get(i); + Object actualValue = actual.get(i); + + assertEqualsSafe(elementType, expectedValue, actualValue); + } + } + + private static void assertEqualsSafe(Types.MapType map, Map expected, Map actual) { + Type keyType = map.keyType(); + Type valueType = map.valueType(); + + for (Object expectedKey : expected.keySet()) { + Object matchingKey = null; + for (Object actualKey : actual.keySet()) { + try { + assertEqualsSafe(keyType, expectedKey, actualKey); + matchingKey = actualKey; + } catch (AssertionError e) { + // failed + } + } + + assertThat(matchingKey).as("Should have a matching key").isNotNull(); + assertEqualsSafe(valueType, expected.get(expectedKey), actual.get(matchingKey)); + } + } + + private static final OffsetDateTime EPOCH = Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC); + private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); + + @SuppressWarnings("unchecked") + private static void assertEqualsSafe(Type type, Object expected, Object actual) { + if (expected == null && actual == null) { + return; + } + + switch (type.typeId()) { + case BOOLEAN: + case INTEGER: + case LONG: + case FLOAT: + case DOUBLE: + assertThat(actual).as("Primitive value should be equal to expected").isEqualTo(expected); + break; + case DATE: + assertThat(expected).as("Should be an int").isInstanceOf(Integer.class); + assertThat(actual).as("Should be a Date").isInstanceOf(Date.class); + LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, (Integer) expected); + assertThat(actual) + .as("ISO-8601 date should be equal") + .asString() + .isEqualTo(String.valueOf(date)); + break; + case TIMESTAMP: + Types.TimestampType timestampType = (Types.TimestampType) type; + + assertThat(expected).as("Should be a long").isInstanceOf(Long.class); + if (timestampType.shouldAdjustToUTC()) { + assertThat(actual).as("Should be a Timestamp").isInstanceOf(Timestamp.class); + + Timestamp ts = (Timestamp) actual; + // milliseconds from nanos has already been added by getTime + long tsMicros = (ts.getTime() * 1000) + ((ts.getNanos() / 1000) % 1000); + assertThat(tsMicros).as("Timestamp micros should be equal").isEqualTo(expected); + } else { + assertThat(actual).as("Should be a LocalDateTime").isInstanceOf(LocalDateTime.class); + + LocalDateTime ts = (LocalDateTime) actual; + Instant instant = ts.toInstant(ZoneOffset.UTC); + // milliseconds from nanos has already been added by getTime + long tsMicros = (instant.toEpochMilli() * 1000) + ((ts.getNano() / 1000) % 1000); + assertThat(tsMicros).as("Timestamp micros should be equal").isEqualTo(expected); + } + break; + case STRING: + assertThat(actual).isInstanceOf(String.class).isEqualTo(String.valueOf(expected)); + break; + case UUID: + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + assertThat(actual) + .isInstanceOf(String.class) + .asString() + .isEqualTo(String.valueOf(expected)); + break; + case FIXED: + // generated data is written using Avro or Parquet/Avro so generated rows use + // GenericData.Fixed, but default values are converted from Iceberg's internal + // representation so the expected value may be either class. + byte[] expectedBytes; + if (expected instanceof ByteBuffer) { + expectedBytes = ByteBuffers.toByteArray((ByteBuffer) expected); + } else if (expected instanceof GenericData.Fixed) { + expectedBytes = ((GenericData.Fixed) expected).bytes(); + } else { + throw new IllegalStateException( + "Invalid expected value, not byte[] or Fixed: " + expected); + } + + assertThat(actual).isInstanceOf(byte[].class).isEqualTo(expectedBytes); + break; + case BINARY: + assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); + assertThat(actual).isInstanceOf(byte[].class).isEqualTo(((ByteBuffer) expected).array()); + break; + case DECIMAL: + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(actual).isInstanceOf(BigDecimal.class).isEqualTo(expected); + break; + case STRUCT: + assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); + assertThat(actual).as("Should be a Row").isInstanceOf(Row.class); + assertEqualsSafe(type.asNestedType().asStructType(), (Record) expected, (Row) actual); + break; + case LIST: + assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); + assertThat(actual).as("Should be a Seq").isInstanceOf(Seq.class); + List asList = seqAsJavaListConverter((Seq) actual).asJava(); + assertEqualsSafe(type.asNestedType().asListType(), (Collection) expected, asList); + break; + case MAP: + assertThat(expected).as("Should expect a Collection").isInstanceOf(Map.class); + assertThat(actual).as("Should be a Map").isInstanceOf(scala.collection.Map.class); + Map asMap = + mapAsJavaMapConverter((scala.collection.Map) actual).asJava(); + assertEqualsSafe(type.asNestedType().asMapType(), (Map) expected, asMap); + break; + case TIME: + default: + throw new IllegalArgumentException("Not a supported type: " + type); + } + } + + public static void assertEqualsUnsafe(Types.StructType struct, Record rec, InternalRow row) { + List fields = struct.fields(); + for (int readPos = 0; readPos < fields.size(); readPos += 1) { + Types.NestedField field = fields.get(readPos); + Field writeField = rec.getSchema().getField(field.name()); + + Type fieldType = field.type(); + Object actualValue = row.isNullAt(readPos) ? null : row.get(readPos, convert(fieldType)); + + Object expectedValue; + if (writeField != null) { + int writePos = writeField.pos(); + expectedValue = rec.get(writePos); + } else { + expectedValue = field.initialDefault(); + } + + assertEqualsUnsafe(fieldType, expectedValue, actualValue); + } + } + + private static void assertEqualsUnsafe( + Types.ListType list, Collection expected, ArrayData actual) { + Type elementType = list.elementType(); + List expectedElements = Lists.newArrayList(expected); + for (int i = 0; i < expectedElements.size(); i += 1) { + Object expectedValue = expectedElements.get(i); + Object actualValue = actual.get(i, convert(elementType)); + + assertEqualsUnsafe(elementType, expectedValue, actualValue); + } + } + + private static void assertEqualsUnsafe(Types.MapType map, Map expected, MapData actual) { + Type keyType = map.keyType(); + Type valueType = map.valueType(); + + List> expectedElements = Lists.newArrayList(expected.entrySet()); + ArrayData actualKeys = actual.keyArray(); + ArrayData actualValues = actual.valueArray(); + + for (int i = 0; i < expectedElements.size(); i += 1) { + Map.Entry expectedPair = expectedElements.get(i); + Object actualKey = actualKeys.get(i, convert(keyType)); + Object actualValue = actualValues.get(i, convert(keyType)); + + assertEqualsUnsafe(keyType, expectedPair.getKey(), actualKey); + assertEqualsUnsafe(valueType, expectedPair.getValue(), actualValue); + } + } + + private static void assertEqualsUnsafe(Type type, Object expected, Object actual) { + if (expected == null && actual == null) { + return; + } + + switch (type.typeId()) { + case LONG: + assertThat(actual).as("Should be a long").isInstanceOf(Long.class); + if (expected instanceof Integer) { + assertThat(actual).as("Values didn't match").isEqualTo(((Number) expected).longValue()); + } else { + assertThat(actual).as("Primitive value should be equal to expected").isEqualTo(expected); + } + break; + case DOUBLE: + assertThat(actual).as("Should be a double").isInstanceOf(Double.class); + if (expected instanceof Float) { + assertThat(Double.doubleToLongBits((double) actual)) + .as("Values didn't match") + .isEqualTo(Double.doubleToLongBits(((Number) expected).doubleValue())); + } else { + assertThat(actual).as("Primitive value should be equal to expected").isEqualTo(expected); + } + break; + case INTEGER: + case FLOAT: + case BOOLEAN: + case DATE: + case TIMESTAMP: + assertThat(actual).as("Primitive value should be equal to expected").isEqualTo(expected); + break; + case STRING: + assertThat(actual).isInstanceOf(UTF8String.class).asString().isEqualTo(expected); + break; + case UUID: + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + assertThat(actual) + .isInstanceOf(UTF8String.class) + .asString() + .isEqualTo(String.valueOf(expected)); + break; + case FIXED: + // generated data is written using Avro or Parquet/Avro so generated rows use + // GenericData.Fixed, but default values are converted from Iceberg's internal + // representation so the expected value may be either class. + byte[] expectedBytes; + if (expected instanceof ByteBuffer) { + expectedBytes = ByteBuffers.toByteArray((ByteBuffer) expected); + } else if (expected instanceof GenericData.Fixed) { + expectedBytes = ((GenericData.Fixed) expected).bytes(); + } else { + throw new IllegalStateException( + "Invalid expected value, not byte[] or Fixed: " + expected); + } + + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(actual).as("Bytes should match").isEqualTo(expectedBytes); + break; + case BINARY: + assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); + assertThat(actual).isInstanceOf(byte[].class).isEqualTo(((ByteBuffer) expected).array()); + break; + case DECIMAL: + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(actual).as("Should be a Decimal").isInstanceOf(Decimal.class); + assertThat(((Decimal) actual).toJavaBigDecimal()) + .as("BigDecimals should be equal") + .isEqualTo(expected); + break; + case STRUCT: + assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); + assertThat(actual).as("Should be an InternalRow").isInstanceOf(InternalRow.class); + assertEqualsUnsafe( + type.asNestedType().asStructType(), (Record) expected, (InternalRow) actual); + break; + case LIST: + assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); + assertThat(actual).as("Should be an ArrayData").isInstanceOf(ArrayData.class); + assertEqualsUnsafe( + type.asNestedType().asListType(), (Collection) expected, (ArrayData) actual); + break; + case MAP: + assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); + assertThat(actual).as("Should be an ArrayBasedMapData").isInstanceOf(MapData.class); + assertEqualsUnsafe(type.asNestedType().asMapType(), (Map) expected, (MapData) actual); + break; + case VARIANT: + assertThat(expected).as("Should expect a Variant").isInstanceOf(Variant.class); + assertThat(actual).as("Should be a VariantVal").isInstanceOf(VariantVal.class); + GenericsHelpers.assertEquals((Variant) expected, (VariantVal) actual); + break; + case TIME: + default: + throw new IllegalArgumentException("Not a supported type: " + type); + } + } + + /** + * Check that the given InternalRow is equivalent to the Row. + * + * @param prefix context for error messages + * @param type the type of the row + * @param expected the expected value of the row + * @param actual the actual value of the row + */ + public static void assertEquals( + String prefix, Types.StructType type, InternalRow expected, Row actual) { + if (expected == null || actual == null) { + assertThat(actual).as(prefix).isEqualTo(expected); + } else { + List fields = type.fields(); + for (int c = 0; c < fields.size(); ++c) { + String fieldName = fields.get(c).name(); + Type childType = fields.get(c).type(); + switch (childType.typeId()) { + case BOOLEAN: + case INTEGER: + case LONG: + case FLOAT: + case DOUBLE: + case STRING: + case DECIMAL: + case DATE: + case TIMESTAMP: + assertThat(getPrimitiveValue(actual, c, childType)) + .as(prefix + "." + fieldName + " - " + childType) + .isEqualTo(getValue(expected, c, childType)); + break; + case UUID: + case FIXED: + case BINARY: + assertEqualBytes( + prefix + "." + fieldName, + (byte[]) getValue(expected, c, childType), + (byte[]) actual.get(c)); + break; + case STRUCT: + { + Types.StructType st = (Types.StructType) childType; + assertEquals( + prefix + "." + fieldName, + st, + expected.getStruct(c, st.fields().size()), + actual.getStruct(c)); + break; + } + case LIST: + assertEqualsLists( + prefix + "." + fieldName, + childType.asListType(), + expected.getArray(c), + toList((Seq) actual.get(c))); + break; + case MAP: + assertEqualsMaps( + prefix + "." + fieldName, + childType.asMapType(), + expected.getMap(c), + toJavaMap((scala.collection.Map) actual.getMap(c))); + break; + default: + throw new IllegalArgumentException("Unhandled type " + childType); + } + } + } + } + + private static void assertEqualsLists( + String prefix, Types.ListType type, ArrayData expected, List actual) { + if (expected == null || actual == null) { + assertThat(actual).as(prefix).isEqualTo(expected); + } else { + assertThat(actual).as(prefix + "length").hasSize(expected.numElements()); + Type childType = type.elementType(); + for (int e = 0; e < expected.numElements(); ++e) { + switch (childType.typeId()) { + case BOOLEAN: + case INTEGER: + case LONG: + case FLOAT: + case DOUBLE: + case STRING: + case DECIMAL: + case DATE: + case TIMESTAMP: + assertThat(actual) + .as(prefix + ".elem " + e + " - " + childType) + .element(e) + .isEqualTo(getValue(expected, e, childType)); + break; + case UUID: + case FIXED: + case BINARY: + assertEqualBytes( + prefix + ".elem " + e, + (byte[]) getValue(expected, e, childType), + (byte[]) actual.get(e)); + break; + case STRUCT: + { + Types.StructType st = (Types.StructType) childType; + assertEquals( + prefix + ".elem " + e, + st, + expected.getStruct(e, st.fields().size()), + (Row) actual.get(e)); + break; + } + case LIST: + assertEqualsLists( + prefix + ".elem " + e, + childType.asListType(), + expected.getArray(e), + toList((Seq) actual.get(e))); + break; + case MAP: + assertEqualsMaps( + prefix + ".elem " + e, + childType.asMapType(), + expected.getMap(e), + toJavaMap((scala.collection.Map) actual.get(e))); + break; + default: + throw new IllegalArgumentException("Unhandled type " + childType); + } + } + } + } + + private static void assertEqualsMaps( + String prefix, Types.MapType type, MapData expected, Map actual) { + if (expected == null || actual == null) { + assertThat(actual).as(prefix).isEqualTo(expected); + } else { + Type keyType = type.keyType(); + Type valueType = type.valueType(); + ArrayData expectedKeyArray = expected.keyArray(); + ArrayData expectedValueArray = expected.valueArray(); + assertThat(actual).as(prefix + " length").hasSize(expectedKeyArray.numElements()); + for (int e = 0; e < expected.numElements(); ++e) { + Object expectedKey = getValue(expectedKeyArray, e, keyType); + Object actualValue = actual.get(expectedKey); + if (actualValue == null) { + assertThat(expected.valueArray().isNullAt(e)) + .as(prefix + ".key=" + expectedKey + " has null") + .isTrue(); + } else { + switch (valueType.typeId()) { + case BOOLEAN: + case INTEGER: + case LONG: + case FLOAT: + case DOUBLE: + case STRING: + case DECIMAL: + case DATE: + case TIMESTAMP: + assertThat(actual.get(expectedKey)) + .as(prefix + ".key=" + expectedKey + " - " + valueType) + .isEqualTo(getValue(expectedValueArray, e, valueType)); + break; + case UUID: + case FIXED: + case BINARY: + assertEqualBytes( + prefix + ".key=" + expectedKey, + (byte[]) getValue(expectedValueArray, e, valueType), + (byte[]) actual.get(expectedKey)); + break; + case STRUCT: + { + Types.StructType st = (Types.StructType) valueType; + assertEquals( + prefix + ".key=" + expectedKey, + st, + expectedValueArray.getStruct(e, st.fields().size()), + (Row) actual.get(expectedKey)); + break; + } + case LIST: + assertEqualsLists( + prefix + ".key=" + expectedKey, + valueType.asListType(), + expectedValueArray.getArray(e), + toList((Seq) actual.get(expectedKey))); + break; + case MAP: + assertEqualsMaps( + prefix + ".key=" + expectedKey, + valueType.asMapType(), + expectedValueArray.getMap(e), + toJavaMap((scala.collection.Map) actual.get(expectedKey))); + break; + default: + throw new IllegalArgumentException("Unhandled type " + valueType); + } + } + } + } + } + + private static Object getValue(SpecializedGetters container, int ord, Type type) { + if (container.isNullAt(ord)) { + return null; + } + switch (type.typeId()) { + case BOOLEAN: + return container.getBoolean(ord); + case INTEGER: + return container.getInt(ord); + case LONG: + return container.getLong(ord); + case FLOAT: + return container.getFloat(ord); + case DOUBLE: + return container.getDouble(ord); + case STRING: + return container.getUTF8String(ord).toString(); + case BINARY: + case FIXED: + case UUID: + return container.getBinary(ord); + case DATE: + return new DateWritable(container.getInt(ord)).get(); + case TIMESTAMP: + return DateTimeUtils.toJavaTimestamp(container.getLong(ord)); + case DECIMAL: + { + Types.DecimalType dt = (Types.DecimalType) type; + return container.getDecimal(ord, dt.precision(), dt.scale()).toJavaBigDecimal(); + } + case STRUCT: + Types.StructType struct = type.asStructType(); + InternalRow internalRow = container.getStruct(ord, struct.fields().size()); + Object[] data = new Object[struct.fields().size()]; + for (int i = 0; i < data.length; i += 1) { + if (internalRow.isNullAt(i)) { + data[i] = null; + } else { + data[i] = getValue(internalRow, i, struct.fields().get(i).type()); + } + } + return new GenericRow(data); + default: + throw new IllegalArgumentException("Unhandled type " + type); + } + } + + private static Object getPrimitiveValue(Row row, int ord, Type type) { + if (row.isNullAt(ord)) { + return null; + } + switch (type.typeId()) { + case BOOLEAN: + return row.getBoolean(ord); + case INTEGER: + return row.getInt(ord); + case LONG: + return row.getLong(ord); + case FLOAT: + return row.getFloat(ord); + case DOUBLE: + return row.getDouble(ord); + case STRING: + return row.getString(ord); + case BINARY: + case FIXED: + case UUID: + return row.get(ord); + case DATE: + return row.getDate(ord); + case TIMESTAMP: + return row.getTimestamp(ord); + case DECIMAL: + return row.getDecimal(ord); + default: + throw new IllegalArgumentException("Unhandled type " + type); + } + } + + private static Map toJavaMap(scala.collection.Map map) { + return map == null ? null : mapAsJavaMapConverter(map).asJava(); + } + + private static List toList(Seq val) { + return val == null ? null : seqAsJavaListConverter(val).asJava(); + } + + private static void assertEqualBytes(String context, byte[] expected, byte[] actual) { + assertThat(actual).as(context).isEqualTo(expected); + } + + static void assertEquals(Schema schema, Object expected, Object actual) { + assertEquals("schema", convert(schema), expected, actual); + } + + private static void assertEquals(String context, DataType type, Object expected, Object actual) { + if (expected == null && actual == null) { + return; + } + + if (type instanceof StructType) { + assertThat(expected) + .as("Expected should be an InternalRow: " + context) + .isInstanceOf(InternalRow.class); + assertThat(actual) + .as("Actual should be an InternalRow: " + context) + .isInstanceOf(InternalRow.class); + assertEquals(context, (StructType) type, (InternalRow) expected, (InternalRow) actual); + + } else if (type instanceof ArrayType) { + assertThat(expected) + .as("Expected should be an ArrayData: " + context) + .isInstanceOf(ArrayData.class); + assertThat(actual) + .as("Actual should be an ArrayData: " + context) + .isInstanceOf(ArrayData.class); + assertEquals(context, (ArrayType) type, (ArrayData) expected, (ArrayData) actual); + + } else if (type instanceof MapType) { + assertThat(expected) + .as("Expected should be a MapData: " + context) + .isInstanceOf(MapData.class); + assertThat(actual).as("Actual should be a MapData: " + context).isInstanceOf(MapData.class); + assertEquals(context, (MapType) type, (MapData) expected, (MapData) actual); + + } else if (type instanceof BinaryType) { + assertEqualBytes(context, (byte[]) expected, (byte[]) actual); + } else { + assertThat(actual).as("Value should match expected: " + context).isEqualTo(expected); + } + } + + private static void assertEquals( + String context, StructType struct, InternalRow expected, InternalRow actual) { + assertThat(actual.numFields()) + .as("Should have correct number of fields") + .isEqualTo(struct.size()); + for (int i = 0; i < actual.numFields(); i += 1) { + StructField field = struct.fields()[i]; + DataType type = field.dataType(); + + assertEquals( + context + "." + field.name(), + type, + expected.isNullAt(i) ? null : expected.get(i, type), + actual.isNullAt(i) ? null : actual.get(i, type)); + } + } + + private static void assertEquals( + String context, ArrayType array, ArrayData expected, ArrayData actual) { + assertThat(actual.numElements()) + .as("Should have the same number of elements") + .isEqualTo(expected.numElements()); + DataType type = array.elementType(); + for (int i = 0; i < actual.numElements(); i += 1) { + assertEquals( + context + ".element", + type, + expected.isNullAt(i) ? null : expected.get(i, type), + actual.isNullAt(i) ? null : actual.get(i, type)); + } + } + + private static void assertEquals(String context, MapType map, MapData expected, MapData actual) { + assertThat(actual.numElements()) + .as("Should have the same number of elements") + .isEqualTo(expected.numElements()); + + DataType keyType = map.keyType(); + ArrayData expectedKeys = expected.keyArray(); + ArrayData expectedValues = expected.valueArray(); + + DataType valueType = map.valueType(); + ArrayData actualKeys = actual.keyArray(); + ArrayData actualValues = actual.valueArray(); + + for (int i = 0; i < actual.numElements(); i += 1) { + assertEquals( + context + ".key", + keyType, + expectedKeys.isNullAt(i) ? null : expectedKeys.get(i, keyType), + actualKeys.isNullAt(i) ? null : actualKeys.get(i, keyType)); + assertEquals( + context + ".value", + valueType, + expectedValues.isNullAt(i) ? null : expectedValues.get(i, valueType), + actualValues.isNullAt(i) ? null : actualValues.get(i, valueType)); + } + } + + public static List dataManifests(Table table) { + return table.currentSnapshot().dataManifests(table.io()); + } + + public static List deleteManifests(Table table) { + return table.currentSnapshot().deleteManifests(table.io()); + } + + public static List dataFiles(Table table) { + return dataFiles(table, null); + } + + public static List dataFiles(Table table, String branch) { + TableScan scan = table.newScan(); + if (branch != null) { + scan = scan.useRef(branch); + } + + CloseableIterable tasks = scan.includeColumnStats().planFiles(); + return Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + } + + public static Set deleteFiles(Table table) { + DeleteFileSet deleteFiles = DeleteFileSet.create(); + + for (FileScanTask task : table.newScan().planFiles()) { + deleteFiles.addAll(task.deletes()); + } + + return deleteFiles; + } + + public static Set deleteFiles(Table table, Snapshot snapshot) { + DeleteFileSet deleteFiles = DeleteFileSet.create(); + + for (FileScanTask task : table.newScan().useSnapshot(snapshot.snapshotId()).planFiles()) { + deleteFiles.addAll(task.deletes()); + } + + return deleteFiles; + } + + public static Set reachableManifestPaths(Table table) { + return StreamSupport.stream(table.snapshots().spliterator(), false) + .flatMap(s -> s.allManifests(table.io()).stream()) + .map(ManifestFile::path) + .collect(Collectors.toSet()); + } + + public static void asMetadataRecord(GenericData.Record file, FileContent content) { + file.put(0, content.id()); + file.put(3, 0); // specId + } + + public static void asMetadataRecord(GenericData.Record file) { + file.put(0, FileContent.DATA.id()); + file.put(3, 0); // specId + } + + // suppress the readable metrics and first-row-id that are not in manifest files + private static final Set DERIVED_FIELDS = Set.of("readable_metrics", "first_row_id"); + + public static Dataset selectNonDerived(Dataset metadataTable) { + StructField[] fields = metadataTable.schema().fields(); + return metadataTable.select( + Stream.of(fields) + .filter(f -> !DERIVED_FIELDS.contains(f.name())) + .map(f -> new Column(f.name())) + .toArray(Column[]::new)); + } + + public static Types.StructType nonDerivedSchema(Dataset metadataTable) { + return SparkSchemaUtil.convert(TestHelpers.selectNonDerived(metadataTable).schema()).asStruct(); + } + + public static class CustomizedDeleteFilter extends DeleteFilter { + private final boolean hasDeletes; + + protected CustomizedDeleteFilter( + boolean hasDeletes, Schema tableSchema, Schema projectedSchema) { + super("", List.of(), tableSchema, projectedSchema, new DeleteCounter(), true); + this.hasDeletes = hasDeletes; + } + + @Override + protected StructLike asStructLike(InternalRow record) { + return null; + } + + @Override + protected InputFile getInputFile(String location) { + return null; + } + + @Override + public boolean hasPosDeletes() { + return hasDeletes; + } + + @Override + public PositionDeleteIndex deletedRowPositions() { + PositionDeleteIndex deletedRowPos = new CustomizedPositionDeleteIndex(); + if (hasDeletes) { + deletedRowPos.delete(98, 103); + } + + return deletedRowPos; + } + } + + public static class CustomizedPositionDeleteIndex implements PositionDeleteIndex { + private final Set deleteIndex; + + private CustomizedPositionDeleteIndex() { + deleteIndex = Sets.newHashSet(); + } + + @Override + public void delete(long position) { + deleteIndex.add(position); + } + + @Override + public void delete(long posStart, long posEnd) { + for (long l = posStart; l < posEnd; l++) { + delete(l); + } + } + + @Override + public boolean isDeleted(long position) { + return deleteIndex.contains(position); + } + + @Override + public boolean isEmpty() { + return deleteIndex.isEmpty(); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestOrcWrite.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestOrcWrite.java new file mode 100644 index 000000000000..e149e57e8144 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestOrcWrite.java @@ -0,0 +1,59 @@ +/* + * 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.data; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestOrcWrite { + @TempDir private Path temp; + + private static final Schema SCHEMA = + new Schema( + optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); + + @Test + public void splitOffsets() throws IOException { + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + + Iterable rows = RandomData.generateSpark(SCHEMA, 1, 0L); + FileAppender writer = + ORC.write(Files.localOutput(testFile)) + .createWriterFunc(SparkOrcWriter::new) + .schema(SCHEMA) + .build(); + + writer.addAll(rows); + writer.close(); + assertThat(writer.splitOffsets()).as("Split offsets not present").isNotEmpty(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroReader.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroReader.java new file mode 100644 index 000000000000..3f9b4bb587ba --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroReader.java @@ -0,0 +1,237 @@ +/* + * 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.data; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.Iterator; +import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetAvroValueReaders; +import org.apache.iceberg.parquet.ParquetSchemaUtil; +import org.apache.iceberg.types.Types; +import org.apache.parquet.schema.MessageType; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestParquetAvroReader { + @TempDir private Path temp; + + private static final Schema COMPLEX_SCHEMA = + new Schema( + required(1, "roots", Types.LongType.get()), + optional(3, "lime", Types.ListType.ofRequired(4, Types.DoubleType.get())), + required( + 5, + "strict", + Types.StructType.of( + required(9, "tangerine", Types.StringType.get()), + optional( + 6, + "hopeful", + Types.StructType.of( + required(7, "steel", Types.FloatType.get()), + required(8, "lantern", Types.DateType.get()))), + optional(10, "vehement", Types.LongType.get()))), + optional( + 11, + "metamorphosis", + Types.MapType.ofRequired( + 12, 13, Types.StringType.get(), Types.TimestampType.withoutZone())), + required( + 14, + "winter", + Types.ListType.ofOptional( + 15, + Types.StructType.of( + optional(16, "beet", Types.DoubleType.get()), + required(17, "stamp", Types.TimeType.get()), + optional(18, "wheeze", Types.StringType.get())))), + optional( + 19, + "renovate", + Types.MapType.ofRequired( + 20, + 21, + Types.StringType.get(), + Types.StructType.of( + optional(22, "jumpy", Types.DoubleType.get()), + required(23, "koala", Types.TimeType.get()), + required(24, "couch rope", Types.IntegerType.get())))), + optional(2, "slide", Types.StringType.get()), + required(25, "foo", Types.DecimalType.of(7, 5))); + + @Disabled + public void testStructSchema() throws IOException { + Schema structSchema = + new Schema( + required(1, "circumvent", Types.LongType.get()), + optional(2, "antarctica", Types.StringType.get()), + optional(3, "fluent", Types.DoubleType.get()), + required( + 4, + "quell", + Types.StructType.of( + required(5, "operator", Types.BooleanType.get()), + optional(6, "fanta", Types.IntegerType.get()), + optional(7, "cable", Types.FloatType.get()))), + required(8, "chimney", Types.TimestampType.withZone()), + required(9, "wool", Types.DateType.get())); + + File testFile = writeTestData(structSchema, 5_000_000, 1059); + // RandomData uses the root record name "test", which must match for records to be equal + MessageType readSchema = ParquetSchemaUtil.convert(structSchema, "test"); + + long sum = 0; + long sumSq = 0; + int warmups = 2; + int trials = 10; + + for (int i = 0; i < warmups + trials; i += 1) { + // clean up as much memory as possible to avoid a large GC during the timed run + System.gc(); + + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(structSchema) + .createReaderFunc( + fileSchema -> ParquetAvroValueReaders.buildReader(structSchema, readSchema)) + .build()) { + long start = System.currentTimeMillis(); + long val = 0; + long count = 0; + for (Record record : reader) { + // access something to ensure the compiler doesn't optimize this away + val ^= (Long) record.get(0); + count += 1; + } + long end = System.currentTimeMillis(); + long duration = end - start; + + if (i >= warmups) { + sum += duration; + sumSq += duration * duration; + } + } + } + + double mean = ((double) sum) / trials; + double stddev = Math.sqrt((((double) sumSq) / trials) - (mean * mean)); + } + + @Disabled + public void testWithOldReadPath() throws IOException { + File testFile = writeTestData(COMPLEX_SCHEMA, 500_000, 1985); + // RandomData uses the root record name "test", which must match for records to be equal + MessageType readSchema = ParquetSchemaUtil.convert(COMPLEX_SCHEMA, "test"); + + for (int i = 0; i < 5; i += 1) { + // clean up as much memory as possible to avoid a large GC during the timed run + System.gc(); + + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)).project(COMPLEX_SCHEMA).build()) { + long start = System.currentTimeMillis(); + long val = 0; + long count = 0; + for (Record record : reader) { + // access something to ensure the compiler doesn't optimize this away + val ^= (Long) record.get(0); + count += 1; + } + long end = System.currentTimeMillis(); + } + + // clean up as much memory as possible to avoid a large GC during the timed run + System.gc(); + + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(COMPLEX_SCHEMA) + .createReaderFunc( + fileSchema -> ParquetAvroValueReaders.buildReader(COMPLEX_SCHEMA, readSchema)) + .build()) { + long start = System.currentTimeMillis(); + long val = 0; + long count = 0; + for (Record record : reader) { + // access something to ensure the compiler doesn't optimize this away + val ^= (Long) record.get(0); + count += 1; + } + long end = System.currentTimeMillis(); + } + } + } + + @Test + public void testCorrectness() throws IOException { + Iterable records = RandomData.generate(COMPLEX_SCHEMA, 50_000, 34139); + + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + + try (FileAppender writer = + Parquet.write(Files.localOutput(testFile)).schema(COMPLEX_SCHEMA).build()) { + writer.addAll(records); + } + + // RandomData uses the root record name "test", which must match for records to be equal + MessageType readSchema = ParquetSchemaUtil.convert(COMPLEX_SCHEMA, "test"); + + // verify that the new read path is correct + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(COMPLEX_SCHEMA) + .createReaderFunc( + fileSchema -> ParquetAvroValueReaders.buildReader(COMPLEX_SCHEMA, readSchema)) + .reuseContainers() + .build()) { + int recordNum = 0; + Iterator iter = records.iterator(); + for (Record actual : reader) { + Record expected = iter.next(); + assertThat(actual).as("Record " + recordNum + " should match expected").isEqualTo(expected); + recordNum += 1; + } + } + } + + private File writeTestData(Schema schema, int numRecords, int seed) throws IOException { + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + + try (FileAppender writer = + Parquet.write(Files.localOutput(testFile)).schema(schema).build()) { + writer.addAll(RandomData.generate(schema, numRecords, seed)); + } + + return testFile; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroWriter.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroWriter.java new file mode 100644 index 000000000000..83f8f7f168b1 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestParquetAvroWriter.java @@ -0,0 +1,123 @@ +/* + * 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.data; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.Iterator; +import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetAvroValueReaders; +import org.apache.iceberg.parquet.ParquetAvroWriter; +import org.apache.iceberg.parquet.ParquetSchemaUtil; +import org.apache.iceberg.types.Types; +import org.apache.parquet.schema.MessageType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestParquetAvroWriter { + @TempDir private Path temp; + + private static final Schema COMPLEX_SCHEMA = + new Schema( + required(1, "roots", Types.LongType.get()), + optional(3, "lime", Types.ListType.ofRequired(4, Types.DoubleType.get())), + required( + 5, + "strict", + Types.StructType.of( + required(9, "tangerine", Types.StringType.get()), + optional( + 6, + "hopeful", + Types.StructType.of( + required(7, "steel", Types.FloatType.get()), + required(8, "lantern", Types.DateType.get()))), + optional(10, "vehement", Types.LongType.get()))), + optional( + 11, + "metamorphosis", + Types.MapType.ofRequired( + 12, 13, Types.StringType.get(), Types.TimestampType.withoutZone())), + required( + 14, + "winter", + Types.ListType.ofOptional( + 15, + Types.StructType.of( + optional(16, "beet", Types.DoubleType.get()), + required(17, "stamp", Types.TimeType.get()), + optional(18, "wheeze", Types.StringType.get())))), + optional( + 19, + "renovate", + Types.MapType.ofRequired( + 20, + 21, + Types.StringType.get(), + Types.StructType.of( + optional(22, "jumpy", Types.DoubleType.get()), + required(23, "koala", Types.TimeType.get()), + required(24, "couch rope", Types.IntegerType.get())))), + optional(2, "slide", Types.StringType.get())); + + @Test + public void testCorrectness() throws IOException { + Iterable records = RandomData.generate(COMPLEX_SCHEMA, 50_000, 34139); + + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + + try (FileAppender writer = + Parquet.write(Files.localOutput(testFile)) + .schema(COMPLEX_SCHEMA) + .createWriterFunc(ParquetAvroWriter::buildWriter) + .build()) { + writer.addAll(records); + } + + // RandomData uses the root record name "test", which must match for records to be equal + MessageType readSchema = ParquetSchemaUtil.convert(COMPLEX_SCHEMA, "test"); + + // verify that the new read path is correct + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(COMPLEX_SCHEMA) + .createReaderFunc( + fileSchema -> ParquetAvroValueReaders.buildReader(COMPLEX_SCHEMA, readSchema)) + .build()) { + int recordNum = 0; + Iterator iter = records.iterator(); + for (Record actual : reader) { + Record expected = iter.next(); + assertThat(actual).as("Record " + recordNum + " should match expected").isEqualTo(expected); + recordNum += 1; + } + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.java new file mode 100644 index 000000000000..0dc8b48b2317 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroEnums.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.data; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericData.Record; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestSparkAvroEnums { + + @TempDir private Path temp; + + @Test + public void writeAndValidateEnums() throws IOException { + org.apache.avro.Schema avroSchema = + SchemaBuilder.record("root") + .fields() + .name("enumCol") + .type() + .nullable() + .enumeration("testEnum") + .symbols("SYMB1", "SYMB2") + .enumDefault("SYMB2") + .endRecord(); + + org.apache.avro.Schema enumSchema = avroSchema.getField("enumCol").schema().getTypes().get(0); + Record enumRecord1 = new GenericData.Record(avroSchema); + enumRecord1.put("enumCol", new GenericData.EnumSymbol(enumSchema, "SYMB1")); + Record enumRecord2 = new GenericData.Record(avroSchema); + enumRecord2.put("enumCol", new GenericData.EnumSymbol(enumSchema, "SYMB2")); + Record enumRecord3 = new GenericData.Record(avroSchema); // null enum + List expected = ImmutableList.of(enumRecord1, enumRecord2, enumRecord3); + + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + + try (DataFileWriter writer = new DataFileWriter<>(new GenericDatumWriter<>())) { + writer.create(avroSchema, testFile); + writer.append(enumRecord1); + writer.append(enumRecord2); + writer.append(enumRecord3); + } + + Schema schema = new Schema(AvroSchemaUtil.convert(avroSchema).asStructType().fields()); + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createResolvingReader(SparkPlannedAvroReader::create) + .project(schema) + .build()) { + rows = Lists.newArrayList(reader); + } + + // Iceberg will return enums as strings, so we compare string values for the enum field + for (int i = 0; i < expected.size(); i += 1) { + String expectedEnumString = + expected.get(i).get("enumCol") == null ? null : expected.get(i).get("enumCol").toString(); + String sparkString = + rows.get(i).getUTF8String(0) == null ? null : rows.get(i).getUTF8String(0).toString(); + assertThat(sparkString).isEqualTo(expectedEnumString); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java new file mode 100644 index 000000000000..2d844b7bcea0 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkAvroReader.java @@ -0,0 +1,91 @@ +/* + * 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.data; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.catalyst.InternalRow; + +public class TestSparkAvroReader extends AvroDataTestBase { + @Override + protected void writeAndValidate(Schema schema) throws IOException { + writeAndValidate(schema, schema); + } + + @Override + protected void writeAndValidate( + Schema writeSchema, Schema expectedSchema, List records) + throws IOException { + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + + try (DataWriter dataWriter = + Avro.writeData(Files.localOutput(testFile)) + .schema(writeSchema) + .createWriterFunc(org.apache.iceberg.data.avro.DataWriter::create) + .withSpec(PartitionSpec.unpartitioned()) + .build()) { + for (org.apache.iceberg.data.Record rec : records) { + dataWriter.write(rec); + } + } + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createResolvingReader(schema -> SparkPlannedAvroReader.create(schema, ID_TO_CONSTANT)) + .project(expectedSchema) + .build()) { + rows = Lists.newArrayList(reader); + } + + for (int i = 0; i < records.size(); i += 1) { + GenericsHelpers.assertEqualsUnsafe( + expectedSchema.asStruct(), records.get(i), rows.get(i), ID_TO_CONSTANT, i); + } + } + + @Override + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + List expected = RandomGenericData.generate(writeSchema, 100, 0L); + writeAndValidate(writeSchema, expectedSchema, expected); + } + + @Override + protected boolean supportsDefaultValues() { + return true; + } + + @Override + protected boolean supportsRowLineage() { + return true; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkDateTimes.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkDateTimes.java new file mode 100644 index 000000000000..dbe386e89980 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkDateTimes.java @@ -0,0 +1,78 @@ +/* + * 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.data; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.ZoneId; +import java.util.TimeZone; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.apache.spark.sql.catalyst.util.TimestampFormatter; +import org.junit.jupiter.api.Test; + +public class TestSparkDateTimes { + @Test + public void testSparkDate() { + // checkSparkDate("1582-10-14"); // -141428 + checkSparkDate("1582-10-15"); // first day of the gregorian calendar + checkSparkDate("1601-08-12"); + checkSparkDate("1801-07-04"); + checkSparkDate("1901-08-12"); + checkSparkDate("1969-12-31"); + checkSparkDate("1970-01-01"); + checkSparkDate("2017-12-25"); + checkSparkDate("2043-08-11"); + checkSparkDate("2111-05-03"); + checkSparkDate("2224-02-29"); + checkSparkDate("3224-10-05"); + } + + public void checkSparkDate(String dateString) { + Literal date = Literal.of(dateString).to(Types.DateType.get()); + assertThat(DateTimeUtils.toJavaDate(date.value())) + .as("Should be the same date (" + date.value() + ")") + .asString() + .isEqualTo(dateString); + } + + @Test + public void testSparkTimestamp() { + TimeZone currentTz = TimeZone.getDefault(); + try { + TimeZone.setDefault(TimeZone.getTimeZone("UTC")); + checkSparkTimestamp("1582-10-15T15:51:08.440219+00:00", "1582-10-15 15:51:08.440219"); + checkSparkTimestamp("1970-01-01T00:00:00.000000+00:00", "1970-01-01 00:00:00"); + checkSparkTimestamp("2043-08-11T12:30:01.000001+00:00", "2043-08-11 12:30:01.000001"); + } finally { + TimeZone.setDefault(currentTz); + } + } + + public void checkSparkTimestamp(String timestampString, String sparkRepr) { + Literal ts = Literal.of(timestampString).to(Types.TimestampType.withZone()); + ZoneId zoneId = DateTimeUtils.getZoneId("UTC"); + TimestampFormatter formatter = TimestampFormatter.getFractionFormatter(zoneId); + String sparkTimestamp = formatter.format(ts.value()); + assertThat(sparkTimestamp) + .as("Should be the same timestamp (" + ts.value() + ")") + .isEqualTo(sparkRepr); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java new file mode 100644 index 000000000000..13acaa1e3a7b --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReadMetadataColumns.java @@ -0,0 +1,255 @@ +/* + * 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.data; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Files; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; +import org.apache.iceberg.spark.source.BatchReaderUtil; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.orc.OrcConf; +import org.apache.orc.OrcFile; +import org.apache.orc.Reader; +import org.apache.orc.StripeInformation; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkOrcReadMetadataColumns { + private static final Schema DATA_SCHEMA = + new Schema( + required(100, "id", Types.LongType.get()), required(101, "data", Types.StringType.get())); + + private static final Schema PROJECTION_SCHEMA = + new Schema( + required(100, "id", Types.LongType.get()), + required(101, "data", Types.StringType.get()), + MetadataColumns.ROW_POSITION, + MetadataColumns.IS_DELETED); + + private static final DeleteFilter NO_DELETES_FILTER = + new TestHelpers.CustomizedDeleteFilter(false, DATA_SCHEMA, PROJECTION_SCHEMA); + + private static final int NUM_ROWS = 1000; + private static final int RECORDS_PER_BATCH = 10; + private static final List DATA_ROWS; + private static final List EXPECTED_ROWS; + + static { + DATA_ROWS = Lists.newArrayListWithCapacity(NUM_ROWS); + for (long i = 0; i < NUM_ROWS; i++) { + InternalRow row = new GenericInternalRow(DATA_SCHEMA.columns().size()); + row.update(0, i); + row.update(1, UTF8String.fromString("str" + i)); + DATA_ROWS.add(row); + } + + EXPECTED_ROWS = Lists.newArrayListWithCapacity(NUM_ROWS); + for (long i = 0; i < NUM_ROWS; i++) { + InternalRow row = new GenericInternalRow(PROJECTION_SCHEMA.columns().size()); + row.update(0, i); + row.update(1, UTF8String.fromString("str" + i)); + row.update(2, i); + row.update(3, false); + EXPECTED_ROWS.add(row); + } + } + + @Parameters(name = "vectorized = {0}") + public static Collection parameters() { + return Arrays.asList(false, true); + } + + @TempDir private java.nio.file.Path temp; + + @Parameter private boolean vectorized; + private File testFile; + + @BeforeEach + public void writeFile() throws IOException { + testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + + try (FileAppender writer = + ORC.write(Files.localOutput(testFile)) + .createWriterFunc(SparkOrcWriter::new) + .schema(DATA_SCHEMA) + // write in such a way that the file contains 10 stripes each with 100 rows + .set("iceberg.orc.vectorbatch.size", "100") + .set(OrcConf.ROWS_BETWEEN_CHECKS.getAttribute(), "100") + .set(OrcConf.STRIPE_SIZE.getAttribute(), "1") + .build()) { + writer.addAll(DATA_ROWS); + } + } + + @TestTemplate + public void testReadRowNumbers() throws IOException { + readAndValidate(null, null, null, EXPECTED_ROWS, NO_DELETES_FILTER); + } + + @TestTemplate + public void testReadRowNumbersWithDelete() throws IOException { + assumeThat(vectorized).isTrue(); + + List expectedRowsAfterDelete = Lists.newArrayList(); + EXPECTED_ROWS.forEach(row -> expectedRowsAfterDelete.add(row.copy())); + // remove row at position 98, 99, 100, 101, 102, this crosses two row groups [0, 100) and [100, + // 200) + for (int i = 98; i <= 102; i++) { + expectedRowsAfterDelete.get(i).update(3, true); + } + + DeleteFilter deleteFilter = + new TestHelpers.CustomizedDeleteFilter(true, DATA_SCHEMA, PROJECTION_SCHEMA); + + readAndValidate(null, null, null, expectedRowsAfterDelete, deleteFilter); + } + + @TestTemplate + public void testReadRowNumbersWithFilter() throws IOException { + readAndValidate( + Expressions.greaterThanOrEqual("id", 500), + null, + null, + EXPECTED_ROWS.subList(500, 1000), + NO_DELETES_FILTER); + } + + @TestTemplate + public void testReadRowNumbersWithSplits() throws IOException { + Reader reader; + try { + OrcFile.ReaderOptions readerOptions = + OrcFile.readerOptions(new Configuration()).useUTCTimestamp(true); + reader = OrcFile.createReader(new Path(testFile.toString()), readerOptions); + } catch (IOException ioe) { + throw new RuntimeIOException(ioe, "Failed to open file: %s", testFile); + } + List splitOffsets = + reader.getStripes().stream().map(StripeInformation::getOffset).collect(Collectors.toList()); + List splitLengths = + reader.getStripes().stream().map(StripeInformation::getLength).collect(Collectors.toList()); + + for (int i = 0; i < 10; i++) { + readAndValidate( + null, + splitOffsets.get(i), + splitLengths.get(i), + EXPECTED_ROWS.subList(i * 100, (i + 1) * 100), + NO_DELETES_FILTER); + } + } + + private void readAndValidate( + Expression filter, + Long splitStart, + Long splitLength, + List expected, + DeleteFilter deleteFilter) + throws IOException { + Schema projectionWithoutMetadataFields = + TypeUtil.selectNot(PROJECTION_SCHEMA, MetadataColumns.metadataFieldIds()); + CloseableIterable reader = null; + try { + ORC.ReadBuilder builder = + ORC.read(Files.localInput(testFile)).project(projectionWithoutMetadataFields); + + if (vectorized) { + builder = + builder + .recordsPerBatch(RECORDS_PER_BATCH) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader( + PROJECTION_SCHEMA, readOrcSchema, ImmutableMap.of())); + } else { + builder = + builder.createReaderFunc( + readOrcSchema -> new SparkOrcReader(PROJECTION_SCHEMA, readOrcSchema)); + } + + if (filter != null) { + builder = builder.filter(filter); + } + + if (splitStart != null && splitLength != null) { + builder = builder.split(splitStart, splitLength); + } + + if (vectorized) { + reader = batchesToRows(BatchReaderUtil.applyDeleteFilter(builder.build(), deleteFilter)); + } else { + reader = builder.build(); + } + + final Iterator actualRows = reader.iterator(); + final Iterator expectedRows = expected.iterator(); + while (expectedRows.hasNext()) { + assertThat(actualRows).as("Should have expected number of rows").hasNext(); + TestHelpers.assertEquals(PROJECTION_SCHEMA, expectedRows.next(), actualRows.next()); + } + assertThat(actualRows).as("Should not have extra rows").isExhausted(); + } finally { + if (reader != null) { + reader.close(); + } + } + } + + private CloseableIterable batchesToRows(CloseableIterable batches) { + return CloseableIterable.combine( + Iterables.concat(Iterables.transform(batches, b -> (Iterable) b::rowIterator)), + batches); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java new file mode 100644 index 000000000000..3fcfe6845c99 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkOrcReader.java @@ -0,0 +1,127 @@ +/* + * 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.data; + +import static org.apache.iceberg.spark.data.TestHelpers.assertEquals; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterators; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.junit.jupiter.api.Test; + +public class TestSparkOrcReader extends AvroDataTestBase { + @Override + protected void writeAndValidate(Schema schema) throws IOException { + final Iterable expected = RandomData.generateSpark(schema, 100, 0L); + + writeAndValidateRecords(schema, expected); + } + + @Test + public void writeAndValidateRepeatingRecords() throws IOException { + Schema structSchema = + new Schema( + required(100, "id", Types.LongType.get()), + required(101, "data", Types.StringType.get())); + List expectedRepeating = + Collections.nCopies(100, RandomData.generateSpark(structSchema, 1, 0L).iterator().next()); + + writeAndValidateRecords(structSchema, expectedRepeating); + } + + private void writeAndValidateRecords(Schema schema, Iterable expected) + throws IOException { + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + + try (FileAppender writer = + ORC.write(Files.localOutput(testFile)) + .createWriterFunc(SparkOrcWriter::new) + .schema(schema) + .build()) { + writer.addAll(expected); + } + + try (CloseableIterable reader = + ORC.read(Files.localInput(testFile)) + .project(schema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(schema, readOrcSchema)) + .build()) { + final Iterator actualRows = reader.iterator(); + final Iterator expectedRows = expected.iterator(); + while (expectedRows.hasNext()) { + assertThat(actualRows).as("Should have expected number of rows").hasNext(); + assertEquals(schema, expectedRows.next(), actualRows.next()); + } + assertThat(actualRows).as("Should not have extra rows").isExhausted(); + } + + try (CloseableIterable reader = + ORC.read(Files.localInput(testFile)) + .project(schema) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader(schema, readOrcSchema, ImmutableMap.of())) + .build()) { + final Iterator actualRows = batchesToRows(reader.iterator()); + final Iterator expectedRows = expected.iterator(); + while (expectedRows.hasNext()) { + assertThat(actualRows).as("Should have expected number of rows").hasNext(); + assertEquals(schema, expectedRows.next(), actualRows.next()); + } + assertThat(actualRows).as("Should not have extra rows").isExhausted(); + } + } + + private Iterator batchesToRows(Iterator batches) { + return Iterators.concat(Iterators.transform(batches, ColumnarBatch::rowIterator)); + } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create ListType with unknown element type"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create MapType with unknown value type"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java new file mode 100644 index 000000000000..e2e5a98ccb8b --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReadMetadataColumns.java @@ -0,0 +1,283 @@ +/* + * 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.data; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Files; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetSchemaUtil; +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.SparkSchemaUtil; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; +import org.apache.iceberg.spark.source.BatchReaderUtil; +import org.apache.iceberg.types.Types; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkParquetReadMetadataColumns { + private static final Schema DATA_SCHEMA = + new Schema( + required(100, "id", Types.LongType.get()), required(101, "data", Types.StringType.get())); + + private static final Schema PROJECTION_SCHEMA = + new Schema( + required(100, "id", Types.LongType.get()), + required(101, "data", Types.StringType.get()), + MetadataColumns.ROW_POSITION, + MetadataColumns.IS_DELETED); + + private static final int NUM_ROWS = 1000; + private static final List DATA_ROWS; + private static final List EXPECTED_ROWS; + private static final int NUM_ROW_GROUPS = 10; + private static final int ROWS_PER_SPLIT = NUM_ROWS / NUM_ROW_GROUPS; + private static final int RECORDS_PER_BATCH = ROWS_PER_SPLIT / 10; + + static { + DATA_ROWS = Lists.newArrayListWithCapacity(NUM_ROWS); + for (long i = 0; i < NUM_ROWS; i += 1) { + InternalRow row = new GenericInternalRow(DATA_SCHEMA.columns().size()); + if (i >= NUM_ROWS / 2) { + row.update(0, 2 * i); + } else { + row.update(0, i); + } + row.update(1, UTF8String.fromString("str" + i)); + DATA_ROWS.add(row); + } + + EXPECTED_ROWS = Lists.newArrayListWithCapacity(NUM_ROWS); + for (long i = 0; i < NUM_ROWS; i += 1) { + InternalRow row = new GenericInternalRow(PROJECTION_SCHEMA.columns().size()); + if (i >= NUM_ROWS / 2) { + row.update(0, 2 * i); + } else { + row.update(0, i); + } + row.update(1, UTF8String.fromString("str" + i)); + row.update(2, i); + row.update(3, false); + EXPECTED_ROWS.add(row); + } + } + + @Parameters(name = "vectorized = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {false}, new Object[] {true}}; + } + + @TempDir protected java.nio.file.Path temp; + + @Parameter private boolean vectorized; + private File testFile; + + @BeforeEach + public void writeFile() throws IOException { + List fileSplits = Lists.newArrayList(); + StructType struct = SparkSchemaUtil.convert(DATA_SCHEMA); + Configuration conf = new Configuration(); + + testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + ParquetFileWriter parquetFileWriter = + new ParquetFileWriter( + conf, + ParquetSchemaUtil.convert(DATA_SCHEMA, "testSchema"), + new Path(testFile.getAbsolutePath())); + + parquetFileWriter.start(); + for (int i = 0; i < NUM_ROW_GROUPS; i += 1) { + File split = File.createTempFile("junit", null, temp.toFile()); + assertThat(split.delete()).as("Delete should succeed").isTrue(); + fileSplits.add(new Path(split.getAbsolutePath())); + try (FileAppender writer = + Parquet.write(Files.localOutput(split)) + .createWriterFunc(msgType -> SparkParquetWriters.buildWriter(struct, msgType)) + .schema(DATA_SCHEMA) + .overwrite() + .build()) { + writer.addAll(DATA_ROWS.subList(i * ROWS_PER_SPLIT, (i + 1) * ROWS_PER_SPLIT)); + } + parquetFileWriter.appendFile( + HadoopInputFile.fromPath(new Path(split.getAbsolutePath()), conf)); + } + parquetFileWriter.end( + ParquetFileWriter.mergeMetadataFiles(fileSplits, conf) + .getFileMetaData() + .getKeyValueMetaData()); + } + + @TestTemplate + public void testReadRowNumbers() throws IOException { + readAndValidate(null, null, null, EXPECTED_ROWS); + } + + @TestTemplate + public void testReadRowNumbersWithDelete() throws IOException { + assumeThat(vectorized).isTrue(); + + List expectedRowsAfterDelete = Lists.newArrayList(); + EXPECTED_ROWS.forEach(row -> expectedRowsAfterDelete.add(row.copy())); + // remove row at position 98, 99, 100, 101, 102, this crosses two row groups [0, 100) and [100, + // 200) + for (int i = 98; i <= 102; i++) { + expectedRowsAfterDelete.get(i).update(3, true); + } + + Parquet.ReadBuilder builder = + Parquet.read(Files.localInput(testFile)).project(PROJECTION_SCHEMA); + + DeleteFilter deleteFilter = + new TestHelpers.CustomizedDeleteFilter(true, DATA_SCHEMA, PROJECTION_SCHEMA); + + builder.createBatchedReaderFunc( + fileSchema -> + VectorizedSparkParquetReaders.buildReader( + PROJECTION_SCHEMA, fileSchema, Maps.newHashMap())); + builder.recordsPerBatch(RECORDS_PER_BATCH); + + validate(expectedRowsAfterDelete, builder, deleteFilter); + } + + @TestTemplate + public void testReadRowNumbersWithFilter() throws IOException { + // current iceberg supports row group filter. + for (int i = 1; i < 5; i += 1) { + readAndValidate( + Expressions.and( + Expressions.lessThan("id", NUM_ROWS / 2), + Expressions.greaterThanOrEqual("id", i * ROWS_PER_SPLIT)), + null, + null, + EXPECTED_ROWS.subList(i * ROWS_PER_SPLIT, NUM_ROWS / 2)); + } + } + + @TestTemplate + public void testReadRowNumbersWithSplits() throws IOException { + ParquetFileReader fileReader = + new ParquetFileReader( + HadoopInputFile.fromPath(new Path(testFile.getAbsolutePath()), new Configuration()), + ParquetReadOptions.builder().build()); + List rowGroups = fileReader.getRowGroups(); + for (int i = 0; i < NUM_ROW_GROUPS; i += 1) { + readAndValidate( + null, + rowGroups.get(i).getColumns().get(0).getStartingPos(), + rowGroups.get(i).getCompressedSize(), + EXPECTED_ROWS.subList(i * ROWS_PER_SPLIT, (i + 1) * ROWS_PER_SPLIT)); + } + } + + private void readAndValidate( + Expression filter, Long splitStart, Long splitLength, List expected) + throws IOException { + Parquet.ReadBuilder builder = + Parquet.read(Files.localInput(testFile)).project(PROJECTION_SCHEMA); + + if (vectorized) { + builder.createBatchedReaderFunc( + fileSchema -> + VectorizedSparkParquetReaders.buildReader( + PROJECTION_SCHEMA, fileSchema, Maps.newHashMap())); + builder.recordsPerBatch(RECORDS_PER_BATCH); + } else { + builder = + builder.createReaderFunc( + msgType -> SparkParquetReaders.buildReader(PROJECTION_SCHEMA, msgType)); + } + + if (filter != null) { + builder = builder.filter(filter); + } + + if (splitStart != null && splitLength != null) { + builder = builder.split(splitStart, splitLength); + } + + validate( + expected, + builder, + new TestHelpers.CustomizedDeleteFilter(false, DATA_SCHEMA, PROJECTION_SCHEMA)); + } + + private void validate( + List expected, Parquet.ReadBuilder builder, DeleteFilter filter) + throws IOException { + try (CloseableIterable reader = reader(builder, filter)) { + final Iterator actualRows = reader.iterator(); + + for (InternalRow internalRow : expected) { + assertThat(actualRows).as("Should have expected number of rows").hasNext(); + TestHelpers.assertEquals(PROJECTION_SCHEMA, internalRow, actualRows.next()); + } + + assertThat(actualRows).as("Should not have extra rows").isExhausted(); + } + } + + private CloseableIterable reader( + Parquet.ReadBuilder builder, DeleteFilter filter) { + if (!vectorized) { + return builder.build(); + } else { + return batchesToRows(BatchReaderUtil.applyDeleteFilter(builder.build(), filter)); + } + } + + private CloseableIterable batchesToRows(CloseableIterable batches) { + return CloseableIterable.combine( + Iterables.concat(Iterables.transform(batches, b -> (Iterable) b::rowIterator)), + batches); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java new file mode 100644 index 000000000000..1d1ccca1a2fb --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java @@ -0,0 +1,263 @@ +/* + * 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.data; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.inmemory.InMemoryOutputFile; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetUtil; +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.types.Types; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.api.WriteSupport; +import org.apache.parquet.hadoop.util.HadoopOutputFile; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.Test; + +public class TestSparkParquetReader extends AvroDataTestBase { + @Override + protected void writeAndValidate(Schema schema) throws IOException { + writeAndValidate(schema, schema); + } + + @Override + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + List expected = RandomGenericData.generate(writeSchema, 100, 0L); + writeAndValidate(writeSchema, expectedSchema, expected); + } + + @Override + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema, List expected) + throws IOException { + OutputFile output = new InMemoryOutputFile(); + try (FileAppender writer = + Parquet.write(output) + .schema(writeSchema) + .createWriterFunc(GenericParquetWriter::create) + .named("test") + .build()) { + writer.addAll(expected); + } + + try (CloseableIterable reader = + Parquet.read(output.toInputFile()) + .project(expectedSchema) + .createReaderFunc( + type -> SparkParquetReaders.buildReader(expectedSchema, type, ID_TO_CONSTANT)) + .build()) { + Iterator rows = reader.iterator(); + int pos = 0; + for (Record record : expected) { + assertThat(rows).as("Should have expected number of rows").hasNext(); + GenericsHelpers.assertEqualsUnsafe( + expectedSchema.asStruct(), record, rows.next(), ID_TO_CONSTANT, pos); + pos += 1; + } + assertThat(rows).as("Should not have extra rows").isExhausted(); + } + } + + @Override + protected boolean supportsRowLineage() { + return true; + } + + @Override + protected boolean supportsDefaultValues() { + return true; + } + + @Override + protected boolean supportsVariant() { + return true; + } + + protected List rowsFromFile(InputFile inputFile, Schema schema) throws IOException { + try (CloseableIterable reader = + Parquet.read(inputFile) + .project(schema) + .createReaderFunc(type -> SparkParquetReaders.buildReader(schema, type)) + .build()) { + return Lists.newArrayList(reader); + } + } + + protected Table tableFromInputFile(InputFile inputFile, Schema schema) throws IOException { + HadoopTables tables = new HadoopTables(); + Table table = + tables.create( + schema, + PartitionSpec.unpartitioned(), + ImmutableMap.of(), + java.nio.file.Files.createTempDirectory(temp, null).toFile().getCanonicalPath()); + + table + .newAppend() + .appendFile( + DataFiles.builder(PartitionSpec.unpartitioned()) + .withFormat(FileFormat.PARQUET) + .withInputFile(inputFile) + .withMetrics(ParquetUtil.fileMetrics(inputFile, MetricsConfig.getDefault())) + .withFileSizeInBytes(inputFile.getLength()) + .build()) + .commit(); + + return table; + } + + @Test + public void testInt96TimestampProducedBySparkIsReadCorrectly() throws IOException { + String outputFilePath = String.format("%s/%s", temp.toAbsolutePath(), "parquet_int96.parquet"); + HadoopOutputFile outputFile = + HadoopOutputFile.fromPath( + new org.apache.hadoop.fs.Path(outputFilePath), new Configuration()); + Schema schema = new Schema(required(1, "ts", Types.TimestampType.withZone())); + StructType sparkSchema = + new StructType( + new StructField[] { + new StructField("ts", DataTypes.TimestampType, true, Metadata.empty()) + }); + List rows = Lists.newArrayList(RandomData.generateSpark(schema, 10, 0L)); + + try (ParquetWriter writer = + new NativeSparkWriterBuilder(outputFile) + .set("org.apache.spark.sql.parquet.row.attributes", sparkSchema.json()) + .set("spark.sql.parquet.writeLegacyFormat", "false") + .set("spark.sql.parquet.outputTimestampType", "INT96") + .set("spark.sql.parquet.fieldId.write.enabled", "true") + .build()) { + for (InternalRow row : rows) { + writer.write(row); + } + } + + InputFile parquetInputFile = Files.localInput(outputFilePath); + List readRows = rowsFromFile(parquetInputFile, schema); + + assertThat(readRows).hasSameSizeAs(rows); + assertThat(readRows).isEqualTo(rows); + + // Now we try to import that file as an Iceberg table to make sure Iceberg can read + // Int96 end to end. + Table int96Table = tableFromInputFile(parquetInputFile, schema); + List tableRecords = Lists.newArrayList(IcebergGenerics.read(int96Table).build()); + + assertThat(tableRecords).hasSameSizeAs(rows); + + for (int i = 0; i < tableRecords.size(); i++) { + GenericsHelpers.assertEqualsUnsafe(schema.asStruct(), tableRecords.get(i), rows.get(i)); + } + } + + /** + * Native Spark ParquetWriter.Builder implementation so that we can write timestamps using Spark's + * native ParquetWriteSupport. + */ + private static class NativeSparkWriterBuilder + extends ParquetWriter.Builder { + private final Map config = Maps.newHashMap(); + + NativeSparkWriterBuilder(org.apache.parquet.io.OutputFile path) { + super(path); + } + + public NativeSparkWriterBuilder set(String property, String value) { + this.config.put(property, value); + return self(); + } + + @Override + protected NativeSparkWriterBuilder self() { + return this; + } + + @Override + protected WriteSupport getWriteSupport(Configuration configuration) { + for (Map.Entry entry : config.entrySet()) { + configuration.set(entry.getKey(), entry.getValue()); + } + + return new org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport(); + } + } + + @Test + public void testMissingRequiredWithoutDefault() { + Schema writeSchema = new Schema(required(1, "id", Types.LongType.get())); + + Schema expectedSchema = + new Schema( + required(1, "id", Types.LongType.get()), + Types.NestedField.required("missing_str") + .withId(6) + .ofType(Types.StringType.get()) + .withDoc("Missing required field with no default") + .build()); + + assertThatThrownBy(() -> writeAndValidate(writeSchema, expectedSchema)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Missing required field: missing_str"); + } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert element Parquet: unknown"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert value Parquet: unknown"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java new file mode 100644 index 000000000000..0606e9321560 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java @@ -0,0 +1,182 @@ +/* + * 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.data; + +import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX; +import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_FPP_PREFIX; +import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_NDV_PREFIX; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Path; +import java.util.Iterator; +import java.util.OptionalLong; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetSchemaUtil; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.Types; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.schema.MessageType; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestSparkParquetWriter { + @TempDir private Path temp; + + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "id_long", Types.LongType.get())); + + private static final Schema COMPLEX_SCHEMA = + new Schema( + required(1, "roots", Types.LongType.get()), + optional(3, "lime", Types.ListType.ofRequired(4, Types.DoubleType.get())), + required( + 5, + "strict", + Types.StructType.of( + required(9, "tangerine", Types.StringType.get()), + optional( + 6, + "hopeful", + Types.StructType.of( + required(7, "steel", Types.FloatType.get()), + required(8, "lantern", Types.DateType.get()))), + optional(10, "vehement", Types.LongType.get()))), + optional( + 11, + "metamorphosis", + Types.MapType.ofRequired( + 12, 13, Types.StringType.get(), Types.TimestampType.withZone())), + required( + 14, + "winter", + Types.ListType.ofOptional( + 15, + Types.StructType.of( + optional(16, "beet", Types.DoubleType.get()), + required(17, "stamp", Types.FloatType.get()), + optional(18, "wheeze", Types.StringType.get())))), + optional( + 19, + "renovate", + Types.MapType.ofRequired( + 20, + 21, + Types.StringType.get(), + Types.StructType.of( + optional(22, "jumpy", Types.DoubleType.get()), + required(23, "koala", Types.UUIDType.get()), + required(24, "couch rope", Types.IntegerType.get())))), + optional(2, "slide", Types.StringType.get())); + + @Test + public void testCorrectness() throws IOException { + int numRows = 50_000; + Iterable records = RandomData.generateSpark(COMPLEX_SCHEMA, numRows, 19981); + + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + + try (FileAppender writer = + Parquet.write(Files.localOutput(testFile)) + .schema(COMPLEX_SCHEMA) + .createWriterFunc( + msgType -> + SparkParquetWriters.buildWriter( + SparkSchemaUtil.convert(COMPLEX_SCHEMA), msgType)) + .build()) { + writer.addAll(records); + } + + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(COMPLEX_SCHEMA) + .createReaderFunc(type -> SparkParquetReaders.buildReader(COMPLEX_SCHEMA, type)) + .build()) { + Iterator expected = records.iterator(); + Iterator rows = reader.iterator(); + for (int i = 0; i < numRows; i += 1) { + assertThat(rows).as("Should have expected number of rows").hasNext(); + TestHelpers.assertEquals(COMPLEX_SCHEMA, expected.next(), rows.next()); + } + assertThat(rows).as("Should not have extra rows").isExhausted(); + } + } + + @Test + public void testFpp() throws IOException, NoSuchFieldException, IllegalAccessException { + File testFile = File.createTempFile("junit", null, temp.toFile()); + try (FileAppender writer = + Parquet.write(Files.localOutput(testFile)) + .schema(SCHEMA) + .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id", "true") + .set(PARQUET_BLOOM_FILTER_COLUMN_FPP_PREFIX + "id", "0.05") + .createWriterFunc( + msgType -> + SparkParquetWriters.buildWriter(SparkSchemaUtil.convert(SCHEMA), msgType)) + .build()) { + // Using reflection to access the private 'props' field in ParquetWriter + Field propsField = writer.getClass().getDeclaredField("props"); + propsField.setAccessible(true); + ParquetProperties props = (ParquetProperties) propsField.get(writer); + MessageType parquetSchema = ParquetSchemaUtil.convert(SCHEMA, "test"); + ColumnDescriptor descriptor = parquetSchema.getColumnDescription(new String[] {"id"}); + double fpp = props.getBloomFilterFPP(descriptor).getAsDouble(); + assertThat(fpp).isEqualTo(0.05); + } + } + + @Test + public void testNdv() throws IOException, NoSuchFieldException, IllegalAccessException { + final long expectedNdv = 1000; + final String col = "id"; + File testFile = File.createTempFile("junit", null, temp.toFile()); + try (FileAppender writer = + Parquet.write(Files.localOutput(testFile)) + .schema(SCHEMA) + .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + col, "true") + .set(PARQUET_BLOOM_FILTER_COLUMN_NDV_PREFIX + col, Long.toString(expectedNdv)) + .createWriterFunc( + msgType -> + SparkParquetWriters.buildWriter(SparkSchemaUtil.convert(SCHEMA), msgType)) + .build()) { + // Using reflection to access the private 'props' field in ParquetWriter + Field propsField = writer.getClass().getDeclaredField("props"); + propsField.setAccessible(true); + ParquetProperties props = (ParquetProperties) propsField.get(writer); + MessageType parquetSchema = ParquetSchemaUtil.convert(SCHEMA, "test"); + ColumnDescriptor descriptor = parquetSchema.getColumnDescription(new String[] {col}); + OptionalLong bloomFilterNDV = props.getBloomFilterNDV(descriptor); + assertThat(bloomFilterNDV).isPresent(); + assertThat(bloomFilterNDV.getAsLong()).isEqualTo(expectedNdv); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java new file mode 100644 index 000000000000..634327a81d86 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkRecordOrcReaderWriter.java @@ -0,0 +1,170 @@ +/* + * 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.data; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.io.IOException; +import java.math.BigDecimal; +import java.util.Iterator; +import java.util.List; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.data.orc.GenericOrcWriter; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.jupiter.api.Test; + +public class TestSparkRecordOrcReaderWriter extends AvroDataTestBase { + private static final int NUM_RECORDS = 200; + + private void writeAndValidate(Schema schema, List expectedRecords) throws IOException { + final File originalFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(originalFile.delete()).as("Delete should succeed").isTrue(); + + // Write few generic records into the original test file. + try (FileAppender writer = + ORC.write(Files.localOutput(originalFile)) + .createWriterFunc(GenericOrcWriter::buildWriter) + .schema(schema) + .build()) { + writer.addAll(expectedRecords); + } + + // Read into spark InternalRow from the original test file. + List internalRows = Lists.newArrayList(); + try (CloseableIterable reader = + ORC.read(Files.localInput(originalFile)) + .project(schema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(schema, readOrcSchema)) + .build()) { + reader.forEach(internalRows::add); + assertEqualsUnsafe(schema.asStruct(), expectedRecords, reader, expectedRecords.size()); + } + + final File anotherFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(anotherFile.delete()).as("Delete should succeed").isTrue(); + + // Write those spark InternalRows into a new file again. + try (FileAppender writer = + ORC.write(Files.localOutput(anotherFile)) + .createWriterFunc(SparkOrcWriter::new) + .schema(schema) + .build()) { + writer.addAll(internalRows); + } + + // Check whether the InternalRows are expected records. + try (CloseableIterable reader = + ORC.read(Files.localInput(anotherFile)) + .project(schema) + .createReaderFunc(readOrcSchema -> new SparkOrcReader(schema, readOrcSchema)) + .build()) { + assertEqualsUnsafe(schema.asStruct(), expectedRecords, reader, expectedRecords.size()); + } + + // Read into iceberg GenericRecord and check again. + try (CloseableIterable reader = + ORC.read(Files.localInput(anotherFile)) + .createReaderFunc(typeDesc -> GenericOrcReader.buildReader(schema, typeDesc)) + .project(schema) + .build()) { + assertRecordEquals(expectedRecords, reader, expectedRecords.size()); + } + } + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1992L); + writeAndValidate(schema, expectedRecords); + } + + @Test + public void testDecimalWithTrailingZero() throws IOException { + Schema schema = + new Schema( + required(1, "d1", Types.DecimalType.of(10, 2)), + required(2, "d2", Types.DecimalType.of(20, 5)), + required(3, "d3", Types.DecimalType.of(38, 20))); + + List expected = Lists.newArrayList(); + + GenericRecord record = GenericRecord.create(schema); + record.set(0, new BigDecimal("101.00")); + record.set(1, new BigDecimal("10.00E-3")); + record.set(2, new BigDecimal("1001.0000E-16")); + + expected.add(record.copy()); + + writeAndValidate(schema, expected); + } + + private static void assertRecordEquals( + Iterable expected, Iterable actual, int size) { + Iterator expectedIter = expected.iterator(); + Iterator actualIter = actual.iterator(); + for (int i = 0; i < size; i += 1) { + assertThat(expectedIter).as("Expected iterator should have more rows").hasNext(); + assertThat(actualIter).as("Actual iterator should have more rows").hasNext(); + assertThat(actualIter.next()).as("Should have same rows.").isEqualTo(expectedIter.next()); + } + assertThat(expectedIter).as("Expected iterator should not have any extra rows.").isExhausted(); + assertThat(actualIter).as("Actual iterator should not have any extra rows.").isExhausted(); + } + + private static void assertEqualsUnsafe( + Types.StructType struct, Iterable expected, Iterable actual, int size) { + Iterator expectedIter = expected.iterator(); + Iterator actualIter = actual.iterator(); + for (int i = 0; i < size; i += 1) { + assertThat(expectedIter).as("Expected iterator should have more rows").hasNext(); + assertThat(actualIter).as("Actual iterator should have more rows").hasNext(); + GenericsHelpers.assertEqualsUnsafe(struct, expectedIter.next(), actualIter.next()); + } + assertThat(expectedIter).as("Expected iterator should not have any extra rows.").isExhausted(); + assertThat(actualIter).as("Actual iterator should not have any extra rows.").isExhausted(); + } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create ListType with unknown element type"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create MapType with unknown value type"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkVariants.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkVariants.java new file mode 100644 index 000000000000..9e80701d308f --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkVariants.java @@ -0,0 +1,344 @@ +/* + * 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.data; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.variants.ShreddedObject; +import org.apache.iceberg.variants.ValueArray; +import org.apache.iceberg.variants.Variant; +import org.apache.iceberg.variants.VariantArray; +import org.apache.iceberg.variants.VariantMetadata; +import org.apache.iceberg.variants.VariantObject; +import org.apache.iceberg.variants.VariantPrimitive; +import org.apache.iceberg.variants.VariantTestUtil; +import org.apache.iceberg.variants.VariantValue; +import org.apache.iceberg.variants.Variants; +import org.apache.spark.SparkRuntimeException; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.VariantType; +import org.apache.spark.sql.types.VariantType$; +import org.apache.spark.types.variant.VariantUtil; +import org.apache.spark.unsafe.types.VariantVal; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.FieldSource; + +public class TestSparkVariants extends TestBase { + private static final VariantPrimitive[] PRIMITIVES = + new VariantPrimitive[] { + Variants.ofNull(), + Variants.of(true), + Variants.of(false), + Variants.of((byte) 34), + Variants.of((byte) -34), + Variants.of((short) 1234), + Variants.of((short) -1234), + Variants.of(12345), + Variants.of(-12345), + Variants.of(9876543210L), + Variants.of(-9876543210L), + Variants.of(10.11F), + Variants.of(-10.11F), + Variants.of(14.3D), + Variants.of(-14.3D), + Variants.ofIsoDate("2024-11-07"), + Variants.ofIsoDate("1957-11-07"), + Variants.ofIsoTimestamptz("2024-11-07T12:33:54.123456+00:00"), + Variants.ofIsoTimestamptz("1957-11-07T12:33:54.123456+00:00"), + Variants.ofIsoTimestampntz("2024-11-07T12:33:54.123456"), + Variants.ofIsoTimestampntz("1957-11-07T12:33:54.123456"), + Variants.of(new BigDecimal("12345.6789")), // decimal4 + Variants.of(new BigDecimal("-12345.6789")), // decimal4 + Variants.of(new BigDecimal("123456789.987654321")), // decimal8 + Variants.of(new BigDecimal("-123456789.987654321")), // decimal8 + Variants.of(new BigDecimal("9876543210.123456789")), // decimal16 + Variants.of(new BigDecimal("-9876543210.123456789")), // decimal16 + Variants.of(ByteBuffer.wrap(new byte[] {0x0a, 0x0b, 0x0c, 0x0d})), + Variants.of("iceberg"), + Variants.ofUUID("f24f9b64-81fa-49d1-b74e-8c09a6e31c56"), + }; + + private static final VariantPrimitive[] UNSUPPORTED_PRIMITIVES = + new VariantPrimitive[] { + Variants.ofIsoTime("12:33:54.123456"), + Variants.ofIsoTimestamptzNanos("2024-11-07T12:33:54.123456789+00:00"), + Variants.ofIsoTimestampntzNanos("2024-11-07T12:33:54.123456789"), + }; + + @Test + public void testIcebergVariantTypeToSparkVariantType() { + // Test that Iceberg's VariantType converts to Spark's VariantType + Types.VariantType icebergVariantType = Types.VariantType.get(); + DataType sparkVariantType = SparkSchemaUtil.convert(icebergVariantType); + + assertThat(sparkVariantType).isInstanceOf(VariantType.class); + } + + @Test + public void testSparkVariantTypeToIcebergVariantType() { + // Test that Spark's VariantType converts to Iceberg's VariantType + org.apache.spark.sql.types.DataType sparkVariantType = VariantType$.MODULE$; + Type icebergVariantType = SparkSchemaUtil.convert(sparkVariantType); + + assertThat(icebergVariantType).isEqualTo(Types.VariantType.get()); + } + + @ParameterizedTest + @FieldSource("PRIMITIVES") + public void testVariantPrimitiveRoundTrip(VariantPrimitive primitive) { + testVariantRoundTrip(Variants.emptyMetadata(), primitive); + } + + @Test + public void testVariantArrayRoundTrip() { + VariantMetadata metadata = Variants.emptyMetadata(); + ValueArray array = Variants.array(); + array.add(Variants.of("hello")); + array.add(Variants.of((byte) 42)); + array.add(Variants.ofNull()); + + testVariantRoundTrip(metadata, array); + } + + @Test + public void testVariantObjectRoundTrip() { + VariantMetadata metadata = Variants.metadata("name", "age", "active"); + ShreddedObject object = Variants.object(metadata); + object.put("name", Variants.of("John Doe")); + object.put("age", Variants.of((byte) 30)); + object.put("active", Variants.of(true)); + + testVariantRoundTrip(metadata, object); + } + + @Test + public void testVariantNestedStructures() { + VariantMetadata metadata = Variants.metadata("user", "scores", "address", "city", "state"); + + // Create nested object: address + ShreddedObject address = Variants.object(metadata); + address.put("city", Variants.of("Anytown")); + address.put("state", Variants.of("CA")); + + // Create array of scores + ValueArray scores = Variants.array(); + scores.add(Variants.of((byte) 95)); + scores.add(Variants.of((byte) 87)); + scores.add(Variants.of((byte) 92)); + + // Create main object + ShreddedObject mainObject = Variants.object(metadata); + mainObject.put("user", Variants.of("Jane")); + mainObject.put("scores", scores); + mainObject.put("address", address); + + testVariantRoundTrip(metadata, mainObject); + } + + @ParameterizedTest + @FieldSource("UNSUPPORTED_PRIMITIVES") + public void testUnsupportedOperations(VariantPrimitive primitive) { + // This tests the current state where Spark integration is not fully implemented + // TIME, nano timestamps are not supported in Spark + ByteBuffer valueBuffer = + ByteBuffer.allocate(primitive.sizeInBytes()).order(ByteOrder.LITTLE_ENDIAN); + primitive.writeTo(valueBuffer, 0); + + org.apache.spark.types.variant.Variant sparkVariant = + new org.apache.spark.types.variant.Variant( + valueBuffer.array(), VariantTestUtil.emptyMetadata().array()); + + assertThatThrownBy(sparkVariant::getType) + .as("Unsupported variant type in Spark") + .isInstanceOf(SparkRuntimeException.class) + .hasMessageContaining("UNKNOWN_PRIMITIVE_TYPE_IN_VARIANT"); + } + + private void testVariantRoundTrip(VariantMetadata metadata, VariantValue value) { + // Create Iceberg variant + Variant icebergVariant = Variant.of(metadata, value); + + // Serialize to bytes + ByteBuffer metadataBuffer = + ByteBuffer.allocate(metadata.sizeInBytes()).order(ByteOrder.LITTLE_ENDIAN); + metadata.writeTo(metadataBuffer, 0); + + ByteBuffer valueBuffer = + ByteBuffer.allocate(value.sizeInBytes()).order(ByteOrder.LITTLE_ENDIAN); + value.writeTo(valueBuffer, 0); + + // Create Spark VariantVal from the same bytes + byte[] metadataBytes = ByteBuffers.toByteArray(metadataBuffer); + byte[] valueBytes = ByteBuffers.toByteArray(valueBuffer); + VariantVal sparkVariant = new VariantVal(valueBytes, metadataBytes); + + GenericsHelpers.assertEquals(icebergVariant, sparkVariant); + assertEquals( + icebergVariant.value(), + new org.apache.spark.types.variant.Variant( + sparkVariant.getValue(), sparkVariant.getMetadata())); + + // TODO: Round-trip: use Spark VariantBuilder to build a Spark variant from Iceberg variant and + // deserialize back to Iceberg variant currently VariantBuilder doesn't have an easy way to + // construct array/object. + } + + private static void assertEquals( + VariantValue expected, org.apache.spark.types.variant.Variant actual) { + assertThat(actual).isNotNull(); + assertThat(expected).isNotNull(); + + switch (expected.type()) { + case OBJECT: + assertThat(actual.getType()).isEqualTo(VariantUtil.Type.OBJECT); + VariantObject expectedObject = expected.asObject(); + assertThat(actual.objectSize()) + .as("Variant object num fields should match") + .isEqualTo(expectedObject.numFields()); + + for (String fieldName : expectedObject.fieldNames()) { + assertEquals(expectedObject.get(fieldName), actual.getFieldByKey(fieldName)); + } + break; + case ARRAY: + assertThat(actual.getType()).isEqualTo(VariantUtil.Type.ARRAY); + VariantArray expectedArray = expected.asArray(); + assertThat(actual.arraySize()) + .as("Variant array num element should match") + .isEqualTo(expectedArray.numElements()); + + for (int i = 0; i < expectedArray.numElements(); i += 1) { + assertEquals(expectedArray.get(i), actual.getElementAtIndex(i)); + } + break; + case NULL: + assertThat(actual.getType()) + .as("Variant primitive type should match") + .isEqualTo(VariantUtil.Type.NULL); + break; + case BOOLEAN_TRUE: + case BOOLEAN_FALSE: + assertThat(actual.getType()) + .as("Variant primitive type should match") + .isEqualTo(VariantUtil.Type.BOOLEAN); + assertThat(actual.getBoolean()) + .as("Variant primitive value should match") + .isEqualTo(expected.asPrimitive().get()); + break; + case INT8: + case INT16: + case INT32: + case INT64: + assertThat(actual.getType()) + .as("Variant primitive type should match") + .isEqualTo(VariantUtil.Type.LONG); + assertThat(actual.getLong()) + .as("Variant primitive value should match") + .isEqualTo(((Number) expected.asPrimitive().get()).longValue()); + break; + case DOUBLE: + assertThat(actual.getType()) + .as("Variant primitive type should match") + .isEqualTo(VariantUtil.Type.DOUBLE); + assertThat(actual.getDouble()) + .as("Variant primitive value should match") + .isEqualTo(expected.asPrimitive().get()); + break; + case DECIMAL4: + case DECIMAL8: + case DECIMAL16: + assertThat(actual.getType()) + .as("Variant primitive type should match") + .isEqualTo(VariantUtil.Type.DECIMAL); + // For decimal, Spark strips trailing zeros + assertThat(actual.getDecimal()) + .as("Variant primitive value should match") + .isEqualTo(((BigDecimal) expected.asPrimitive().get()).stripTrailingZeros()); + break; + case DATE: + assertThat(actual.getType()) + .as("Variant primitive type should match") + .isEqualTo(VariantUtil.Type.DATE); + assertThat(actual.getLong()) + .as("Variant primitive value should match") + .isEqualTo(((Number) expected.asPrimitive().get()).longValue()); + break; + case TIMESTAMPTZ: + assertThat(actual.getType()) + .as("Variant primitive type should match") + .isEqualTo(VariantUtil.Type.TIMESTAMP); + assertThat(actual.getLong()) + .as("Variant primitive value should match") + .isEqualTo(expected.asPrimitive().get()); + break; + case TIMESTAMPNTZ: + assertThat(actual.getType()) + .as("Variant primitive type should match") + .isEqualTo(VariantUtil.Type.TIMESTAMP_NTZ); + assertThat(actual.getLong()) + .as("Variant primitive value should match") + .isEqualTo(expected.asPrimitive().get()); + break; + case FLOAT: + assertThat(actual.getType()) + .as("Variant primitive type should match") + .isEqualTo(VariantUtil.Type.FLOAT); + assertThat(actual.getFloat()) + .as("Variant primitive value should match") + .isEqualTo(expected.asPrimitive().get()); + break; + case BINARY: + assertThat(actual.getType()) + .as("Variant primitive type should match") + .isEqualTo(VariantUtil.Type.BINARY); + assertThat(ByteBuffer.wrap(actual.getBinary())) + .as("Variant primitive value should match") + .isEqualTo(expected.asPrimitive().get()); + break; + case STRING: + assertThat(actual.getType()) + .as("Variant primitive type should match") + .isEqualTo(VariantUtil.Type.STRING); + assertThat(actual.getString()) + .as("Variant primitive value should match") + .isEqualTo(expected.asPrimitive().get()); + break; + case UUID: + assertThat(actual.getType()) + .as("Variant primitive type should match") + .isEqualTo(VariantUtil.Type.UUID); + assertThat(actual.getUuid()) + .as("Variant primitive value should match") + .isEqualTo(expected.asPrimitive().get()); + break; + default: + throw new UnsupportedOperationException("Unsupported variant type: " + expected.type()); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestVectorizedOrcDataReader.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestVectorizedOrcDataReader.java new file mode 100644 index 000000000000..891cb9481ce6 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/TestVectorizedOrcDataReader.java @@ -0,0 +1,167 @@ +/* + * 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.data; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.orc.GenericOrcWriter; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.orc.ORC; +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.Iterators; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders; +import org.apache.iceberg.types.Types; +import org.apache.orc.OrcConf; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestVectorizedOrcDataReader { + @TempDir public static Path temp; + + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "binary", Types.BinaryType.get()), + Types.NestedField.required( + 4, "array", Types.ListType.ofOptional(5, Types.IntegerType.get()))); + private static OutputFile outputFile; + + @BeforeAll + public static void createDataFile() throws IOException { + GenericRecord bufferRecord = GenericRecord.create(SCHEMA); + + ImmutableList.Builder builder = ImmutableList.builder(); + builder.add( + bufferRecord.copy( + ImmutableMap.of("id", 1L, "data", "a", "array", Collections.singletonList(1)))); + builder.add( + bufferRecord.copy(ImmutableMap.of("id", 2L, "data", "b", "array", Arrays.asList(2, 3)))); + builder.add( + bufferRecord.copy(ImmutableMap.of("id", 3L, "data", "c", "array", Arrays.asList(3, 4, 5)))); + builder.add( + bufferRecord.copy( + ImmutableMap.of("id", 4L, "data", "d", "array", Arrays.asList(4, 5, 6, 7)))); + builder.add( + bufferRecord.copy( + ImmutableMap.of("id", 5L, "data", "e", "array", Arrays.asList(5, 6, 7, 8, 9)))); + + outputFile = Files.localOutput(File.createTempFile("test", ".orc", temp.toFile())); + + try (DataWriter dataWriter = + ORC.writeData(outputFile) + .schema(SCHEMA) + .createWriterFunc(GenericOrcWriter::buildWriter) + .overwrite() + .withSpec(PartitionSpec.unpartitioned()) + .build()) { + for (Record record : builder.build()) { + dataWriter.write(record); + } + } + } + + private Iterator batchesToRows(Iterator batches) { + return Iterators.concat(Iterators.transform(batches, ColumnarBatch::rowIterator)); + } + + private void validateAllRows(Iterator rows) { + long rowCount = 0; + long expId = 1; + char expChar = 'a'; + while (rows.hasNext()) { + InternalRow row = rows.next(); + assertThat(row.getLong(0)).isEqualTo(expId); + assertThat(row.getString(1)).isEqualTo(Character.toString(expChar)); + assertThat(row.isNullAt(2)).isTrue(); + expId += 1; + expChar += 1; + rowCount += 1; + } + assertThat(rowCount).isEqualTo(5); + } + + @Test + public void testReader() throws IOException { + try (CloseableIterable reader = + ORC.read(outputFile.toInputFile()) + .project(SCHEMA) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader(SCHEMA, readOrcSchema, ImmutableMap.of())) + .build()) { + validateAllRows(batchesToRows(reader.iterator())); + } + } + + @Test + public void testReaderWithFilter() throws IOException { + try (CloseableIterable reader = + ORC.read(outputFile.toInputFile()) + .project(SCHEMA) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader(SCHEMA, readOrcSchema, ImmutableMap.of())) + .filter(Expressions.equal("id", 3L)) + .config(OrcConf.ALLOW_SARG_TO_FILTER.getAttribute(), String.valueOf(true)) + .build()) { + validateAllRows(batchesToRows(reader.iterator())); + } + } + + @Test + public void testWithFilterWithSelected() throws IOException { + try (CloseableIterable reader = + ORC.read(outputFile.toInputFile()) + .project(SCHEMA) + .createBatchedReaderFunc( + readOrcSchema -> + VectorizedSparkOrcReaders.buildReader(SCHEMA, readOrcSchema, ImmutableMap.of())) + .filter(Expressions.equal("id", 3L)) + .config(OrcConf.ALLOW_SARG_TO_FILTER.getAttribute(), String.valueOf(true)) + .config(OrcConf.READER_USE_SELECTED.getAttribute(), String.valueOf(true)) + .build()) { + Iterator rows = batchesToRows(reader.iterator()); + assertThat(rows).hasNext(); + InternalRow row = rows.next(); + assertThat(row.getLong(0)).isEqualTo(3L); + assertThat(row.getString(1)).isEqualTo("c"); + assertThat(row.getArray(3).toIntArray()).isEqualTo(new int[] {3, 4, 5}); + assertThat(rows).isExhausted(); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/TestColumnarBatchUtil.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/TestColumnarBatchUtil.java new file mode 100644 index 000000000000..dadbe3e788b7 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/TestColumnarBatchUtil.java @@ -0,0 +1,302 @@ +/* + * 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.data.vectorized; + +import static java.util.Collections.nCopies; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.util.Arrays; +import java.util.function.Predicate; +import java.util.stream.Stream; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.deletes.PositionDeleteIndex; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class TestColumnarBatchUtil { + + private ColumnVector[] columnVectors; + private DeleteFilter deleteFilter; + + @BeforeEach + public void before() { + columnVectors = mockColumnVector(); + deleteFilter = mock(DeleteFilter.class); + } + + @Test + public void testBuildRowIdMappingNoDeletes() { + when(deleteFilter.hasPosDeletes()).thenReturn(true); + PositionDeleteIndex deletedRowPos = mock(PositionDeleteIndex.class); + + for (long i = 0; i <= 10; i++) { + when(deletedRowPos.isDeleted(i)).thenReturn(false); + } + + when(deleteFilter.deletedRowPositions()).thenReturn(deletedRowPos); + var rowIdMapping = ColumnarBatchUtil.buildRowIdMapping(columnVectors, deleteFilter, 0, 10); + assertThat(rowIdMapping).isNull(); + } + + @Test + public void testBuildRowIdMappingPositionDeletesOnly() { + when(deleteFilter.hasPosDeletes()).thenReturn(true); + PositionDeleteIndex deletedRowPos = mock(PositionDeleteIndex.class); + + // 5 position deletes + for (long i = 98; i < 103; i++) { + when(deletedRowPos.isDeleted(i)).thenReturn(true); + } + + when(deleteFilter.deletedRowPositions()).thenReturn(deletedRowPos); + + var rowIdMapping = ColumnarBatchUtil.buildRowIdMapping(columnVectors, deleteFilter, 0, 200); + assertThat(rowIdMapping).isNotNull(); + + int[] rowIds = (int[]) rowIdMapping.first(); + int liveRows = (int) rowIdMapping.second(); + + for (int id : rowIds) { + assertThat(id < 98 || id > 102).isTrue(); + } + + assertThat(rowIds.length).isEqualTo(200); + assertThat(liveRows).isEqualTo(195); + } + + @Test + public void testBuildRowIdMappingEqualityDeletesOnly() { + // Define raw equality delete predicate — delete rows where value == 42 + Predicate rawEqDelete = row -> row.getInt(0) == 42; + + // Mimic real eqDeletedRowFilter(): keep row only if it does NOT match delete condition + Predicate eqDeletePredicate = + Stream.of(rawEqDelete).map(Predicate::negate).reduce(Predicate::and).orElse(t -> true); + + // Mock DeleteFilter + when(deleteFilter.hasPosDeletes()).thenReturn(false); + when(deleteFilter.deletedRowPositions()).thenReturn(null); + when(deleteFilter.eqDeletedRowFilter()).thenReturn(eqDeletePredicate); + + var rowIdMapping = ColumnarBatchUtil.buildRowIdMapping(columnVectors, deleteFilter, 0, 5); + + assertThat(rowIdMapping).isNotNull(); + int[] rowIds = (int[]) rowIdMapping.first(); + int liveRows = (Integer) rowIdMapping.second(); + + // Expect to keep positions 0, 1, 3, 4 → values 40, 41, 43, 44 + assertThat(liveRows).isEqualTo(4); + assertThat(Arrays.copyOf(rowIds, liveRows)).containsExactly(0, 1, 3, 4); + } + + @Test + public void testBuildRowIdMappingPositionAndEqualityDeletes() { + + // Define raw equality delete predicate — delete rows where value == 42 + Predicate rawEqDelete = row -> row.getInt(0) == 42; + + // Mimic real eqDeletedRowFilter(): keep row only if it does NOT match delete condition + Predicate eqDeletePredicate = + Stream.of(rawEqDelete).map(Predicate::negate).reduce(Predicate::and).orElse(t -> true); + when(deleteFilter.eqDeletedRowFilter()).thenReturn(eqDeletePredicate); + + PositionDeleteIndex deletedRowPos = mock(PositionDeleteIndex.class); + when(deletedRowPos.isDeleted(1)).thenReturn(true); // 41 + when(deletedRowPos.isDeleted(4)).thenReturn(true); // 44 + when(deleteFilter.hasPosDeletes()).thenReturn(true); + when(deleteFilter.deletedRowPositions()).thenReturn(deletedRowPos); + + var rowIdMapping = ColumnarBatchUtil.buildRowIdMapping(columnVectors, deleteFilter, 0, 5); + + assertThat(rowIdMapping).isNotNull(); + int[] rowIds = (int[]) rowIdMapping.first(); + int liveRows = (Integer) rowIdMapping.second(); + + assertThat(liveRows).isEqualTo(2); + assertThat(Arrays.copyOf(rowIds, liveRows)).containsExactly(0, 3); + } + + @Test + void testBuildRowIdMappingEmptyColumVectors() { + ColumnVector[] columnVectorsZero = new ColumnVector[0]; + + PositionDeleteIndex deletedRowPos = mock(PositionDeleteIndex.class); + when(deletedRowPos.isDeleted(1)).thenReturn(true); + when(deletedRowPos.isDeleted(4)).thenReturn(true); + when(deleteFilter.hasPosDeletes()).thenReturn(true); + when(deleteFilter.deletedRowPositions()).thenReturn(deletedRowPos); + + var rowIdMapping = ColumnarBatchUtil.buildRowIdMapping(columnVectorsZero, deleteFilter, 0, 0); + + // Empty batch size, expect no rows deleted. + assertThat(rowIdMapping).isNull(); + } + + @Test + void testBuildRowIdMapAllRowsDeleted() { + + // Define raw equality delete predicate — delete rows where value == 42 or 43 + Predicate rawEqDelete = row -> row.getInt(0) == 42 || row.getInt(0) == 43; + + // Mimic real eqDeletedRowFilter(): keep row only if it does NOT match delete condition + Predicate eqDeletePredicate = + Stream.of(rawEqDelete).map(Predicate::negate).reduce(Predicate::and).orElse(t -> true); + when(deleteFilter.eqDeletedRowFilter()).thenReturn(eqDeletePredicate); + + PositionDeleteIndex deletedRowPos = mock(PositionDeleteIndex.class); + when(deletedRowPos.isDeleted(0)).thenReturn(true); // 40 + when(deletedRowPos.isDeleted(1)).thenReturn(true); // 41 + when(deletedRowPos.isDeleted(4)).thenReturn(true); // 44 + when(deleteFilter.hasPosDeletes()).thenReturn(true); + when(deleteFilter.deletedRowPositions()).thenReturn(deletedRowPos); + + var rowIdMapping = ColumnarBatchUtil.buildRowIdMapping(columnVectors, deleteFilter, 0, 5); + + assertThat(rowIdMapping).isNotNull(); + int[] rowIds = (int[]) rowIdMapping.first(); + int liveRows = (Integer) rowIdMapping.second(); + + // Expect all rows to be deleted + assertThat(liveRows).isEqualTo(0); + assertThat(rowIds).containsExactly(0, 0, 0, 0, 0); + } + + @Test + void testBuildIsDeletedPositionDeletes() { + PositionDeleteIndex deletedRowPos = mock(PositionDeleteIndex.class); + when(deleteFilter.deletedRowPositions()).thenReturn(deletedRowPos); + + for (long i = 98; i < 100; i++) { + when(deletedRowPos.isDeleted(i)).thenReturn(true); + } + + var isDeleted = ColumnarBatchUtil.buildIsDeleted(columnVectors, deleteFilter, 0, 100); + + assertThat(isDeleted).isNotNull(); + assertThat(isDeleted.length).isEqualTo(100); + + for (int i = 98; i < 100; i++) { + assertThat(isDeleted[i]).isTrue(); + } + + for (int i = 0; i < 98; i++) { + assertThat(isDeleted[i]).isFalse(); + } + } + + @Test + void testBuildIsDeletedEqualityDeletes() { + // Define raw equality delete predicate — delete rows where value == 42 or 43 + Predicate rawEqDelete = row -> row.getInt(0) == 42 || row.getInt(0) == 43; + + // Mimic real eqDeletedRowFilter(): keep row only if it does NOT match delete condition + Predicate eqDeletePredicate = + Stream.of(rawEqDelete).map(Predicate::negate).reduce(Predicate::and).orElse(t -> true); + when(deleteFilter.eqDeletedRowFilter()).thenReturn(eqDeletePredicate); + + var isDeleted = ColumnarBatchUtil.buildIsDeleted(columnVectors, deleteFilter, 0, 5); + + for (int i = 0; i < isDeleted.length; i++) { + if (i == 2 || i == 3) { // 42 and 43 + assertThat(isDeleted[i]).isTrue(); + } else { + assertThat(isDeleted[i]).isFalse(); + } + } + } + + @Test + void testBuildIsDeletedPositionAndEqualityDeletes() { + // Define raw equality delete predicate — delete rows where value == 42 + Predicate rawEqDelete = row -> row.getInt(0) == 42; + + // Mimic real eqDeletedRowFilter(): keep row only if it does NOT match delete condition + Predicate eqDeletePredicate = + Stream.of(rawEqDelete).map(Predicate::negate).reduce(Predicate::and).orElse(t -> true); + when(deleteFilter.eqDeletedRowFilter()).thenReturn(eqDeletePredicate); + + PositionDeleteIndex deletedRowPos = mock(PositionDeleteIndex.class); + when(deletedRowPos.isDeleted(1)).thenReturn(true); // 41 + when(deletedRowPos.isDeleted(4)).thenReturn(true); // 44 + when(deleteFilter.hasPosDeletes()).thenReturn(true); + when(deleteFilter.deletedRowPositions()).thenReturn(deletedRowPos); + + var isDeleted = ColumnarBatchUtil.buildIsDeleted(columnVectors, deleteFilter, 0, 5); + + for (int i = 0; i < isDeleted.length; i++) { + if (i == 0 || i == 3) { + assertThat(isDeleted[i]).isFalse(); + } else { + assertThat(isDeleted[i]).isTrue(); // 42, 41, 44 are deleted + } + } + } + + @Test + void testBuildIsDeletedNoDeletes() { + var result = ColumnarBatchUtil.buildIsDeleted(columnVectors, null, 0, 5); + assertThat(result).isNotNull(); + for (int i = 0; i < 5; i++) { + assertThat(result[i]).isFalse(); + } + } + + @Test + void testRemoveExtraColumns() { + ColumnVector[] vectors = new ColumnVector[5]; + for (int i = 0; i < 5; i++) { + vectors[i] = mock(ColumnVector.class); + } + when(deleteFilter.expectedSchema()).thenReturn(mock(Schema.class)); + when(deleteFilter.expectedSchema().columns()).thenReturn(nCopies(3, null)); + + ColumnVector[] result = ColumnarBatchUtil.removeExtraColumns(deleteFilter, vectors); + assertThat(result.length).isEqualTo(3); + } + + @Test + void testRemoveExtraColumnsNotNeeded() { + ColumnVector[] vectors = new ColumnVector[3]; + for (int i = 0; i < 3; i++) { + vectors[i] = mock(ColumnVector.class); + } + when(deleteFilter.expectedSchema()).thenReturn(mock(Schema.class)); + when(deleteFilter.expectedSchema().columns()).thenReturn(nCopies(3, null)); + + ColumnVector[] result = ColumnarBatchUtil.removeExtraColumns(deleteFilter, vectors); + assertThat(result.length).isEqualTo(3); + } + + private ColumnVector[] mockColumnVector() { + // Create a mocked Int column vector with values: 40, 41, 42, 43, 44 + ColumnVector intVector = mock(ColumnVector.class); + when(intVector.getInt(0)).thenReturn(40); + when(intVector.getInt(1)).thenReturn(41); + when(intVector.getInt(2)).thenReturn(42); + when(intVector.getInt(3)).thenReturn(43); + when(intVector.getInt(4)).thenReturn(44); + + return new ColumnVector[] {intVector}; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java new file mode 100644 index 000000000000..284fa0b0552f --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryEncodedVectorizedReads.java @@ -0,0 +1,199 @@ +/* + * 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.data.vectorized.parquet; + +import static org.apache.iceberg.TableProperties.PARQUET_DICT_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.PARQUET_PAGE_ROW_LIMIT; +import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Iterator; +import java.util.List; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.base.Function; +import org.apache.iceberg.relocated.com.google.common.collect.FluentIterable; +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.Iterables; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +public class TestParquetDictionaryEncodedVectorizedReads extends TestParquetVectorizedReads { + + protected static SparkSession spark = null; + + @BeforeAll + public static void startSpark() { + spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + if (spark != null) { + spark.stop(); + spark = null; + } + } + + @Override + Iterable generateData( + Schema schema, + int numRecords, + long seed, + float nullPercentage, + Function transform) { + Iterable data = + RandomGenericData.generateDictionaryEncodableRecords( + schema, numRecords, seed, nullPercentage); + return transform == IDENTITY ? data : Iterables.transform(data, transform); + } + + @Test + @Override + @Disabled // Ignored since this code path is already tested in TestParquetVectorizedReads + public void testVectorizedReadsWithNewContainers() throws IOException {} + + @Test + public void testMixedDictionaryNonDictionaryReads() throws IOException { + Schema schema = new Schema(SUPPORTED_PRIMITIVES.fields()); + File dictionaryEncodedFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(dictionaryEncodedFile.delete()).as("Delete should succeed").isTrue(); + Iterable dictionaryEncodableData = + RandomGenericData.generateDictionaryEncodableRecords(schema, 10000, 0L); + try (FileAppender writer = getParquetWriter(schema, dictionaryEncodedFile)) { + writer.addAll(dictionaryEncodableData); + } + + File plainEncodingFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(plainEncodingFile.delete()).as("Delete should succeed").isTrue(); + Iterable nonDictionaryData = RandomGenericData.generate(schema, 10000, 0L); + try (FileAppender writer = getParquetWriter(schema, plainEncodingFile)) { + writer.addAll(nonDictionaryData); + } + + int rowGroupSize = PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT; + File mixedFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(mixedFile.delete()).as("Delete should succeed").isTrue(); + Parquet.concat( + ImmutableList.of(dictionaryEncodedFile, plainEncodingFile, dictionaryEncodedFile), + mixedFile, + rowGroupSize, + schema, + ImmutableMap.of()); + assertRecordsMatch( + schema, + 30000, + FluentIterable.concat(dictionaryEncodableData, nonDictionaryData, dictionaryEncodableData), + mixedFile, + true, + BATCH_SIZE); + } + + @Test + public void testBinaryNotAllPagesDictionaryEncoded() throws IOException { + Schema schema = new Schema(Types.NestedField.required(1, "bytes", Types.BinaryType.get())); + File parquetFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(parquetFile.delete()).as("Delete should succeed").isTrue(); + + Iterable records = RandomGenericData.generateFallbackRecords(schema, 500, 0L, 100); + try (FileAppender writer = + Parquet.write(Files.localOutput(parquetFile)) + .schema(schema) + .set(PARQUET_DICT_SIZE_BYTES, "4096") + .set(PARQUET_PAGE_ROW_LIMIT, "100") + .createWriterFunc(GenericParquetWriter::create) + .build()) { + writer.addAll(records); + } + + // After the above, parquetFile contains one column chunk of binary data in five pages, + // the first two RLE dictionary encoded, and the remaining three plain encoded. + assertRecordsMatch(schema, 500, records, parquetFile, true, BATCH_SIZE); + } + + /** + * decimal_dict_and_plain_encoding.parquet contains one column chunk of decimal(38, 0) data in two + * pages, one RLE dictionary encoded and one plain encoded, each with 200 rows. + */ + @Test + public void testDecimalNotAllPagesDictionaryEncoded() throws Exception { + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.DecimalType.of(38, 0))); + Path path = + Paths.get( + getClass() + .getClassLoader() + .getResource("decimal_dict_and_plain_encoding.parquet") + .toURI()); + + Dataset df = spark.read().parquet(path.toString()); + List expected = df.collectAsList(); + long expectedSize = df.count(); + + assertNoLeak( + "testDecimalNotAllPagesDictionaryEncoded", + allocator -> { + Parquet.ReadBuilder readBuilder = + Parquet.read(Files.localInput(path.toFile())) + .project(schema) + .createBatchedReaderFunc( + type -> + VectorizedSparkParquetReaders.buildReader( + schema, type, ImmutableMap.of(), allocator)); + + try (CloseableIterable batchReader = readBuilder.build()) { + Iterator expectedIter = expected.iterator(); + Iterator batches = batchReader.iterator(); + int numRowsRead = 0; + while (batches.hasNext()) { + ColumnarBatch batch = batches.next(); + numRowsRead += batch.numRows(); + TestHelpers.assertEqualsBatchWithRows(schema.asStruct(), expectedIter, batch); + } + assertThat(numRowsRead).isEqualTo(expectedSize); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryFallbackToPlainEncodingVectorizedReads.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryFallbackToPlainEncodingVectorizedReads.java new file mode 100644 index 000000000000..9f9c2b961d38 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetDictionaryFallbackToPlainEncodingVectorizedReads.java @@ -0,0 +1,77 @@ +/* + * 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.data.vectorized.parquet; + +import java.io.File; +import java.io.IOException; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.base.Function; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; + +public class TestParquetDictionaryFallbackToPlainEncodingVectorizedReads + extends TestParquetVectorizedReads { + private static final int NUM_ROWS = 1_000_000; + + @Override + protected int getNumRows() { + return NUM_ROWS; + } + + @Override + Iterable generateData( + Schema schema, + int numRecords, + long seed, + float nullPercentage, + Function transform) { + // TODO: take into account nullPercentage when generating fallback encoding data + Iterable data = + RandomGenericData.generateFallbackRecords(schema, numRecords, seed, numRecords / 20); + return transform == IDENTITY ? data : Iterables.transform(data, transform); + } + + @Override + FileAppender getParquetWriter(Schema schema, File testFile) throws IOException { + return Parquet.write(Files.localOutput(testFile)) + .schema(schema) + .createWriterFunc(GenericParquetWriter::create) + .named("test") + .set(TableProperties.PARQUET_DICT_SIZE_BYTES, "512000") + .build(); + } + + @Test + @Override + @Disabled // Fallback encoding not triggered when data is mostly null + public void testMostlyNullsForOptionalFields() {} + + @Test + @Override + @Disabled // Ignored since this code path is already tested in TestParquetVectorizedReads + public void testVectorizedReadsWithNewContainers() throws IOException {} +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java new file mode 100644 index 000000000000..46a6a302e1c4 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/data/vectorized/parquet/TestParquetVectorizedReads.java @@ -0,0 +1,548 @@ +/* + * 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.data.vectorized.parquet; + +import static java.nio.file.StandardCopyOption.REPLACE_EXISTING; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +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.io.InputStream; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.function.Consumer; +import java.util.stream.Stream; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.arrow.ArrowAllocation; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.base.Function; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +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.ImmutableMap; +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.data.AvroDataTestBase; +import org.apache.iceberg.spark.data.GenericsHelpers; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.SparkParquetReaders; +import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; +import org.apache.iceberg.types.Type.PrimitiveType; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.Type; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +public class TestParquetVectorizedReads extends AvroDataTestBase { + private static final int NUM_ROWS = 200_000; + static final int BATCH_SIZE = 10_000; + + private static final String PLAIN = "PLAIN"; + private static final List GOLDEN_FILE_ENCODINGS = + ImmutableList.of("PLAIN_DICTIONARY", "RLE_DICTIONARY", "DELTA_BINARY_PACKED"); + private static final Map GOLDEN_FILE_TYPES = + ImmutableMap.of( + "string", Types.StringType.get(), + "float", Types.FloatType.get(), + "int32", Types.IntegerType.get(), + "int64", Types.LongType.get(), + "binary", Types.BinaryType.get(), + "boolean", Types.BooleanType.get()); + + static final Function IDENTITY = record -> record; + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + writeAndValidate(schema, schema); + } + + @Override + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + writeAndValidate( + writeSchema, + expectedSchema, + getNumRows(), + 29714278L, + RandomData.DEFAULT_NULL_PERCENTAGE, + true, + BATCH_SIZE, + IDENTITY); + } + + @Override + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema, List records) + throws IOException { + writeAndValidate( + writeSchema, expectedSchema, true, BATCH_SIZE, records.size(), records, ID_TO_CONSTANT); + } + + @Override + protected boolean supportsDefaultValues() { + return true; + } + + @Override + protected boolean supportsNestedTypes() { + return false; + } + + @Override + protected boolean supportsRowLineage() { + return true; + } + + private void writeAndValidate( + Schema schema, int numRecords, long seed, float nullPercentage, boolean reuseContainers) + throws IOException { + writeAndValidate( + schema, schema, numRecords, seed, nullPercentage, reuseContainers, BATCH_SIZE, IDENTITY); + } + + private void writeAndValidate( + Schema writeSchema, + Schema expectedSchema, + int numRecords, + long seed, + float nullPercentage, + boolean reuseContainers, + int batchSize, + Function transform) + throws IOException { + writeAndValidate( + writeSchema, + expectedSchema, + numRecords, + seed, + nullPercentage, + reuseContainers, + batchSize, + transform, + ImmutableMap.of()); + } + + private void writeAndValidate( + Schema writeSchema, + Schema expectedSchema, + int numRecords, + long seed, + float nullPercentage, + boolean reuseContainers, + int batchSize, + Function transform, + Map idToConstant) + throws IOException { + // Write test data + assumeThat( + TypeUtil.find( + writeSchema, + type -> type.isMapType() && type.asMapType().keyType() != Types.StringType.get())) + .as("Parquet Avro cannot write non-string map keys") + .isNull(); + + Iterable expected = + generateData(writeSchema, numRecords, seed, nullPercentage, transform); + + writeAndValidate( + writeSchema, + expectedSchema, + reuseContainers, + batchSize, + numRecords, + expected, + idToConstant); + } + + private void writeAndValidate( + Schema writeSchema, + Schema expectedSchema, + boolean reuseContainers, + int batchSize, + int numRecords, + Iterable expected, + Map idToConstant) + throws IOException { + // write a test parquet file using iceberg writer + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + + try (FileAppender writer = getParquetWriter(writeSchema, testFile)) { + writer.addAll(expected); + } + + assertRecordsMatch( + expectedSchema, numRecords, expected, testFile, reuseContainers, batchSize, idToConstant); + } + + protected int getNumRows() { + return NUM_ROWS; + } + + Iterable generateData( + Schema schema, + int numRecords, + long seed, + float nullPercentage, + Function transform) { + Iterable data = RandomGenericData.generate(schema, numRecords, seed); + return transform == IDENTITY ? data : Iterables.transform(data, transform); + } + + FileAppender getParquetWriter(Schema schema, File testFile) throws IOException { + return Parquet.write(Files.localOutput(testFile)) + .schema(schema) + .createWriterFunc(GenericParquetWriter::create) + .named("test") + .build(); + } + + FileAppender getParquetV2Writer(Schema schema, File testFile) throws IOException { + return Parquet.write(Files.localOutput(testFile)) + .schema(schema) + .createWriterFunc(GenericParquetWriter::create) + .named("test") + .writerVersion(ParquetProperties.WriterVersion.PARQUET_2_0) + .build(); + } + + void assertRecordsMatch( + Schema schema, + int expectedSize, + Iterable expected, + File testFile, + boolean reuseContainers, + int batchSize) + throws IOException { + assertRecordsMatch( + schema, expectedSize, expected, testFile, reuseContainers, batchSize, ImmutableMap.of()); + } + + void assertRecordsMatch( + Schema schema, + int expectedSize, + Iterable expected, + File testFile, + boolean reuseContainers, + int batchSize, + Map idToConstant) + throws IOException { + assertNoLeak( + testFile.getName(), + allocator -> { + Parquet.ReadBuilder readBuilder = + Parquet.read(Files.localInput(testFile)) + .project(schema) + .recordsPerBatch(batchSize) + .createBatchedReaderFunc( + type -> + VectorizedSparkParquetReaders.buildReader( + schema, type, idToConstant, allocator)); + if (reuseContainers) { + readBuilder.reuseContainers(); + } + try (CloseableIterable batchReader = readBuilder.build()) { + Iterator expectedIter = expected.iterator(); + Iterator batches = batchReader.iterator(); + int numRowsRead = 0; + while (batches.hasNext()) { + ColumnarBatch batch = batches.next(); + GenericsHelpers.assertEqualsBatch( + schema.asStruct(), expectedIter, batch, idToConstant, numRowsRead); + numRowsRead += batch.numRows(); + } + assertThat(numRowsRead).isEqualTo(expectedSize); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + @Test + @Override + public void testNestedStruct() { + assertThatThrownBy( + () -> + VectorizedSparkParquetReaders.buildReader( + TypeUtil.assignIncreasingFreshIds( + new Schema(required(1, "struct", SUPPORTED_PRIMITIVES))), + new MessageType( + "struct", new GroupType(Type.Repetition.OPTIONAL, "struct").withId(1)), + Maps.newHashMap())) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Vectorized reads are not supported yet for struct fields"); + } + + @Test + public void testMostlyNullsForOptionalFields() throws IOException { + writeAndValidate( + TypeUtil.assignIncreasingFreshIds(new Schema(SUPPORTED_PRIMITIVES.fields())), + getNumRows(), + 0L, + 0.99f, + true); + } + + @Test + public void testSettingArrowValidityVector() throws IOException { + writeAndValidate( + new Schema(Lists.transform(SUPPORTED_PRIMITIVES.fields(), Types.NestedField::asOptional)), + getNumRows(), + 0L, + RandomData.DEFAULT_NULL_PERCENTAGE, + true); + } + + @Test + public void testVectorizedReadsWithNewContainers() throws IOException { + writeAndValidate( + TypeUtil.assignIncreasingFreshIds(new Schema(SUPPORTED_PRIMITIVES.fields())), + getNumRows(), + 0L, + RandomData.DEFAULT_NULL_PERCENTAGE, + false); + } + + @Test + public void testVectorizedReadsWithReallocatedArrowBuffers() throws IOException { + // With a batch size of 2, 256 bytes are allocated in the VarCharVector. By adding strings of + // length 512, the vector will need to be reallocated for storing the batch. + Schema schema = + new Schema( + Lists.newArrayList( + SUPPORTED_PRIMITIVES.field("id"), SUPPORTED_PRIMITIVES.field("data"))); + int dataOrdinal = 1; + + writeAndValidate( + schema, + schema, + 10, + 0L, + RandomData.DEFAULT_NULL_PERCENTAGE, + true, + 2, + record -> { + if (record.get(dataOrdinal, String.class) != null) { + record.set( + dataOrdinal, Strings.padEnd(record.get(dataOrdinal, String.class), 512, 'a')); + } else { + record.set(dataOrdinal, Strings.padEnd("", 512, 'a')); + } + return record; + }); + } + + @Test + public void testReadsForTypePromotedColumns() throws Exception { + Schema writeSchema = + new Schema( + required(100, "id", Types.LongType.get()), + optional(101, "int_data", Types.IntegerType.get()), + optional(102, "float_data", Types.FloatType.get()), + optional(103, "decimal_data", Types.DecimalType.of(10, 5))); + + File dataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(dataFile.delete()).as("Delete should succeed").isTrue(); + Iterable data = + generateData(writeSchema, 30000, 0L, RandomData.DEFAULT_NULL_PERCENTAGE, IDENTITY); + try (FileAppender writer = getParquetWriter(writeSchema, dataFile)) { + writer.addAll(data); + } + + Schema readSchema = + new Schema( + required(100, "id", Types.LongType.get()), + optional(101, "int_data", Types.LongType.get()), + optional(102, "float_data", Types.DoubleType.get()), + optional(103, "decimal_data", Types.DecimalType.of(25, 5))); + + assertRecordsMatch(readSchema, 30000, data, dataFile, false, BATCH_SIZE); + } + + @Test + public void testSupportedReadsForParquetV2() throws Exception { + // Float and double column types are written using plain encoding with Parquet V2, + // also Parquet V2 will dictionary encode decimals that use fixed length binary + // (i.e. decimals > 8 bytes). Int and long types use DELTA_BINARY_PACKED. + Schema schema = + new Schema( + optional(102, "float_data", Types.FloatType.get()), + optional(103, "double_data", Types.DoubleType.get()), + optional(104, "decimal_data", Types.DecimalType.of(25, 5)), + optional(105, "int_data", Types.IntegerType.get()), + optional(106, "long_data", Types.LongType.get())); + + File dataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(dataFile.delete()).as("Delete should succeed").isTrue(); + Iterable data = + generateData(schema, 30000, 0L, RandomData.DEFAULT_NULL_PERCENTAGE, IDENTITY); + try (FileAppender writer = getParquetV2Writer(schema, dataFile)) { + writer.addAll(data); + } + assertRecordsMatch(schema, 30000, data, dataFile, false, BATCH_SIZE); + } + + @Test + public void testUnsupportedReadsForParquetV2() throws Exception { + // Some types use delta encoding and which are not supported for vectorized reads + Schema schema = new Schema(SUPPORTED_PRIMITIVES.fields()); + File dataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(dataFile.delete()).as("Delete should succeed").isTrue(); + Iterable data = + generateData(schema, 30000, 0L, RandomData.DEFAULT_NULL_PERCENTAGE, IDENTITY); + try (FileAppender writer = getParquetV2Writer(schema, dataFile)) { + writer.addAll(data); + } + assertThatThrownBy(() -> assertRecordsMatch(schema, 30000, data, dataFile, false, BATCH_SIZE)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageStartingWith("Cannot support vectorized reads for column") + .hasMessageEndingWith("Disable vectorized reads to read this table/file"); + } + + @Test + public void testUuidReads() throws Exception { + // Just one row to maintain dictionary encoding + int numRows = 1; + Schema schema = new Schema(optional(100, "uuid", Types.UUIDType.get())); + + File dataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(dataFile.delete()).as("Delete should succeed").isTrue(); + Iterable data = generateData(schema, numRows, 0L, 0, IDENTITY); + try (FileAppender writer = getParquetV2Writer(schema, dataFile)) { + writer.addAll(data); + } + assertRecordsMatch(schema, numRows, data, dataFile, false, BATCH_SIZE); + } + + private void assertIdenticalFileContents( + File actual, File expected, Schema schema, boolean vectorized) throws IOException { + try (CloseableIterable expectedIterator = + Parquet.read(Files.localInput(expected)) + .project(schema) + .createReaderFunc(msgType -> GenericParquetReaders.buildReader(schema, msgType)) + .build()) { + List expectedRecords = Lists.newArrayList(expectedIterator); + if (vectorized) { + assertRecordsMatch( + schema, expectedRecords.size(), expectedRecords, actual, false, BATCH_SIZE); + } else { + try (CloseableIterable actualIterator = + Parquet.read(Files.localInput(actual)) + .project(schema) + .createReaderFunc(msgType -> SparkParquetReaders.buildReader(schema, msgType)) + .build()) { + List actualRecords = Lists.newArrayList(actualIterator); + assertThat(actualRecords).hasSameSizeAs(expectedRecords); + for (int i = 0; i < actualRecords.size(); i++) { + GenericsHelpers.assertEqualsUnsafe( + schema.asStruct(), expectedRecords.get(i), actualRecords.get(i)); + } + } + } + } + } + + static Stream goldenFilesAndEncodings() { + return GOLDEN_FILE_ENCODINGS.stream() + .flatMap( + encoding -> + GOLDEN_FILE_TYPES.entrySet().stream() + .flatMap( + e -> + Stream.of(true, false) + .map( + vectorized -> + Arguments.of( + encoding, e.getKey(), e.getValue(), vectorized)))); + } + + private File resourceUrlToLocalFile(URL url) throws IOException, URISyntaxException { + if ("file".equals(url.getProtocol())) { + return Paths.get(url.toURI()).toFile(); + } + + String name = Paths.get(url.getPath()).getFileName().toString(); // e.g., string.parquet + String suffix = name.contains(".") ? name.substring(name.lastIndexOf('.')) : ""; + File tmp = File.createTempFile("golden-", suffix, temp.toFile()); + try (InputStream in = url.openStream()) { + java.nio.file.Files.copy(in, tmp.toPath(), REPLACE_EXISTING); + } + return tmp; + } + + @ParameterizedTest + @MethodSource("goldenFilesAndEncodings") + public void testGoldenFiles( + String encoding, String typeName, PrimitiveType primitiveType, boolean vectorized) + throws Exception { + Path goldenResourcePath = Paths.get("encodings", encoding, typeName + ".parquet"); + URL goldenFileUrl = getClass().getClassLoader().getResource(goldenResourcePath.toString()); + assumeThat(goldenFileUrl).as("type/encoding pair exists").isNotNull(); + + Path plainResourcePath = Paths.get("encodings", PLAIN, typeName + ".parquet"); + URL plainFileUrl = getClass().getClassLoader().getResource(plainResourcePath.toString()); + Preconditions.checkState( + plainFileUrl != null, "PLAIN encoded file should exist: " + plainResourcePath); + + Schema expectedSchema = new Schema(optional(1, "data", primitiveType)); + assertIdenticalFileContents( + resourceUrlToLocalFile(goldenFileUrl), + resourceUrlToLocalFile(plainFileUrl), + expectedSchema, + vectorized); + } + + protected void assertNoLeak(String testName, Consumer testFunction) { + BufferAllocator allocator = + ArrowAllocation.rootAllocator().newChildAllocator(testName, 0, Long.MAX_VALUE); + try { + testFunction.accept(allocator); + assertThat(allocator.getAllocatedMemory()) + .as( + "Should have released all memory prior to closing. Expected to find 0 bytes of memory in use.") + .isEqualTo(0L); + } finally { + allocator.close(); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java new file mode 100644 index 000000000000..38ce0d4d95f1 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java @@ -0,0 +1,158 @@ +/* + * 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.functions; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; +import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.DecimalType; +import org.junit.jupiter.api.Test; + +public class TestSparkFunctions { + + @Test + public void testBuildYearsFunctionFromClass() { + UnboundFunction expected = new YearsFunction(); + + YearsFunction.DateToYearsFunction dateToYearsFunc = new YearsFunction.DateToYearsFunction(); + checkBuildFunc(dateToYearsFunc, expected); + + YearsFunction.TimestampToYearsFunction tsToYearsFunc = + new YearsFunction.TimestampToYearsFunction(); + checkBuildFunc(tsToYearsFunc, expected); + + YearsFunction.TimestampNtzToYearsFunction tsNtzToYearsFunc = + new YearsFunction.TimestampNtzToYearsFunction(); + checkBuildFunc(tsNtzToYearsFunc, expected); + } + + @Test + public void testBuildMonthsFunctionFromClass() { + UnboundFunction expected = new MonthsFunction(); + + MonthsFunction.DateToMonthsFunction dateToMonthsFunc = + new MonthsFunction.DateToMonthsFunction(); + checkBuildFunc(dateToMonthsFunc, expected); + + MonthsFunction.TimestampToMonthsFunction tsToMonthsFunc = + new MonthsFunction.TimestampToMonthsFunction(); + checkBuildFunc(tsToMonthsFunc, expected); + + MonthsFunction.TimestampNtzToMonthsFunction tsNtzToMonthsFunc = + new MonthsFunction.TimestampNtzToMonthsFunction(); + checkBuildFunc(tsNtzToMonthsFunc, expected); + } + + @Test + public void testBuildDaysFunctionFromClass() { + UnboundFunction expected = new DaysFunction(); + + DaysFunction.DateToDaysFunction dateToDaysFunc = new DaysFunction.DateToDaysFunction(); + checkBuildFunc(dateToDaysFunc, expected); + + DaysFunction.TimestampToDaysFunction tsToDaysFunc = new DaysFunction.TimestampToDaysFunction(); + checkBuildFunc(tsToDaysFunc, expected); + + DaysFunction.TimestampNtzToDaysFunction tsNtzToDaysFunc = + new DaysFunction.TimestampNtzToDaysFunction(); + checkBuildFunc(tsNtzToDaysFunc, expected); + } + + @Test + public void testBuildHoursFunctionFromClass() { + UnboundFunction expected = new HoursFunction(); + + HoursFunction.TimestampToHoursFunction tsToHoursFunc = + new HoursFunction.TimestampToHoursFunction(); + checkBuildFunc(tsToHoursFunc, expected); + + HoursFunction.TimestampNtzToHoursFunction tsNtzToHoursFunc = + new HoursFunction.TimestampNtzToHoursFunction(); + checkBuildFunc(tsNtzToHoursFunc, expected); + } + + @Test + public void testBuildBucketFunctionFromClass() { + UnboundFunction expected = new BucketFunction(); + + BucketFunction.BucketInt bucketDateFunc = new BucketFunction.BucketInt(DataTypes.DateType); + checkBuildFunc(bucketDateFunc, expected); + + BucketFunction.BucketInt bucketIntFunc = new BucketFunction.BucketInt(DataTypes.IntegerType); + checkBuildFunc(bucketIntFunc, expected); + + BucketFunction.BucketLong bucketLongFunc = new BucketFunction.BucketLong(DataTypes.LongType); + checkBuildFunc(bucketLongFunc, expected); + + BucketFunction.BucketLong bucketTsFunc = new BucketFunction.BucketLong(DataTypes.TimestampType); + checkBuildFunc(bucketTsFunc, expected); + + BucketFunction.BucketLong bucketTsNtzFunc = + new BucketFunction.BucketLong(DataTypes.TimestampNTZType); + checkBuildFunc(bucketTsNtzFunc, expected); + + BucketFunction.BucketDecimal bucketDecimalFunc = + new BucketFunction.BucketDecimal(new DecimalType()); + checkBuildFunc(bucketDecimalFunc, expected); + + BucketFunction.BucketString bucketStringFunc = new BucketFunction.BucketString(); + checkBuildFunc(bucketStringFunc, expected); + + BucketFunction.BucketBinary bucketBinary = new BucketFunction.BucketBinary(); + checkBuildFunc(bucketBinary, expected); + } + + @Test + public void testBuildTruncateFunctionFromClass() { + UnboundFunction expected = new TruncateFunction(); + + TruncateFunction.TruncateTinyInt truncateTinyIntFunc = new TruncateFunction.TruncateTinyInt(); + checkBuildFunc(truncateTinyIntFunc, expected); + + TruncateFunction.TruncateSmallInt truncateSmallIntFunc = + new TruncateFunction.TruncateSmallInt(); + checkBuildFunc(truncateSmallIntFunc, expected); + + TruncateFunction.TruncateInt truncateIntFunc = new TruncateFunction.TruncateInt(); + checkBuildFunc(truncateIntFunc, expected); + + TruncateFunction.TruncateBigInt truncateBigIntFunc = new TruncateFunction.TruncateBigInt(); + checkBuildFunc(truncateBigIntFunc, expected); + + TruncateFunction.TruncateDecimal truncateDecimalFunc = + new TruncateFunction.TruncateDecimal(10, 9); + checkBuildFunc(truncateDecimalFunc, expected); + + TruncateFunction.TruncateString truncateStringFunc = new TruncateFunction.TruncateString(); + checkBuildFunc(truncateStringFunc, expected); + + TruncateFunction.TruncateBinary truncateBinaryFunc = new TruncateFunction.TruncateBinary(); + checkBuildFunc(truncateBinaryFunc, expected); + } + + private void checkBuildFunc(ScalarFunction function, UnboundFunction expected) { + UnboundFunction actual = SparkFunctions.loadFunctionByClass(function.getClass()); + + assertThat(actual).isNotNull(); + assertThat(actual.name()).isEqualTo(expected.name()); + assertThat(actual.description()).isEqualTo(expected.description()); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java new file mode 100644 index 000000000000..e5d03a4efba5 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/BatchReaderUtil.java @@ -0,0 +1,34 @@ +/* + * 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.data.DeleteFilter; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +public class BatchReaderUtil { + private BatchReaderUtil() {} + + public static CloseableIterable applyDeleteFilter( + CloseableIterable batches, DeleteFilter filter) { + return CloseableIterable.transform( + batches, new BaseBatchReader.BatchDeleteFilter(filter)::filterBatch); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ComplexRecord.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ComplexRecord.java new file mode 100644 index 000000000000..42e8552578cd --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ComplexRecord.java @@ -0,0 +1,74 @@ +/* + * 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.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +public class ComplexRecord { + private long id; + private NestedRecord struct; + + public ComplexRecord() {} + + public ComplexRecord(long id, NestedRecord struct) { + this.id = id; + this.struct = struct; + } + + public long getId() { + return id; + } + + public void setId(long id) { + this.id = id; + } + + public NestedRecord getStruct() { + return struct; + } + + public void setStruct(NestedRecord struct) { + this.struct = struct; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + ComplexRecord record = (ComplexRecord) o; + return id == record.id && Objects.equal(struct, record.struct); + } + + @Override + public int hashCode() { + return Objects.hashCode(id, struct); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("id", id).add("struct", struct).toString(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java new file mode 100644 index 000000000000..9b5d8f5978ba --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/DataFrameWriteTestBase.java @@ -0,0 +1,149 @@ +/* + * 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.spark.SparkSchemaUtil.convert; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.Iterator; +import java.util.List; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.Tables; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataWriter; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.data.GenericsHelpers; +import org.apache.iceberg.spark.data.SparkPlannedAvroReader; +import org.apache.iceberg.types.Types; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.DataFrameWriter; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public abstract class DataFrameWriteTestBase extends ScanTestBase { + @TempDir private Path temp; + + @Override + protected boolean supportsDefaultValues() { + // disable default value tests because this tests the write path + return false; + } + + @Override + protected void writeRecords(Table table, List records) throws IOException { + Schema tableSchema = table.schema(); // use the table schema because ids are reassigned + + Dataset df = createDataset(records, tableSchema); + DataFrameWriter writer = df.write().format("iceberg").mode("append"); + + writer.save(table.location()); + + // refresh the in-memory table state to pick up Spark's write + table.refresh(); + } + + private Dataset createDataset(List records, Schema schema) throws IOException { + // this uses the SparkAvroReader to create a DataFrame from the list of records + // it assumes that SparkAvroReader is correct + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + + try (FileAppender writer = + Avro.write(Files.localOutput(testFile)) + .schema(schema) + .createWriterFunc(DataWriter::create) + .named("test") + .build()) { + writer.addAll(records); + } + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .createResolvingReader(SparkPlannedAvroReader::create) + .project(schema) + .build()) { + rows = Lists.newArrayList(reader); + } + + // verify that the dataframe matches + assertThat(rows).hasSameSizeAs(records); + Iterator recordIter = records.iterator(); + for (InternalRow row : rows) { + GenericsHelpers.assertEqualsUnsafe(schema.asStruct(), recordIter.next(), row); + } + + JavaRDD rdd = sc.parallelize(rows); + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + + return ((org.apache.spark.sql.classic.SparkSession) spark) + .internalCreateDataFrame(JavaRDD.toRDD(rdd), convert(schema), false); + } + + @Test + public void testAlternateLocation() throws IOException { + Schema schema = new Schema(required(1, "id", Types.LongType.get())); + + File location = temp.resolve("table_location").toFile(); + File altLocation = temp.resolve("alt_location").toFile(); + + Tables tables = new HadoopTables(spark.sessionState().newHadoopConf()); + Table table = tables.create(schema, PartitionSpec.unpartitioned(), location.toString()); + + // override the table's data location + table + .updateProperties() + .set(TableProperties.WRITE_DATA_LOCATION, altLocation.getAbsolutePath()) + .commit(); + + writeRecords(table, RandomGenericData.generate(table.schema(), 100, 87112L)); + + table + .currentSnapshot() + .addedDataFiles(table.io()) + .forEach( + dataFile -> + assertThat(dataFile.location()) + .as( + String.format( + "File should have the parent directory %s, but has: %s.", + altLocation, dataFile.location())) + .startsWith(altLocation + "/")); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/FilePathLastModifiedRecord.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/FilePathLastModifiedRecord.java new file mode 100644 index 000000000000..c62c1de6ba33 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/FilePathLastModifiedRecord.java @@ -0,0 +1,80 @@ +/* + * 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.sql.Timestamp; +import java.util.Objects; + +public class FilePathLastModifiedRecord { + private String filePath; + private Timestamp lastModified; + + public FilePathLastModifiedRecord() {} + + public FilePathLastModifiedRecord(String filePath, Timestamp lastModified) { + this.filePath = filePath; + this.lastModified = lastModified; + } + + public String getFilePath() { + return filePath; + } + + public void setFilePath(String filePath) { + this.filePath = filePath; + } + + public Timestamp getLastModified() { + return lastModified; + } + + public void setLastModified(Timestamp lastModified) { + this.lastModified = lastModified; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FilePathLastModifiedRecord that = (FilePathLastModifiedRecord) o; + return Objects.equals(filePath, that.filePath) + && Objects.equals(lastModified, that.lastModified); + } + + @Override + public int hashCode() { + return Objects.hash(filePath, lastModified); + } + + @Override + public String toString() { + return "FilePathLastModifiedRecord{" + + "filePath='" + + filePath + + '\'' + + ", lastModified='" + + lastModified + + '\'' + + '}'; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/FourColumnRecord.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/FourColumnRecord.java new file mode 100644 index 000000000000..0f9529e4d105 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/FourColumnRecord.java @@ -0,0 +1,106 @@ +/* + * 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.Objects; + +public class FourColumnRecord { + private Integer c1; + private String c2; + private String c3; + private String c4; + + public FourColumnRecord() {} + + public FourColumnRecord(Integer c1, String c2, String c3, String c4) { + this.c1 = c1; + this.c2 = c2; + this.c3 = c3; + this.c4 = c4; + } + + public Integer getC1() { + return c1; + } + + public void setC1(Integer c1) { + this.c1 = c1; + } + + public String getC2() { + return c2; + } + + public void setC2(String c2) { + this.c2 = c2; + } + + public String getC3() { + return c3; + } + + public void setC3(String c3) { + this.c3 = c3; + } + + public String getC4() { + return c4; + } + + public void setC4(String c4) { + this.c4 = c4; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FourColumnRecord that = (FourColumnRecord) o; + return Objects.equals(c1, that.c1) + && Objects.equals(c2, that.c2) + && Objects.equals(c3, that.c3) + && Objects.equals(c3, that.c4); + } + + @Override + public int hashCode() { + return Objects.hash(c1, c2, c3, c4); + } + + @Override + public String toString() { + return "ThreeColumnRecord{" + + "c1=" + + c1 + + ", c2='" + + c2 + + '\'' + + ", c3='" + + c3 + + '\'' + + ", c4='" + + c4 + + '\'' + + '}'; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java new file mode 100644 index 000000000000..875b1009c37f --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java @@ -0,0 +1,119 @@ +/* + * 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.time.Instant; +import java.util.concurrent.atomic.AtomicInteger; + +public class LogMessage { + private static final AtomicInteger ID_COUNTER = new AtomicInteger(0); + + static LogMessage debug(String date, String message) { + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "DEBUG", message); + } + + static LogMessage debug(String date, String message, Instant timestamp) { + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "DEBUG", message, timestamp); + } + + static LogMessage info(String date, String message) { + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "INFO", message); + } + + static LogMessage info(String date, String message, Instant timestamp) { + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "INFO", message, timestamp); + } + + static LogMessage error(String date, String message) { + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "ERROR", message); + } + + static LogMessage error(String date, String message, Instant timestamp) { + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "ERROR", message, timestamp); + } + + static LogMessage warn(String date, String message) { + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "WARN", message); + } + + static LogMessage warn(String date, String message, Instant timestamp) { + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "WARN", message, timestamp); + } + + private int id; + private String date; + private String level; + private String message; + private Instant timestamp; + + private LogMessage(int id, String date, String level, String message) { + this.id = id; + this.date = date; + this.level = level; + this.message = message; + } + + private LogMessage(int id, String date, String level, String message, Instant timestamp) { + this.id = id; + this.date = date; + this.level = level; + this.message = message; + this.timestamp = timestamp; + } + + public int getId() { + return id; + } + + public void setId(int id) { + this.id = id; + } + + public String getDate() { + return date; + } + + public void setDate(String date) { + this.date = date; + } + + public String getLevel() { + return level; + } + + public void setLevel(String level) { + this.level = level; + } + + public String getMessage() { + return message; + } + + public void setMessage(String message) { + this.message = message; + } + + public Instant getTimestamp() { + return timestamp; + } + + public void setTimestamp(Instant timestamp) { + this.timestamp = timestamp; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java new file mode 100644 index 000000000000..b6f172248ea9 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java @@ -0,0 +1,75 @@ +/* + * 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.Map; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableProvider; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.sources.DataSourceRegister; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +public class ManualSource implements TableProvider, DataSourceRegister { + public static final String SHORT_NAME = "manual_source"; + public static final String TABLE_NAME = "TABLE_NAME"; + private static final Map TABLE_MAP = Maps.newHashMap(); + + public static void setTable(String name, Table table) { + Preconditions.checkArgument( + !TABLE_MAP.containsKey(name), "Cannot set " + name + ". It is already set"); + TABLE_MAP.put(name, table); + } + + public static void clearTables() { + TABLE_MAP.clear(); + } + + @Override + public String shortName() { + return SHORT_NAME; + } + + @Override + public StructType inferSchema(CaseInsensitiveStringMap options) { + return getTable(null, null, options).schema(); + } + + @Override + public Transform[] inferPartitioning(CaseInsensitiveStringMap options) { + return getTable(null, null, options).partitioning(); + } + + @Override + public org.apache.spark.sql.connector.catalog.Table getTable( + StructType schema, Transform[] partitioning, Map properties) { + Preconditions.checkArgument( + properties.containsKey(TABLE_NAME), "Missing property " + TABLE_NAME); + String tableName = properties.get(TABLE_NAME); + Preconditions.checkArgument(TABLE_MAP.containsKey(tableName), "Table missing " + tableName); + return TABLE_MAP.get(tableName); + } + + @Override + public boolean supportsExternalMetadata() { + return false; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/NestedRecord.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/NestedRecord.java new file mode 100644 index 000000000000..ca36bfd4938b --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/NestedRecord.java @@ -0,0 +1,77 @@ +/* + * 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.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +public class NestedRecord { + private long innerId; + private String innerName; + + public NestedRecord() {} + + public NestedRecord(long innerId, String innerName) { + this.innerId = innerId; + this.innerName = innerName; + } + + public long getInnerId() { + return innerId; + } + + public String getInnerName() { + return innerName; + } + + public void setInnerId(long iId) { + innerId = iId; + } + + public void setInnerName(String name) { + innerName = name; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + NestedRecord that = (NestedRecord) o; + return innerId == that.innerId && Objects.equal(innerName, that.innerName); + } + + @Override + public int hashCode() { + return Objects.hashCode(innerId, innerName); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("innerId", innerId) + .add("innerName", innerName) + .toString(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java new file mode 100644 index 000000000000..33c842d94be1 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ScanTestBase.java @@ -0,0 +1,138 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.data.AvroDataTestBase; +import org.apache.iceberg.spark.data.GenericsHelpers; +import org.apache.iceberg.types.TypeUtil; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; + +/** An AvroDataScan test that validates data by reading through Spark */ +public abstract class ScanTestBase extends AvroDataTestBase { + private static final Configuration CONF = new Configuration(); + + protected static SparkSession spark = null; + protected static JavaSparkContext sc = null; + + @BeforeAll + public static void startSpark() { + ScanTestBase.spark = + SparkSession.builder() + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .master("local[2]") + .getOrCreate(); + ScanTestBase.sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = ScanTestBase.spark; + ScanTestBase.spark = null; + ScanTestBase.sc = null; + currentSpark.stop(); + } + + @TempDir private Path temp; + + protected void configureTable(Table table) {} + + protected abstract void writeRecords(Table table, List records) throws IOException; + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + writeAndValidate(schema, schema); + } + + @Override + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + File parent = temp.resolve("scan_test").toFile(); + File location = new File(parent, "test"); + + HadoopTables tables = new HadoopTables(CONF); + // If V3 spec features are used, set the format version to 3 + Map tableProperties = ImmutableMap.of(TableProperties.FORMAT_VERSION, "3"); + Table table = + tables.create( + writeSchema, PartitionSpec.unpartitioned(), tableProperties, location.toString()); + configureTable(table); + + // Important: use the table's schema for the rest of the test + // When tables are created, the column ids are reassigned. + List expected = RandomGenericData.generate(table.schema(), 100, 1L); + + writeRecords(table, expected); + + // update the table schema to the expected schema + if (!expectedSchema.sameSchema(table.schema())) { + Schema expectedSchemaWithTableIds = + TypeUtil.reassignOrRefreshIds(expectedSchema, table.schema()); + int highestFieldId = + Math.max(table.schema().highestFieldId(), expectedSchema.highestFieldId()); + + // don't use the table API because tests cover incompatible update cases + TableOperations ops = ((BaseTable) table).operations(); + TableMetadata builder = + TableMetadata.buildFrom(ops.current()) + .upgradeFormatVersion(3) + .setCurrentSchema(expectedSchemaWithTableIds, highestFieldId) + .build(); + ops.commit(ops.current(), builder); + } + + Dataset df = spark.read().format("iceberg").load(table.location()); + + List rows = df.collectAsList(); + assertThat(rows).as("Should contain 100 rows").hasSize(100); + + for (int i = 0; i < expected.size(); i += 1) { + GenericsHelpers.assertEqualsSafe(table.schema().asStruct(), expected.get(i), rows.get(i)); + } + } + + @Override + protected boolean supportsDefaultValues() { + return true; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/SimpleRecord.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/SimpleRecord.java new file mode 100644 index 000000000000..550e20b9338e --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/SimpleRecord.java @@ -0,0 +1,78 @@ +/* + * 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.relocated.com.google.common.base.Objects; + +public class SimpleRecord { + private Integer id; + private String data; + + public SimpleRecord() {} + + public SimpleRecord(Integer id, String data) { + this.id = id; + this.data = data; + } + + public Integer getId() { + return id; + } + + public void setId(Integer id) { + this.id = id; + } + + public String getData() { + return data; + } + + public void setData(String data) { + this.data = data; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + SimpleRecord record = (SimpleRecord) o; + return Objects.equal(id, record.id) && Objects.equal(data, record.data); + } + + @Override + public int hashCode() { + return Objects.hashCode(id, data); + } + + @Override + public String toString() { + StringBuilder buffer = new StringBuilder(); + buffer.append("{\"id\"="); + buffer.append(id); + buffer.append(",\"data\"=\""); + buffer.append(data); + buffer.append("\"}"); + return buffer.toString(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/SparkSQLExecutionHelper.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/SparkSQLExecutionHelper.java new file mode 100644 index 000000000000..cdc380b1b6be --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/SparkSQLExecutionHelper.java @@ -0,0 +1,70 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.execution.ui.SQLAppStatusStore; +import org.apache.spark.sql.execution.ui.SQLExecutionUIData; +import org.apache.spark.sql.execution.ui.SQLPlanMetric; +import org.awaitility.Awaitility; +import scala.Option; + +public class SparkSQLExecutionHelper { + + private SparkSQLExecutionHelper() {} + + /** + * Finds the value of a specified metric for the last SQL query that was executed. Metric values + * are stored in the `SQLAppStatusStore` as strings. + * + * @param spark SparkSession used to run the SQL query + * @param metricName name of the metric + * @return value of the metric + */ + public static String lastExecutedMetricValue(SparkSession spark, String metricName) { + SQLAppStatusStore statusStore = spark.sharedState().statusStore(); + SQLExecutionUIData lastExecution = statusStore.executionsList().last(); + Option sqlPlanMetric = + lastExecution.metrics().find(metric -> metric.name().equals(metricName)); + assertThat(sqlPlanMetric.isDefined()) + .as(String.format("Metric '%s' not found in last execution", metricName)) + .isTrue(); + long metricId = sqlPlanMetric.get().accumulatorId(); + + // Refresh metricValues, they will remain null until the execution is complete and metrics are + // aggregated + Awaitility.await() + .atMost(Duration.ofSeconds(3)) + .pollInterval(Duration.ofMillis(100)) + .untilAsserted( + () -> assertThat(statusStore.execution(lastExecution.executionId()).get()).isNotNull()); + + SQLExecutionUIData exec = statusStore.execution(lastExecution.executionId()).get(); + + assertThat(exec.metricValues()).as("Metric values were not finalized").isNotNull(); + String metricValue = exec.metricValues().get(metricId).getOrElse(null); + assertThat(metricValue) + .as(String.format("Metric '%s' was not finalized", metricName)) + .isNotNull(); + return metricValue; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroDataFrameWrite.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroDataFrameWrite.java new file mode 100644 index 000000000000..7e6aee981522 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroDataFrameWrite.java @@ -0,0 +1,38 @@ +/* + * 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.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; + +public class TestAvroDataFrameWrite extends DataFrameWriteTestBase { + @Override + protected boolean supportsVariant() { + return true; + } + + @Override + protected void configureTable(Table table) { + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.AVRO.toString()) + .commit(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java new file mode 100644 index 000000000000..79f3cf71b594 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestAvroScan.java @@ -0,0 +1,68 @@ +/* + * 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.Files.localOutput; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.UUID; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataWriter; +import org.apache.iceberg.io.FileAppender; + +public class TestAvroScan extends ScanTestBase { + @Override + protected boolean supportsVariant() { + return true; + } + + @Override + protected void writeRecords(Table table, List records) throws IOException { + File dataFolder = new File(table.location(), "data"); + dataFolder.mkdirs(); + + File avroFile = + new File(dataFolder, FileFormat.AVRO.addExtension(UUID.randomUUID().toString())); + + try (FileAppender writer = + Avro.write(localOutput(avroFile)) + .schema(table.schema()) + .createWriterFunc(DataWriter::create) + .build()) { + writer.addAll(records); + } + + DataFile file = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withFileSizeInBytes(avroFile.length()) + .withRecordCount(records.size()) + .withPath(avroFile.toString()) + .build(); + + table.newAppend().appendFile(file).commit(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java new file mode 100644 index 000000000000..1e53710a0f7f --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestBaseReader.java @@ -0,0 +1,289 @@ +/* + * 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.FileFormat.PARQUET; +import static org.apache.iceberg.Files.localOutput; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; +import org.apache.avro.generic.GenericData; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestBaseReader { + + @TempDir private Path temp; + + private Table table; + + // Simulates the closeable iterator of data to be read + private static class CloseableIntegerRange implements CloseableIterator { + boolean closed; + Iterator iter; + + CloseableIntegerRange(long range) { + this.closed = false; + this.iter = IntStream.range(0, (int) range).iterator(); + } + + @Override + public void close() { + this.closed = true; + } + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public Integer next() { + return iter.next(); + } + } + + // Main reader class to test base class iteration logic. + // Keeps track of iterator closure. + private static class ClosureTrackingReader extends BaseReader { + private final Map tracker = Maps.newHashMap(); + + ClosureTrackingReader(Table table, List tasks) { + super(table, new BaseCombinedScanTask(tasks), null, null, false, true); + } + + @Override + protected Stream> referencedFiles(FileScanTask task) { + return Stream.of(); + } + + @Override + protected CloseableIterator open(FileScanTask task) { + CloseableIntegerRange intRange = new CloseableIntegerRange(task.file().recordCount()); + tracker.put(getKey(task), intRange); + return intRange; + } + + public Boolean isIteratorClosed(FileScanTask task) { + return tracker.get(getKey(task)).closed; + } + + public Boolean hasIterator(FileScanTask task) { + return tracker.containsKey(getKey(task)); + } + + private String getKey(FileScanTask task) { + return task.file().location(); + } + } + + @Test + public void testClosureOnDataExhaustion() throws IOException { + Integer totalTasks = 10; + Integer recordPerTask = 10; + List tasks = createFileScanTasks(totalTasks, recordPerTask); + + ClosureTrackingReader reader = new ClosureTrackingReader(table, tasks); + + int countRecords = 0; + while (reader.next()) { + countRecords += 1; + assertThat(reader.get()).as("Reader should return non-null value").isNotNull(); + } + + assertThat(totalTasks * recordPerTask) + .as("Reader returned incorrect number of records") + .isEqualTo(countRecords); + tasks.forEach( + t -> + assertThat(reader.isIteratorClosed(t)) + .as("All iterators should be closed after read exhausion") + .isTrue()); + } + + @Test + public void testClosureDuringIteration() throws IOException { + Integer totalTasks = 2; + Integer recordPerTask = 1; + List tasks = createFileScanTasks(totalTasks, recordPerTask); + assertThat(tasks).hasSize(2); + FileScanTask firstTask = tasks.get(0); + FileScanTask secondTask = tasks.get(1); + + ClosureTrackingReader reader = new ClosureTrackingReader(table, tasks); + + // Total of 2 elements + assertThat(reader.next()).isTrue(); + assertThat(reader.isIteratorClosed(firstTask)) + .as("First iter should not be closed on its last element") + .isFalse(); + + assertThat(reader.next()).isTrue(); + assertThat(reader.isIteratorClosed(firstTask)) + .as("First iter should be closed after moving to second iter") + .isTrue(); + assertThat(reader.isIteratorClosed(secondTask)) + .as("Second iter should not be closed on its last element") + .isFalse(); + + assertThat(reader.next()).isFalse(); + assertThat(reader.isIteratorClosed(firstTask)).isTrue(); + assertThat(reader.isIteratorClosed(secondTask)).isTrue(); + } + + @Test + public void testClosureWithoutAnyRead() throws IOException { + Integer totalTasks = 10; + Integer recordPerTask = 10; + List tasks = createFileScanTasks(totalTasks, recordPerTask); + + ClosureTrackingReader reader = new ClosureTrackingReader(table, tasks); + + reader.close(); + + tasks.forEach( + t -> + assertThat(reader.hasIterator(t)) + .as("Iterator should not be created eagerly for tasks") + .isFalse()); + } + + @Test + public void testExplicitClosure() throws IOException { + Integer totalTasks = 10; + Integer recordPerTask = 10; + List tasks = createFileScanTasks(totalTasks, recordPerTask); + + ClosureTrackingReader reader = new ClosureTrackingReader(table, tasks); + + Integer halfDataSize = (totalTasks * recordPerTask) / 2; + for (int i = 0; i < halfDataSize; i++) { + assertThat(reader.next()).as("Reader should have some element").isTrue(); + assertThat(reader.get()).as("Reader should return non-null value").isNotNull(); + } + + reader.close(); + + // Some tasks might have not been opened yet, so we don't have corresponding tracker for it. + // But all that have been created must be closed. + tasks.forEach( + t -> { + if (reader.hasIterator(t)) { + assertThat(reader.isIteratorClosed(t)) + .as("Iterator should be closed after read exhausion") + .isTrue(); + } + }); + } + + @Test + public void testIdempotentExplicitClosure() throws IOException { + Integer totalTasks = 10; + Integer recordPerTask = 10; + List tasks = createFileScanTasks(totalTasks, recordPerTask); + + ClosureTrackingReader reader = new ClosureTrackingReader(table, tasks); + + // Total 100 elements, only 5 iterators have been created + for (int i = 0; i < 45; i++) { + assertThat(reader.next()).as("Reader should have some element").isTrue(); + assertThat(reader.get()).as("Reader should return non-null value").isNotNull(); + } + + for (int closeAttempt = 0; closeAttempt < 5; closeAttempt++) { + reader.close(); + for (int i = 0; i < 5; i++) { + assertThat(reader.isIteratorClosed(tasks.get(i))) + .as("Iterator should be closed after read exhausion") + .isTrue(); + } + for (int i = 5; i < 10; i++) { + assertThat(reader.hasIterator(tasks.get(i))) + .as("Iterator should not be created eagerly for tasks") + .isFalse(); + } + } + } + + private List createFileScanTasks(Integer totalTasks, Integer recordPerTask) + throws IOException { + String desc = "make_scan_tasks"; + File parent = temp.resolve(desc).toFile(); + File location = new File(parent, "test"); + File dataFolder = new File(location, "data"); + assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue(); + + Schema schema = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + try { + this.table = TestTables.create(location, desc, schema, PartitionSpec.unpartitioned()); + // Important: use the table's schema for the rest of the test + // When tables are created, the column ids are reassigned. + Schema tableSchema = table.schema(); + List expected = RandomData.generateList(tableSchema, recordPerTask, 1L); + + AppendFiles appendFiles = table.newAppend(); + for (int i = 0; i < totalTasks; i++) { + File parquetFile = new File(dataFolder, PARQUET.addExtension(UUID.randomUUID().toString())); + try (FileAppender writer = + Parquet.write(localOutput(parquetFile)).schema(tableSchema).build()) { + writer.addAll(expected); + } + DataFile file = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withFileSizeInBytes(parquetFile.length()) + .withPath(parquetFile.toString()) + .withRecordCount(recordPerTask) + .build(); + appendFiles.appendFile(file); + } + appendFiles.commit(); + + return StreamSupport.stream(table.newScan().planFiles().spliterator(), false) + .collect(Collectors.toList()); + } finally { + TestTables.clearTables(); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java new file mode 100644 index 000000000000..b88f0233e203 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestChangelogReader.java @@ -0,0 +1,263 @@ +/* + * 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.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.ChangelogOperation; +import org.apache.iceberg.ChangelogScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.IncrementalChangelogScan; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestChangelogReader extends TestBase { + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).bucket("data", 16).build(); + private final List records1 = Lists.newArrayList(); + private final List records2 = Lists.newArrayList(); + + private Table table; + private DataFile dataFile1; + private DataFile dataFile2; + + @TempDir private Path temp; + + @BeforeEach + public void before() throws IOException { + table = catalog.createTable(TableIdentifier.of("default", "test"), SCHEMA, SPEC); + // create some data + GenericRecord record = GenericRecord.create(table.schema()); + records1.add(record.copy("id", 29, "data", "a")); + records1.add(record.copy("id", 43, "data", "b")); + records1.add(record.copy("id", 61, "data", "c")); + records1.add(record.copy("id", 89, "data", "d")); + + records2.add(record.copy("id", 100, "data", "e")); + records2.add(record.copy("id", 121, "data", "f")); + records2.add(record.copy("id", 122, "data", "g")); + + // write data to files + dataFile1 = writeDataFile(records1); + dataFile2 = writeDataFile(records2); + } + + @AfterEach + public void after() { + catalog.dropTable(TableIdentifier.of("default", "test")); + } + + @Test + public void testInsert() throws IOException { + table.newAppend().appendFile(dataFile1).commit(); + long snapshotId1 = table.currentSnapshot().snapshotId(); + + table.newAppend().appendFile(dataFile2).commit(); + long snapshotId2 = table.currentSnapshot().snapshotId(); + + CloseableIterable> taskGroups = newScan().planTasks(); + + List rows = Lists.newArrayList(); + + for (ScanTaskGroup taskGroup : taskGroups) { + ChangelogRowReader reader = + new ChangelogRowReader(table, taskGroup, table.schema(), table.schema(), false, true); + while (reader.next()) { + rows.add(reader.get().copy()); + } + reader.close(); + } + + rows.sort((r1, r2) -> r1.getInt(0) - r2.getInt(0)); + + List expectedRows = Lists.newArrayList(); + addExpectedRows(expectedRows, ChangelogOperation.INSERT, snapshotId1, 0, records1); + addExpectedRows(expectedRows, ChangelogOperation.INSERT, snapshotId2, 1, records2); + + assertEquals("Should have expected rows", expectedRows, internalRowsToJava(rows)); + } + + @Test + public void testDelete() throws IOException { + table.newAppend().appendFile(dataFile1).commit(); + long snapshotId1 = table.currentSnapshot().snapshotId(); + + table.newDelete().deleteFile(dataFile1).commit(); + long snapshotId2 = table.currentSnapshot().snapshotId(); + + CloseableIterable> taskGroups = + newScan().fromSnapshotExclusive(snapshotId1).planTasks(); + + List rows = Lists.newArrayList(); + + for (ScanTaskGroup taskGroup : taskGroups) { + ChangelogRowReader reader = + new ChangelogRowReader(table, taskGroup, table.schema(), table.schema(), false, true); + while (reader.next()) { + rows.add(reader.get().copy()); + } + reader.close(); + } + + rows.sort((r1, r2) -> r1.getInt(0) - r2.getInt(0)); + + List expectedRows = Lists.newArrayList(); + addExpectedRows(expectedRows, ChangelogOperation.DELETE, snapshotId2, 0, records1); + + assertEquals("Should have expected rows", expectedRows, internalRowsToJava(rows)); + } + + @Test + public void testDataFileRewrite() throws IOException { + table.newAppend().appendFile(dataFile1).commit(); + table.newAppend().appendFile(dataFile2).commit(); + long snapshotId2 = table.currentSnapshot().snapshotId(); + + table + .newRewrite() + .rewriteFiles(ImmutableSet.of(dataFile1), ImmutableSet.of(dataFile2)) + .commit(); + + // the rewrite operation should generate no Changelog rows + CloseableIterable> taskGroups = + newScan().fromSnapshotExclusive(snapshotId2).planTasks(); + + List rows = Lists.newArrayList(); + + for (ScanTaskGroup taskGroup : taskGroups) { + ChangelogRowReader reader = + new ChangelogRowReader(table, taskGroup, table.schema(), table.schema(), false, true); + while (reader.next()) { + rows.add(reader.get().copy()); + } + reader.close(); + } + + assertThat(rows).as("Should have no rows").isEmpty(); + } + + @Test + public void testMixDeleteAndInsert() throws IOException { + table.newAppend().appendFile(dataFile1).commit(); + long snapshotId1 = table.currentSnapshot().snapshotId(); + + table.newDelete().deleteFile(dataFile1).commit(); + long snapshotId2 = table.currentSnapshot().snapshotId(); + + table.newAppend().appendFile(dataFile2).commit(); + long snapshotId3 = table.currentSnapshot().snapshotId(); + + CloseableIterable> taskGroups = newScan().planTasks(); + + List rows = Lists.newArrayList(); + + for (ScanTaskGroup taskGroup : taskGroups) { + ChangelogRowReader reader = + new ChangelogRowReader(table, taskGroup, table.schema(), table.schema(), false, true); + while (reader.next()) { + rows.add(reader.get().copy()); + } + reader.close(); + } + + // order by the change ordinal + rows.sort( + (r1, r2) -> { + if (r1.getInt(3) != r2.getInt(3)) { + return r1.getInt(3) - r2.getInt(3); + } else { + return r1.getInt(0) - r2.getInt(0); + } + }); + + List expectedRows = Lists.newArrayList(); + addExpectedRows(expectedRows, ChangelogOperation.INSERT, snapshotId1, 0, records1); + addExpectedRows(expectedRows, ChangelogOperation.DELETE, snapshotId2, 1, records1); + addExpectedRows(expectedRows, ChangelogOperation.INSERT, snapshotId3, 2, records2); + + assertEquals("Should have expected rows", expectedRows, internalRowsToJava(rows)); + } + + private IncrementalChangelogScan newScan() { + return table.newIncrementalChangelogScan(); + } + + private List addExpectedRows( + List expectedRows, + ChangelogOperation operation, + long snapshotId, + int changeOrdinal, + List records) { + records.forEach( + r -> + expectedRows.add(row(r.get(0), r.get(1), operation.name(), changeOrdinal, snapshotId))); + return expectedRows; + } + + protected List internalRowsToJava(List rows) { + return rows.stream().map(this::toJava).collect(Collectors.toList()); + } + + private Object[] toJava(InternalRow row) { + Object[] values = new Object[row.numFields()]; + values[0] = row.getInt(0); + values[1] = row.getString(1); + values[2] = row.getString(2); + values[3] = row.getInt(3); + values[4] = row.getLong(4); + return values; + } + + private DataFile writeDataFile(List records) throws IOException { + // records all use IDs that are in bucket id_bucket=0 + return FileHelpers.writeDataFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + records); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java new file mode 100644 index 000000000000..19ec6d13dd5d --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestCompressionSettings.java @@ -0,0 +1,276 @@ +/* + * 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.FileFormat.AVRO; +import static org.apache.iceberg.FileFormat.ORC; +import static org.apache.iceberg.FileFormat.PARQUET; +import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DELETE_AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DELETE_MODE; +import static org.apache.iceberg.TableProperties.DELETE_ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.DELETE_PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.FORMAT_VERSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.spark.SparkSQLProperties.COMPRESSION_CODEC; +import static org.apache.iceberg.spark.SparkSQLProperties.COMPRESSION_LEVEL; +import static org.apache.iceberg.spark.SparkSQLProperties.COMPRESSION_STRATEGY; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +import org.apache.avro.file.DataFileConstants; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.AvroFSInput; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestReader; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.SizeBasedFileRewritePlanner; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.io.InputFile; +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.CatalogTestBase; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.orc.OrcFile; +import org.apache.orc.Reader; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.util.HadoopInputFile; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +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; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestCompressionSettings extends CatalogTestBase { + + private static final Configuration CONF = new Configuration(); + private static final String TABLE_NAME = "testWriteData"; + + private static SparkSession spark = null; + + @Parameter(index = 3) + private FileFormat format; + + @Parameter(index = 4) + private Map properties; + + @TempDir private java.nio.file.Path temp; + + @Parameters( + name = + "catalogName = {0}, implementation = {1}, config = {2}, format = {3}, properties = {4}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties(), + PARQUET, + ImmutableMap.of(COMPRESSION_CODEC, "zstd", COMPRESSION_LEVEL, "1") + }, + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties(), + PARQUET, + ImmutableMap.of(COMPRESSION_CODEC, "gzip") + }, + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties(), + ORC, + ImmutableMap.of(COMPRESSION_CODEC, "zstd", COMPRESSION_STRATEGY, "speed") + }, + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties(), + ORC, + ImmutableMap.of(COMPRESSION_CODEC, "zstd", COMPRESSION_STRATEGY, "compression") + }, + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties(), + AVRO, + ImmutableMap.of(COMPRESSION_CODEC, "snappy", COMPRESSION_LEVEL, "3") + } + }; + } + + @BeforeAll + public static void startSpark() { + TestCompressionSettings.spark = SparkSession.builder().master("local[2]").getOrCreate(); + } + + @BeforeEach + public void resetSpecificConfigurations() { + spark.conf().unset(COMPRESSION_CODEC); + spark.conf().unset(COMPRESSION_LEVEL); + spark.conf().unset(COMPRESSION_STRATEGY); + } + + @AfterEach + public void afterEach() { + spark.sql(String.format("DROP TABLE IF EXISTS %s", TABLE_NAME)); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestCompressionSettings.spark; + TestCompressionSettings.spark = null; + currentSpark.stop(); + } + + @TestTemplate + public void testWriteDataWithDifferentSetting() throws Exception { + sql("CREATE TABLE %s (id int, data string) USING iceberg", TABLE_NAME); + Map tableProperties = Maps.newHashMap(); + tableProperties.put(PARQUET_COMPRESSION, "gzip"); + tableProperties.put(AVRO_COMPRESSION, "gzip"); + tableProperties.put(ORC_COMPRESSION, "zlib"); + tableProperties.put(DELETE_PARQUET_COMPRESSION, "gzip"); + tableProperties.put(DELETE_AVRO_COMPRESSION, "gzip"); + tableProperties.put(DELETE_ORC_COMPRESSION, "zlib"); + tableProperties.put(DELETE_MODE, MERGE_ON_READ.modeName()); + tableProperties.put(FORMAT_VERSION, "2"); + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", TABLE_NAME, DEFAULT_FILE_FORMAT, format); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", + TABLE_NAME, DELETE_DEFAULT_FILE_FORMAT, format); + for (Map.Entry entry : tableProperties.entrySet()) { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", + TABLE_NAME, entry.getKey(), entry.getValue()); + } + + List expectedOrigin = Lists.newArrayList(); + for (int i = 0; i < 1000; i++) { + expectedOrigin.add(new SimpleRecord(i, "hello world" + i)); + } + + Dataset df = spark.createDataFrame(expectedOrigin, SimpleRecord.class); + + for (Map.Entry entry : properties.entrySet()) { + spark.conf().set(entry.getKey(), entry.getValue()); + } + + assertSparkConf(); + + df.select("id", "data") + .writeTo(TABLE_NAME) + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .append(); + Table table = catalog.loadTable(TableIdentifier.of("default", TABLE_NAME)); + List manifestFiles = table.currentSnapshot().dataManifests(table.io()); + try (ManifestReader reader = ManifestFiles.read(manifestFiles.get(0), table.io())) { + DataFile file = reader.iterator().next(); + InputFile inputFile = table.io().newInputFile(file.location()); + assertThat(getCompressionType(inputFile)) + .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); + } + + sql("DELETE from %s where id < 100", TABLE_NAME); + + table.refresh(); + List deleteManifestFiles = table.currentSnapshot().deleteManifests(table.io()); + Map specMap = Maps.newHashMap(); + specMap.put(0, PartitionSpec.unpartitioned()); + try (ManifestReader reader = + ManifestFiles.readDeleteManifest(deleteManifestFiles.get(0), table.io(), specMap)) { + DeleteFile file = reader.iterator().next(); + InputFile inputFile = table.io().newInputFile(file.location()); + assertThat(getCompressionType(inputFile)) + .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); + } + + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .execute(); + table.refresh(); + deleteManifestFiles = table.currentSnapshot().deleteManifests(table.io()); + try (ManifestReader reader = + ManifestFiles.readDeleteManifest(deleteManifestFiles.get(0), table.io(), specMap)) { + DeleteFile file = reader.iterator().next(); + InputFile inputFile = table.io().newInputFile(file.location()); + assertThat(getCompressionType(inputFile)) + .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); + } + } + + private String getCompressionType(InputFile inputFile) throws Exception { + switch (format) { + case ORC: + OrcFile.ReaderOptions readerOptions = OrcFile.readerOptions(CONF).useUTCTimestamp(true); + Reader orcReader = OrcFile.createReader(new Path(inputFile.location()), readerOptions); + return orcReader.getCompressionKind().name(); + case PARQUET: + ParquetMetadata footer = + ParquetFileReader.open(HadoopInputFile.fromPath(new Path(inputFile.location()), CONF)) + .getFooter(); + return footer.getBlocks().get(0).getColumns().get(0).getCodec().name(); + default: + FileContext fc = FileContext.getFileContext(CONF); + GenericDatumReader reader = new GenericDatumReader<>(); + DataFileReader fileReader = + (DataFileReader) + DataFileReader.openReader( + new AvroFSInput(fc, new Path(inputFile.location())), reader); + return fileReader.getMetaString(DataFileConstants.CODEC); + } + } + + private void assertSparkConf() { + String[] propertiesToCheck = {COMPRESSION_CODEC, COMPRESSION_LEVEL, COMPRESSION_STRATEGY}; + for (String prop : propertiesToCheck) { + String expected = properties.getOrDefault(prop, null); + String actual = spark.conf().get(prop, null); + assertThat(actual).isEqualToIgnoringCase(expected); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java new file mode 100644 index 000000000000..186d0b2b5204 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2.java @@ -0,0 +1,390 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatCode; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.math.BigDecimal; +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.iceberg.types.Type; +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; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.internal.SQLConf; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestDataFrameWriterV2 extends TestBaseWithCatalog { + @BeforeEach + public void createTable() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testMergeSchemaFailsWithoutWriterOption() throws Exception { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset twoColDF = + jsonToDF( + "id bigint, data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + twoColDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial 2-column rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("select * from %s order by id", tableName)); + + Dataset threeColDF = + jsonToDF( + "id bigint, data string, new_col float", + "{ \"id\": 3, \"data\": \"c\", \"new_col\": 12.06 }", + "{ \"id\": 4, \"data\": \"d\", \"new_col\": 14.41 }"); + + // this has a different error message than the case without accept-any-schema because it uses + // Iceberg checks + assertThatThrownBy(() -> threeColDF.writeTo(tableName).append()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Field new_col not found in source schema"); + } + + @TestTemplate + public void testMergeSchemaWithoutAcceptAnySchema() throws Exception { + Dataset twoColDF = + jsonToDF( + "id bigint, data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + twoColDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial 2-column rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("select * from %s order by id", tableName)); + + Dataset threeColDF = + jsonToDF( + "id bigint, data string, new_col float", + "{ \"id\": 3, \"data\": \"c\", \"new_col\": 12.06 }", + "{ \"id\": 4, \"data\": \"d\", \"new_col\": 14.41 }"); + + assertThatThrownBy(() -> threeColDF.writeTo(tableName).option("merge-schema", "true").append()) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Cannot write to `testhadoop`.`default`.`table`, the reason is too many data columns"); + } + + @TestTemplate + public void testMergeSchemaSparkProperty() throws Exception { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset twoColDF = + jsonToDF( + "id bigint, data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + twoColDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial 2-column rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("select * from %s order by id", tableName)); + + Dataset threeColDF = + jsonToDF( + "id bigint, data string, new_col float", + "{ \"id\": 3, \"data\": \"c\", \"new_col\": 12.06 }", + "{ \"id\": 4, \"data\": \"d\", \"new_col\": 14.41 }"); + + threeColDF.writeTo(tableName).option("mergeSchema", "true").append(); + + assertEquals( + "Should have 3-column rows", + ImmutableList.of( + row(1L, "a", null), row(2L, "b", null), row(3L, "c", 12.06F), row(4L, "d", 14.41F)), + sql("select * from %s order by id", tableName)); + } + + @TestTemplate + public void testMergeSchemaIcebergProperty() throws Exception { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset twoColDF = + jsonToDF( + "id bigint, data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + twoColDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial 2-column rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("select * from %s order by id", tableName)); + + Dataset threeColDF = + jsonToDF( + "id bigint, data string, new_col float", + "{ \"id\": 3, \"data\": \"c\", \"new_col\": 12.06 }", + "{ \"id\": 4, \"data\": \"d\", \"new_col\": 14.41 }"); + + threeColDF.writeTo(tableName).option("merge-schema", "true").append(); + + assertEquals( + "Should have 3-column rows", + ImmutableList.of( + row(1L, "a", null), row(2L, "b", null), row(3L, "c", 12.06F), row(4L, "d", 14.41F)), + sql("select * from %s order by id", tableName)); + } + + @TestTemplate + public void testWriteWithCaseSensitiveOption() throws NoSuchTableException, ParseException { + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + + SparkSession sparkSession = ((org.apache.spark.sql.classic.SparkSession) spark).cloneSession(); + sparkSession + .sql( + String.format( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA)) + .collect(); + + String schema = "ID bigint, DaTa string"; + ImmutableList records = + ImmutableList.of("{ \"id\": 1, \"data\": \"a\" }", "{ \"id\": 2, \"data\": \"b\" }"); + + // disable spark.sql.caseSensitive + sparkSession.sql(String.format("SET %s=false", SQLConf.CASE_SENSITIVE().key())); + Dataset jsonDF = + sparkSession.createDataset(ImmutableList.copyOf(records), Encoders.STRING()); + Dataset ds = sparkSession.read().schema(schema).json(jsonDF); + // write should succeed + ds.writeTo(tableName).option("merge-schema", "true").option("check-ordering", "false").append(); + List fields = + Spark3Util.loadIcebergTable(sparkSession, tableName).schema().asStruct().fields(); + // Additional columns should not be created + assertThat(fields).hasSize(2); + + // enable spark.sql.caseSensitive + sparkSession.sql(String.format("SET %s=true", SQLConf.CASE_SENSITIVE().key())); + ds.writeTo(tableName).option("merge-schema", "true").option("check-ordering", "false").append(); + fields = Spark3Util.loadIcebergTable(sparkSession, tableName).schema().asStruct().fields(); + assertThat(fields).hasSize(4); + } + + @TestTemplate + public void testMergeSchemaSparkConfiguration() throws Exception { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + Dataset twoColDF = + jsonToDF( + "id bigint, data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + twoColDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial 2-column rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("select * from %s order by id", tableName)); + spark.conf().set("spark.sql.iceberg.merge-schema", "true"); + Dataset threeColDF = + jsonToDF( + "id bigint, data string, salary float", + "{ \"id\": 3, \"data\": \"c\", \"salary\": 120000.34 }", + "{ \"id\": 4, \"data\": \"d\", \"salary\": 140000.56 }"); + + threeColDF.writeTo(tableName).append(); + assertEquals( + "Should have 3-column rows", + ImmutableList.of( + row(1L, "a", null), + row(2L, "b", null), + row(3L, "c", 120000.34F), + row(4L, "d", 140000.56F)), + sql("select * from %s order by id", tableName)); + } + + @TestTemplate + public void testMergeSchemaIgnoreCastingLongToInt() throws Exception { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset bigintDF = + jsonToDF( + "id bigint, data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + bigintDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial rows with long column", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("select * from %s order by id", tableName)); + + Dataset intDF = + jsonToDF( + "id int, data string", + "{ \"id\": 3, \"data\": \"c\" }", + "{ \"id\": 4, \"data\": \"d\" }"); + + // merge-schema=true on writes allows table schema updates when incoming data has schema changes + assertThatCode(() -> intDF.writeTo(tableName).option("merge-schema", "true").append()) + .doesNotThrowAnyException(); + + assertEquals( + "Should include new rows with unchanged long column type", + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d")), + sql("select * from %s order by id", tableName)); + + // verify the column type did not change + Types.NestedField idField = + Spark3Util.loadIcebergTable(spark, tableName).schema().findField("id"); + assertThat(idField.type().typeId()).isEqualTo(Type.TypeID.LONG); + } + + @TestTemplate + public void testMergeSchemaIgnoreCastingDoubleToFloat() throws Exception { + removeTables(); + sql("CREATE TABLE %s (id double, data string) USING iceberg", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset doubleDF = + jsonToDF( + "id double, data string", + "{ \"id\": 1.0, \"data\": \"a\" }", + "{ \"id\": 2.0, \"data\": \"b\" }"); + + doubleDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial rows with double column", + ImmutableList.of(row(1.0, "a"), row(2.0, "b")), + sql("select * from %s order by id", tableName)); + + Dataset floatDF = + jsonToDF( + "id float, data string", + "{ \"id\": 3.0, \"data\": \"c\" }", + "{ \"id\": 4.0, \"data\": \"d\" }"); + + // merge-schema=true on writes allows table schema updates when incoming data has schema changes + assertThatCode(() -> floatDF.writeTo(tableName).option("merge-schema", "true").append()) + .doesNotThrowAnyException(); + + assertEquals( + "Should include new rows with unchanged double column type", + ImmutableList.of(row(1.0, "a"), row(2.0, "b"), row(3.0, "c"), row(4.0, "d")), + sql("select * from %s order by id", tableName)); + + // verify the column type did not change + Types.NestedField idField = + Spark3Util.loadIcebergTable(spark, tableName).schema().findField("id"); + assertThat(idField.type().typeId()).isEqualTo(Type.TypeID.DOUBLE); + } + + @TestTemplate + public void testMergeSchemaIgnoreCastingDecimalToDecimalWithNarrowerPrecision() throws Exception { + removeTables(); + sql("CREATE TABLE %s (id decimal(6,2), data string) USING iceberg", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + + Dataset decimalPrecision6DF = + jsonToDF( + "id decimal(6,2), data string", + "{ \"id\": 1.0, \"data\": \"a\" }", + "{ \"id\": 2.0, \"data\": \"b\" }"); + + decimalPrecision6DF.writeTo(tableName).append(); + + assertEquals( + "Should have initial rows with decimal column with precision 6", + ImmutableList.of(row(new BigDecimal("1.00"), "a"), row(new BigDecimal("2.00"), "b")), + sql("select * from %s order by id", tableName)); + + Dataset decimalPrecision4DF = + jsonToDF( + "id decimal(4,2), data string", + "{ \"id\": 3.0, \"data\": \"c\" }", + "{ \"id\": 4.0, \"data\": \"d\" }"); + + // merge-schema=true on writes allows table schema updates when incoming data has schema changes + assertThatCode( + () -> decimalPrecision4DF.writeTo(tableName).option("merge-schema", "true").append()) + .doesNotThrowAnyException(); + + assertEquals( + "Should include new rows with unchanged decimal precision", + ImmutableList.of( + row(new BigDecimal("1.00"), "a"), + row(new BigDecimal("2.00"), "b"), + row(new BigDecimal("3.00"), "c"), + row(new BigDecimal("4.00"), "d")), + sql("select * from %s order by id", tableName)); + + // verify the decimal column precision did not change + Type idFieldType = + Spark3Util.loadIcebergTable(spark, tableName).schema().findField("id").type(); + assertThat(idFieldType.typeId()).isEqualTo(Type.TypeID.DECIMAL); + Types.DecimalType decimalType = (Types.DecimalType) idFieldType; + assertThat(decimalType.precision()).isEqualTo(6); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java new file mode 100644 index 000000000000..f51a06853a69 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java @@ -0,0 +1,81 @@ +/* + * 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.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestDataFrameWriterV2Coercion extends TestBaseWithCatalog { + + @Parameters( + name = "catalogName = {0}, implementation = {1}, config = {2}, format = {3}, dataType = {4}") + public static Object[][] parameters() { + return new Object[][] { + parameter(FileFormat.AVRO, "byte"), + parameter(FileFormat.ORC, "byte"), + parameter(FileFormat.PARQUET, "byte"), + parameter(FileFormat.AVRO, "short"), + parameter(FileFormat.ORC, "short"), + parameter(FileFormat.PARQUET, "short") + }; + } + + private static Object[] parameter(FileFormat fileFormat, String dataType) { + return new Object[] { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + fileFormat, + dataType + }; + } + + @Parameter(index = 3) + private FileFormat format; + + @Parameter(index = 4) + private String dataType; + + @TestTemplate + public void testByteAndShortCoercion() { + + Dataset df = + jsonToDF( + "id " + dataType + ", data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + df.writeTo(tableName).option("write-format", format.name()).createOrReplace(); + + assertEquals( + "Should have initial 2-column rows", + ImmutableList.of(row(1, "a"), row(2, "b")), + sql("select * from %s order by id", tableName)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java new file mode 100644 index 000000000000..a7702b169a60 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestDataSourceOptions.java @@ -0,0 +1,520 @@ +/* + * 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.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.math.RoundingMode; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.CloseableIterable; +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.relocated.com.google.common.math.LongMath; +import org.apache.iceberg.spark.CommitMetadata; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.functions; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestDataSourceOptions extends TestBaseWithCatalog { + + private static final Configuration CONF = new Configuration(); + private static final Schema SCHEMA = + new Schema( + optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); + private static SparkSession spark = null; + + @BeforeAll + public static void startSpark() { + TestDataSourceOptions.spark = SparkSession.builder().master("local[2]").getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestDataSourceOptions.spark; + TestDataSourceOptions.spark = null; + currentSpark.stop(); + } + + @TestTemplate + public void testWriteFormatOptionOverridesTableProperties() throws IOException { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + options.put(TableProperties.DEFAULT_FILE_FORMAT, "avro"); + Table table = tables.create(SCHEMA, spec, options, tableLocation); + + List expectedRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset df = spark.createDataFrame(expectedRecords, SimpleRecord.class); + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, "parquet") + .mode(SaveMode.Append) + .save(tableLocation); + + try (CloseableIterable tasks = table.newScan().planFiles()) { + tasks.forEach( + task -> { + FileFormat fileFormat = FileFormat.fromFileName(task.file().location()); + assertThat(fileFormat).isEqualTo(FileFormat.PARQUET); + }); + } + } + + @TestTemplate + public void testNoWriteFormatOption() throws IOException { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + options.put(TableProperties.DEFAULT_FILE_FORMAT, "avro"); + Table table = tables.create(SCHEMA, spec, options, tableLocation); + + List expectedRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset df = spark.createDataFrame(expectedRecords, SimpleRecord.class); + df.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + try (CloseableIterable tasks = table.newScan().planFiles()) { + tasks.forEach( + task -> { + FileFormat fileFormat = FileFormat.fromFileName(task.file().location()); + assertThat(fileFormat).isEqualTo(FileFormat.AVRO); + }); + } + } + + @TestTemplate + public void testHadoopOptions() throws IOException { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + Configuration sparkHadoopConf = spark.sessionState().newHadoopConf(); + String originalDefaultFS = sparkHadoopConf.get("fs.default.name"); + + try { + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + tables.create(SCHEMA, spec, options, tableLocation); + + // set an invalid value for 'fs.default.name' in Spark Hadoop config + // to verify that 'hadoop.' data source options are propagated correctly + sparkHadoopConf.set("fs.default.name", "hdfs://localhost:9000"); + + List expectedRecords = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset originalDf = spark.createDataFrame(expectedRecords, SimpleRecord.class); + originalDf + .select("id", "data") + .write() + .format("iceberg") + .mode("append") + .option("hadoop.fs.default.name", "file:///") + .save(tableLocation); + + Dataset resultDf = + spark + .read() + .format("iceberg") + .option("hadoop.fs.default.name", "file:///") + .load(tableLocation); + List resultRecords = + resultDf.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + + assertThat(resultRecords).as("Records should match").isEqualTo(expectedRecords); + } finally { + sparkHadoopConf.set("fs.default.name", originalDefaultFS); + } + } + + @TestTemplate + public void testSplitOptionsOverridesTableProperties() throws IOException { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + options.put(TableProperties.SPLIT_SIZE, String.valueOf(128L * 1024 * 1024)); // 128Mb + options.put( + TableProperties.DEFAULT_FILE_FORMAT, + String.valueOf(FileFormat.AVRO)); // Arbitrarily splittable + Table icebergTable = tables.create(SCHEMA, spec, options, tableLocation); + + List expectedRecords = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset originalDf = spark.createDataFrame(expectedRecords, SimpleRecord.class); + originalDf + .select("id", "data") + .repartition(1) + .write() + .format("iceberg") + .mode("append") + .save(tableLocation); + + List files = + Lists.newArrayList(icebergTable.currentSnapshot().addedDataFiles(icebergTable.io())); + assertThat(files).as("Should have written 1 file").hasSize(1); + + long fileSize = files.get(0).fileSizeInBytes(); + long splitSize = LongMath.divide(fileSize, 2, RoundingMode.CEILING); + + Dataset resultDf = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SPLIT_SIZE, String.valueOf(splitSize)) + .load(tableLocation); + + assertThat(resultDf.javaRDD().getNumPartitions()) + .as("Spark partitions should match") + .isEqualTo(2); + } + + @TestTemplate + public void testIncrementalScanOptions() throws IOException { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + Table table = tables.create(SCHEMA, spec, options, tableLocation); + + List expectedRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + for (SimpleRecord record : expectedRecords) { + Dataset originalDf = + spark.createDataFrame(Lists.newArrayList(record), SimpleRecord.class); + originalDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + } + List snapshotIds = SnapshotUtil.currentAncestorIds(table); + + // start-snapshot-id and snapshot-id are both configured. + assertThatThrownBy( + () -> + spark + .read() + .format("iceberg") + .option("snapshot-id", 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"); + + // end-snapshot-id and as-of-timestamp are both configured. + assertThatThrownBy( + () -> + spark + .read() + .format("iceberg") + .option( + SparkReadOptions.AS_OF_TIMESTAMP, + Long.toString(table.snapshot(snapshotIds.get(3)).timestampMillis())) + .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"); + + // only end-snapshot-id is configured. + assertThatThrownBy( + () -> + spark + .read() + .format("iceberg") + .option("end-snapshot-id", snapshotIds.get(2).toString()) + .load(tableLocation) + .explain()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot set only end-snapshot-id for incremental scans. Please, set start-snapshot-id too."); + + // test (1st snapshot, current snapshot] incremental scan. + Dataset unboundedIncrementalResult = + spark + .read() + .format("iceberg") + .option("start-snapshot-id", snapshotIds.get(3).toString()) + .load(tableLocation); + List result1 = + unboundedIncrementalResult + .orderBy("id") + .as(Encoders.bean(SimpleRecord.class)) + .collectAsList(); + assertThat(result1).as("Records should match").isEqualTo(expectedRecords.subList(1, 4)); + assertThat(unboundedIncrementalResult.count()) + .as("Unprocessed count should match record count") + .isEqualTo(3); + + Row row1 = unboundedIncrementalResult.agg(functions.min("id"), functions.max("id")).head(); + assertThat(row1.getInt(0)).as("min value should match").isEqualTo(2); + assertThat(row1.getInt(1)).as("max value should match").isEqualTo(4); + + // test (2nd snapshot, 3rd snapshot] incremental scan. + Dataset incrementalResult = + spark + .read() + .format("iceberg") + .option("start-snapshot-id", snapshotIds.get(2).toString()) + .option("end-snapshot-id", snapshotIds.get(1).toString()) + .load(tableLocation); + List result2 = + incrementalResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(result2).as("Records should match").isEqualTo(expectedRecords.subList(2, 3)); + assertThat(incrementalResult.count()) + .as("Unprocessed count should match record count") + .isEqualTo(1); + + Row row2 = incrementalResult.agg(functions.min("id"), functions.max("id")).head(); + assertThat(row2.getInt(0)).as("min value should match").isEqualTo(3); + assertThat(row2.getInt(1)).as("max value should match").isEqualTo(3); + } + + @TestTemplate + public void testMetadataSplitSizeOptionOverrideTableProperties() throws IOException { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + Table table = tables.create(SCHEMA, spec, options, tableLocation); + + List expectedRecords = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset originalDf = spark.createDataFrame(expectedRecords, SimpleRecord.class); + // produce 1st manifest + originalDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + // produce 2nd manifest + originalDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + List manifests = table.currentSnapshot().allManifests(table.io()); + + assertThat(manifests).as("Must be 2 manifests").hasSize(2); + + // set the target metadata split size so each manifest ends up in a separate split + table + .updateProperties() + .set(TableProperties.METADATA_SPLIT_SIZE, String.valueOf(manifests.get(0).length())) + .commit(); + + Dataset entriesDf = spark.read().format("iceberg").load(tableLocation + "#entries"); + assertThat(entriesDf.javaRDD().getNumPartitions()).as("Num partitions must match").isEqualTo(2); + + // override the table property using options + entriesDf = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SPLIT_SIZE, String.valueOf(128 * 1024 * 1024)) + .load(tableLocation + "#entries"); + assertThat(entriesDf.javaRDD().getNumPartitions()).as("Num partitions must match").isEqualTo(1); + } + + @TestTemplate + public void testDefaultMetadataSplitSize() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + Table icebergTable = tables.create(SCHEMA, spec, options, tableLocation); + + List expectedRecords = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset originalDf = spark.createDataFrame(expectedRecords, SimpleRecord.class); + originalDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + int splitSize = (int) TableProperties.METADATA_SPLIT_SIZE_DEFAULT; // 32MB split size + + int expectedSplits = + ((int) + tables + .load(tableLocation + "#entries") + .currentSnapshot() + .allManifests(icebergTable.io()) + .get(0) + .length() + + splitSize + - 1) + / splitSize; + + Dataset metadataDf = spark.read().format("iceberg").load(tableLocation + "#entries"); + + int partitionNum = metadataDf.javaRDD().getNumPartitions(); + assertThat(partitionNum).as("Spark partitions should match").isEqualTo(expectedSplits); + } + + @TestTemplate + public void testExtraSnapshotMetadata() throws IOException { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + HadoopTables tables = new HadoopTables(CONF); + tables.create(SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap(), tableLocation); + + List expectedRecords = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset originalDf = spark.createDataFrame(expectedRecords, SimpleRecord.class); + originalDf + .select("id", "data") + .write() + .format("iceberg") + .mode("append") + .option(SparkWriteOptions.SNAPSHOT_PROPERTY_PREFIX + ".extra-key", "someValue") + .option(SparkWriteOptions.SNAPSHOT_PROPERTY_PREFIX + ".another-key", "anotherValue") + .save(tableLocation); + + Table table = tables.load(tableLocation); + + assertThat(table.currentSnapshot().summary()) + .containsEntry("extra-key", "someValue") + .containsEntry("another-key", "anotherValue"); + } + + @TestTemplate + public void testExtraSnapshotMetadataWithSQL() throws InterruptedException, IOException { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + HadoopTables tables = new HadoopTables(CONF); + + Table table = + tables.create(SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap(), tableLocation); + + List expectedRecords = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset originalDf = spark.createDataFrame(expectedRecords, SimpleRecord.class); + originalDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + spark.read().format("iceberg").load(tableLocation).createOrReplaceTempView("target"); + Thread writerThread = + new Thread( + () -> { + Map properties = + ImmutableMap.of( + "writer-thread", + String.valueOf(Thread.currentThread().getName()), + SnapshotSummary.EXTRA_METADATA_PREFIX + "extra-key", + "someValue", + SnapshotSummary.EXTRA_METADATA_PREFIX + "another-key", + "anotherValue"); + CommitMetadata.withCommitProperties( + properties, + () -> { + spark.sql("INSERT INTO target VALUES (3, 'c'), (4, 'd')"); + return 0; + }, + RuntimeException.class); + }); + writerThread.setName("test-extra-commit-message-writer-thread"); + writerThread.start(); + writerThread.join(); + + List snapshots = Lists.newArrayList(table.snapshots()); + assertThat(snapshots).hasSize(2); + assertThat(snapshots.get(0).summary()).doesNotContainKey("writer-thread"); + assertThat(snapshots.get(1).summary()) + .containsEntry("writer-thread", "test-extra-commit-message-writer-thread") + .containsEntry("extra-key", "someValue") + .containsEntry("another-key", "anotherValue"); + } + + @TestTemplate + public void testExtraSnapshotMetadataWithDelete() + throws InterruptedException, NoSuchTableException { + spark.sessionState().conf().setConfString("spark.sql.shuffle.partitions", "1"); + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + List expectedRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset originalDf = spark.createDataFrame(expectedRecords, SimpleRecord.class); + originalDf.repartition(5, new Column("data")).select("id", "data").writeTo(tableName).append(); + Thread writerThread = + new Thread( + () -> { + Map properties = + ImmutableMap.of( + "writer-thread", + String.valueOf(Thread.currentThread().getName()), + SnapshotSummary.EXTRA_METADATA_PREFIX + "extra-key", + "someValue", + SnapshotSummary.EXTRA_METADATA_PREFIX + "another-key", + "anotherValue"); + CommitMetadata.withCommitProperties( + properties, + () -> { + spark.sql("DELETE FROM " + tableName + " where id = 1"); + return 0; + }, + RuntimeException.class); + }); + writerThread.setName("test-extra-commit-message-delete-thread"); + writerThread.start(); + writerThread.join(); + + Table table = validationCatalog.loadTable(tableIdent); + List snapshots = Lists.newArrayList(table.snapshots()); + + assertThat(snapshots).hasSize(2); + assertThat(snapshots.get(0).summary()).doesNotContainKey("writer-thread"); + assertThat(snapshots.get(1).summary()) + .containsEntry("writer-thread", "test-extra-commit-message-delete-thread") + .containsEntry("extra-key", "someValue") + .containsEntry("another-key", "anotherValue"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java new file mode 100644 index 000000000000..61d6501a6847 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java @@ -0,0 +1,760 @@ +/* + * 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.Files.localOutput; +import static org.apache.iceberg.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.apache.iceberg.data.FileHelpers.encrypt; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.nio.file.Path; +import java.time.OffsetDateTime; +import java.util.Arrays; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericFileWriterFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.data.GenericsHelpers; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.expressions.UnsafeRow; +import org.apache.spark.sql.connector.expressions.filter.Predicate; +import org.apache.spark.sql.connector.read.Batch; +import org.apache.spark.sql.connector.read.InputPartition; +import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.connector.read.SupportsPushDownV2Filters; +import org.apache.spark.sql.sources.And; +import org.apache.spark.sql.sources.EqualTo; +import org.apache.spark.sql.sources.Filter; +import org.apache.spark.sql.sources.GreaterThan; +import org.apache.spark.sql.sources.LessThan; +import org.apache.spark.sql.sources.Not; +import org.apache.spark.sql.sources.StringStartsWith; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.assertj.core.api.AbstractObjectAssert; +import org.junit.jupiter.api.AfterAll; +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; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestFilteredScan { + private static final Configuration CONF = new Configuration(); + private static final HadoopTables TABLES = new HadoopTables(CONF); + + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "ts", Types.TimestampType.withZone()), + Types.NestedField.optional(3, "data", Types.StringType.get())); + + private static final PartitionSpec BUCKET_BY_ID = + PartitionSpec.builderFor(SCHEMA).bucket("id", 4).build(); + + private static final PartitionSpec PARTITION_BY_DAY = + PartitionSpec.builderFor(SCHEMA).day("ts").build(); + + private static final PartitionSpec PARTITION_BY_HOUR = + PartitionSpec.builderFor(SCHEMA).hour("ts").build(); + + private static final PartitionSpec PARTITION_BY_DATA = + PartitionSpec.builderFor(SCHEMA).identity("data").build(); + + private static final PartitionSpec PARTITION_BY_ID = + PartitionSpec.builderFor(SCHEMA).identity("id").build(); + + private static SparkSession spark = null; + + @BeforeAll + public static void startSpark() { + TestFilteredScan.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestFilteredScan.spark; + TestFilteredScan.spark = null; + currentSpark.stop(); + } + + @TempDir private Path temp; + + @Parameter(index = 0) + private FileFormat fileFormat; + + @Parameter(index = 1) + private boolean vectorized; + + @Parameter(index = 2) + private PlanningMode planningMode; + + @Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.PARQUET, false, LOCAL}, + {FileFormat.PARQUET, true, DISTRIBUTED}, + {FileFormat.AVRO, false, LOCAL}, + {FileFormat.ORC, false, DISTRIBUTED}, + {FileFormat.ORC, true, LOCAL} + }; + } + + private File parent = null; + private File unpartitioned = null; + private List records = null; + + @BeforeEach + public void writeUnpartitionedTable() throws IOException { + this.parent = temp.resolve("TestFilteredScan").toFile(); + this.unpartitioned = new File(parent, "unpartitioned"); + File dataFolder = new File(unpartitioned, "data"); + assertThat(dataFolder.mkdirs()).as("Mkdir should succeed").isTrue(); + + Table table = + TABLES.create( + SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DATA_PLANNING_MODE, + planningMode.modeName(), + TableProperties.DELETE_PLANNING_MODE, + planningMode.modeName()), + unpartitioned.toString()); + Schema tableSchema = table.schema(); // use the table schema because ids are reassigned + + File testFile = new File(dataFolder, fileFormat.addExtension(UUID.randomUUID().toString())); + + this.records = testRecords(tableSchema); + + DataWriter writer = + new GenericFileWriterFactory.Builder() + .dataFileFormat(fileFormat) + .dataSchema(tableSchema) + .build() + .newDataWriter(encrypt(localOutput(testFile)), PartitionSpec.unpartitioned(), null); + try (writer) { + writer.write(records); + } + + DataFile file = writer.toDataFile(); + + table.newAppend().appendFile(file).commit(); + } + + @TestTemplate + public void testUnpartitionedIDFilters() { + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + for (int i = 0; i < 10; i += 1) { + pushFilters(builder, EqualTo.apply("id", i)); + Batch scan = builder.build().toBatch(); + + InputPartition[] partitions = scan.planInputPartitions(); + assertThat(partitions).as("Should only create one task for a small file").hasSize(1); + + // validate row filtering + assertEqualsSafe( + SCHEMA.asStruct(), expected(i), read(unpartitioned.toString(), vectorized, "id = " + i)); + } + } + + @TestTemplate + public void testUnpartitionedCaseInsensitiveIDFilters() { + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); + + // set spark.sql.caseSensitive to false + String caseSensitivityBeforeTest = TestFilteredScan.spark.conf().get("spark.sql.caseSensitive"); + 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); + + pushFilters( + builder, + EqualTo.apply("ID", i)); // note lower(ID) == lower(id), so there must be a match + Batch scan = builder.build().toBatch(); + + InputPartition[] tasks = scan.planInputPartitions(); + assertThat(tasks).as("Should only create one task for a small file").hasSize(1); + + // validate row filtering + assertEqualsSafe( + SCHEMA.asStruct(), + expected(i), + read(unpartitioned.toString(), vectorized, "id = " + i)); + } + } finally { + // return global conf to previous state + TestFilteredScan.spark.conf().set("spark.sql.caseSensitive", caseSensitivityBeforeTest); + } + } + + @TestTemplate + public void testUnpartitionedTimestampFilter() { + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); + + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + pushFilters(builder, LessThan.apply("ts", "2017-12-22T00:00:00+00:00")); + Batch scan = builder.build().toBatch(); + + InputPartition[] tasks = scan.planInputPartitions(); + assertThat(tasks).as("Should only create one task for a small file").hasSize(1); + + assertEqualsSafe( + SCHEMA.asStruct(), + expected(5, 6, 7, 8, 9), + read( + unpartitioned.toString(), + vectorized, + "ts < cast('2017-12-22 00:00:00+00:00' as timestamp)")); + } + + @TestTemplate + public void limitPushedDownToSparkScan() { + assumeThat(fileFormat) + .as("no need to run this across the entire test matrix") + .isEqualTo(FileFormat.PARQUET); + + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); + + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + long limit = 23; + // simulate Spark pushing down the limit to the scan builder + builder.pushLimit((int) limit); + assertThat(builder).extracting("limit").isEqualTo((int) limit); + + // verify batch scan + AbstractObjectAssert scanAssert = assertThat(builder.build()).extracting("scan"); + if (LOCAL == planningMode) { + scanAssert = scanAssert.extracting("scan"); + } + + scanAssert.extracting("context").extracting("minRowsRequested").isEqualTo(limit); + + // verify changelog scan + assertThat(builder.buildChangelogScan()) + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify CoW scan + assertThat(builder.buildCopyOnWriteScan()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify MoR scan + scanAssert = assertThat(builder.buildMergeOnReadScan()).extracting("scan"); + if (LOCAL == planningMode) { + scanAssert = scanAssert.extracting("scan"); + } + + scanAssert.extracting("context").extracting("minRowsRequested").isEqualTo(limit); + } + + @TestTemplate + public void limitPushedDownToSparkScanForMetadataTable() { + assumeThat(fileFormat) + .as("no need to run this across the entire test matrix") + .isEqualTo(FileFormat.PARQUET); + + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", unpartitioned.toString())); + + // load the snapshots metadata table + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path") + "#snapshots"), options); + + long limit = 23; + // simulate Spark pushing down the limit to the scan builder + builder.pushLimit((int) limit); + assertThat(builder).extracting("limit").isEqualTo((int) limit); + + // verify batch scan + assertThat(builder.build()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify CoW scan + assertThat(builder.buildCopyOnWriteScan()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + + // verify MoR scan + assertThat(builder.buildMergeOnReadScan()) + .extracting("scan") + .extracting("scan") + .extracting("context") + .extracting("minRowsRequested") + .isEqualTo(limit); + } + + @TestTemplate + public void testBucketPartitionedIDFilters() { + Table table = buildPartitionedTable("bucketed_by_id", BUCKET_BY_ID); + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", table.location())); + + Batch unfiltered = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options).build().toBatch(); + assertThat(unfiltered.planInputPartitions()) + .as("Unfiltered table should created 4 read tasks") + .hasSize(4); + + for (int i = 0; i < 10; i += 1) { + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + pushFilters(builder, EqualTo.apply("id", i)); + Batch scan = builder.build().toBatch(); + + InputPartition[] tasks = scan.planInputPartitions(); + + // validate predicate push-down + assertThat(tasks).as("Should only create one task for a single bucket").hasSize(1); + + // validate row filtering + assertEqualsSafe( + SCHEMA.asStruct(), expected(i), read(table.location(), vectorized, "id = " + i)); + } + } + + @SuppressWarnings("checkstyle:AvoidNestedBlocks") + @TestTemplate + public void testDayPartitionedTimestampFilters() { + Table table = buildPartitionedTable("partitioned_by_day", PARTITION_BY_DAY); + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", table.location())); + Batch unfiltered = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options).build().toBatch(); + + assertThat(unfiltered.planInputPartitions()) + .as("Unfiltered table should created 2 read tasks") + .hasSize(2); + + { + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + pushFilters(builder, LessThan.apply("ts", "2017-12-22T00:00:00+00:00")); + Batch scan = builder.build().toBatch(); + + InputPartition[] tasks = scan.planInputPartitions(); + assertThat(tasks).as("Should create one task for 2017-12-21").hasSize(1); + + assertEqualsSafe( + SCHEMA.asStruct(), + expected(5, 6, 7, 8, 9), + read( + table.location(), vectorized, "ts < cast('2017-12-22 00:00:00+00:00' as timestamp)")); + } + + { + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + pushFilters( + builder, + And.apply( + GreaterThan.apply("ts", "2017-12-22T06:00:00+00:00"), + LessThan.apply("ts", "2017-12-22T08:00:00+00:00"))); + Batch scan = builder.build().toBatch(); + + InputPartition[] tasks = scan.planInputPartitions(); + assertThat(tasks).as("Should create one task for 2017-12-22").hasSize(1); + + assertEqualsSafe( + SCHEMA.asStruct(), + expected(1, 2), + read( + table.location(), + vectorized, + "ts > cast('2017-12-22 06:00:00+00:00' as timestamp) and " + + "ts < cast('2017-12-22 08:00:00+00:00' as timestamp)")); + } + } + + @SuppressWarnings("checkstyle:AvoidNestedBlocks") + @TestTemplate + public void testHourPartitionedTimestampFilters() { + Table table = buildPartitionedTable("partitioned_by_hour", PARTITION_BY_HOUR); + + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", table.location())); + Batch unfiltered = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options).build().toBatch(); + + assertThat(unfiltered.planInputPartitions()) + .as("Unfiltered table should created 9 read tasks") + .hasSize(9); + + { + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + pushFilters(builder, LessThan.apply("ts", "2017-12-22T00:00:00+00:00")); + Batch scan = builder.build().toBatch(); + + InputPartition[] tasks = scan.planInputPartitions(); + assertThat(tasks).as("Should create 4 tasks for 2017-12-21: 15, 17, 21, 22").hasSize(4); + + assertEqualsSafe( + SCHEMA.asStruct(), + expected(8, 9, 7, 6, 5), + read( + table.location(), vectorized, "ts < cast('2017-12-22 00:00:00+00:00' as timestamp)")); + } + + { + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + pushFilters( + builder, + And.apply( + GreaterThan.apply("ts", "2017-12-22T06:00:00+00:00"), + LessThan.apply("ts", "2017-12-22T08:00:00+00:00"))); + Batch scan = builder.build().toBatch(); + + InputPartition[] tasks = scan.planInputPartitions(); + assertThat(tasks).as("Should create 2 tasks for 2017-12-22: 6, 7").hasSize(2); + + assertEqualsSafe( + SCHEMA.asStruct(), + expected(2, 1), + read( + table.location(), + vectorized, + "ts > cast('2017-12-22 06:00:00+00:00' as timestamp) and " + + "ts < cast('2017-12-22 08:00:00+00:00' as timestamp)")); + } + } + + @SuppressWarnings("checkstyle:AvoidNestedBlocks") + @TestTemplate + public void testFilterByNonProjectedColumn() { + { + Schema actualProjection = SCHEMA.select("id", "data"); + List expected = Lists.newArrayList(); + for (Record rec : expected(5, 6, 7, 8, 9)) { + expected.add(projectFlat(actualProjection, rec)); + } + + assertEqualsSafe( + actualProjection.asStruct(), + expected, + read( + unpartitioned.toString(), + vectorized, + "ts < cast('2017-12-22 00:00:00+00:00' as timestamp)", + "id", + "data")); + } + + { + // only project id: ts will be projected because of the filter, but data will not be included + + Schema actualProjection = SCHEMA.select("id"); + List expected = Lists.newArrayList(); + for (Record rec : expected(1, 2)) { + expected.add(projectFlat(actualProjection, rec)); + } + + assertEqualsSafe( + actualProjection.asStruct(), + expected, + read( + unpartitioned.toString(), + vectorized, + "ts > cast('2017-12-22 06:00:00+00:00' as timestamp) and " + + "ts < cast('2017-12-22 08:00:00+00:00' as timestamp)", + "id")); + } + } + + @TestTemplate + public void testPartitionedByDataStartsWithFilter() { + Table table = buildPartitionedTable("partitioned_by_data", PARTITION_BY_DATA); + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", table.location())); + + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + pushFilters(builder, new StringStartsWith("data", "junc")); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(1); + } + + @TestTemplate + public void testPartitionedByDataNotStartsWithFilter() { + Table table = buildPartitionedTable("partitioned_by_data", PARTITION_BY_DATA); + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", table.location())); + + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + pushFilters(builder, new Not(new StringStartsWith("data", "junc"))); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(9); + } + + @TestTemplate + public void testPartitionedByIdStartsWith() { + Table table = buildPartitionedTable("partitioned_by_id", PARTITION_BY_ID); + + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", table.location())); + + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + pushFilters(builder, new StringStartsWith("data", "junc")); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(1); + } + + @TestTemplate + public void testPartitionedByIdNotStartsWith() { + Table table = buildPartitionedTable("partitioned_by_id", PARTITION_BY_ID); + + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", table.location())); + + SparkScanBuilder builder = + new SparkScanBuilder(spark, TABLES.load(options.get("path")), options); + + pushFilters(builder, new Not(new StringStartsWith("data", "junc"))); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(9); + } + + @TestTemplate + public void testUnpartitionedStartsWith() { + Dataset df = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(unpartitioned.toString()); + + List matchedData = + df.select("data").where("data LIKE 'jun%'").as(Encoders.STRING()).collectAsList(); + + assertThat(matchedData).singleElement().isEqualTo("junction"); + } + + @TestTemplate + public void testUnpartitionedNotStartsWith() { + Dataset df = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(unpartitioned.toString()); + + List matchedData = + df.select("data").where("data NOT LIKE 'jun%'").as(Encoders.STRING()).collectAsList(); + + List expected = + testRecords(SCHEMA).stream() + .map(r -> r.getField("data").toString()) + .filter(d -> !d.startsWith("jun")) + .collect(Collectors.toList()); + + assertThat(matchedData).hasSize(9).containsExactlyInAnyOrderElementsOf(expected); + } + + private static Record projectFlat(Schema projection, Record record) { + Record result = GenericRecord.create(projection); + List fields = projection.asStruct().fields(); + for (int i = 0; i < fields.size(); i += 1) { + Types.NestedField field = fields.get(i); + result.set(i, record.getField(field.name())); + } + return result; + } + + public static void assertEqualsUnsafe( + Types.StructType struct, List expected, List actual) { + // TODO: match records by ID + int numRecords = Math.min(expected.size(), actual.size()); + for (int i = 0; i < numRecords; i += 1) { + GenericsHelpers.assertEqualsUnsafe(struct, expected.get(i), actual.get(i)); + } + assertThat(actual).as("Number of results should match expected").hasSameSizeAs(expected); + } + + public static void assertEqualsSafe( + Types.StructType struct, List expected, List actual) { + // TODO: match records by ID + int numRecords = Math.min(expected.size(), actual.size()); + for (int i = 0; i < numRecords; i += 1) { + GenericsHelpers.assertEqualsSafe(struct, expected.get(i), actual.get(i)); + } + assertThat(actual).as("Number of results should match expected").hasSameSizeAs(expected); + } + + private List expected(int... ordinals) { + List expected = Lists.newArrayListWithExpectedSize(ordinals.length); + for (int ord : ordinals) { + expected.add(records.get(ord)); + } + return expected; + } + + private void pushFilters(ScanBuilder scan, Filter... filters) { + assertThat(scan).isInstanceOf(SupportsPushDownV2Filters.class); + SupportsPushDownV2Filters filterable = (SupportsPushDownV2Filters) scan; + filterable.pushPredicates(Arrays.stream(filters).map(Filter::toV2).toArray(Predicate[]::new)); + } + + private Table buildPartitionedTable(String desc, PartitionSpec spec) { + File location = new File(parent, desc); + Table table = TABLES.create(SCHEMA, spec, location.toString()); + + // Do not combine or split files because the tests expect a split per partition. + // A target split size of 2048 helps us achieve that. + table.updateProperties().set("read.split.target-size", "2048").commit(); + + // copy the unpartitioned table into the partitioned table to produce the partitioned data + Dataset allRows = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(unpartitioned.toString()); + + // disable fanout writers to locally order records for future verifications + allRows + .write() + .option(SparkWriteOptions.FANOUT_ENABLED, "false") + .format("iceberg") + .mode("append") + .save(table.location()); + + table.refresh(); + + return table; + } + + private List testRecords(Schema schema) { + return Lists.newArrayList( + record(schema, 0L, parse("2017-12-22T09:20:44.294658+00:00"), "junction"), + record(schema, 1L, parse("2017-12-22T07:15:34.582910+00:00"), "alligator"), + record(schema, 2L, parse("2017-12-22T06:02:09.243857+00:00"), ""), + record(schema, 3L, parse("2017-12-22T03:10:11.134509+00:00"), "clapping"), + record(schema, 4L, parse("2017-12-22T00:34:00.184671+00:00"), "brush"), + record(schema, 5L, parse("2017-12-21T22:20:08.935889+00:00"), "trap"), + record(schema, 6L, parse("2017-12-21T21:55:30.589712+00:00"), "element"), + record(schema, 7L, parse("2017-12-21T17:31:14.532797+00:00"), "limited"), + record(schema, 8L, parse("2017-12-21T15:21:51.237521+00:00"), "global"), + record(schema, 9L, parse("2017-12-21T15:02:15.230570+00:00"), "goldfish")); + } + + private static List read(String table, boolean vectorized, String expr) { + return read(table, vectorized, expr, "*"); + } + + private static List read( + String table, boolean vectorized, String expr, String select0, String... selectN) { + Dataset dataset = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(table) + .filter(expr) + .select(select0, selectN); + return dataset.collectAsList(); + } + + private static OffsetDateTime parse(String timestamp) { + return OffsetDateTime.parse(timestamp); + } + + private static Record record(Schema schema, Object... values) { + Record rec = GenericRecord.create(schema); + for (int i = 0; i < values.length; i += 1) { + rec.set(i, values[i]); + } + return rec; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java new file mode 100644 index 000000000000..c4e0d26c1c31 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -0,0 +1,229 @@ +/* + * 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.Files.localInput; +import static org.apache.iceberg.Files.localOutput; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.nio.file.Path; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeoutException; +import org.apache.avro.generic.GenericData; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.execution.streaming.MemoryStream; +import org.apache.spark.sql.streaming.StreamingQuery; +import org.apache.spark.sql.streaming.StreamingQueryException; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import scala.Option; +import scala.collection.JavaConverters; + +public class TestForwardCompatibility { + private static final Configuration CONF = new Configuration(); + + private static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + // create a spec for the schema that uses a "zero" transform that produces all 0s + private static final PartitionSpec UNKNOWN_SPEC = + org.apache.iceberg.TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(0) + .addField("zero", 1, "id_zero") + .build(); + // create a fake spec to use to write table metadata + private static final PartitionSpec FAKE_SPEC = + org.apache.iceberg.TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(0) + .addField("identity", 1, "id_zero") + .build(); + + @TempDir private Path temp; + + private static SparkSession spark = null; + + @BeforeAll + public static void startSpark() { + TestForwardCompatibility.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestForwardCompatibility.spark; + TestForwardCompatibility.spark = null; + currentSpark.stop(); + } + + @Test + public void testSparkWriteFailsUnknownTransform() throws IOException { + File parent = temp.resolve("avro").toFile(); + File location = new File(parent, "test"); + File dataFolder = new File(location, "data"); + assertThat(dataFolder.mkdirs()).isTrue(); + + HadoopTables tables = new HadoopTables(CONF); + tables.create(SCHEMA, UNKNOWN_SPEC, location.toString()); + + List expected = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + assertThatThrownBy( + () -> + df.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(location.toString())) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageEndingWith("Cannot write using unsupported transforms: zero"); + } + + @Test + public void testSparkStreamingWriteFailsUnknownTransform() throws IOException, TimeoutException { + File parent = temp.resolve("avro").toFile(); + File location = new File(parent, "test"); + File dataFolder = new File(location, "data"); + assertThat(dataFolder.mkdirs()).isTrue(); + File checkpoint = new File(parent, "checkpoint"); + assertThat(checkpoint.mkdirs()).isTrue(); + + HadoopTables tables = new HadoopTables(CONF); + tables.create(SCHEMA, UNKNOWN_SPEC, location.toString()); + + MemoryStream inputStream = newMemoryStream(1, spark.sqlContext(), Encoders.INT()); + StreamingQuery query = + inputStream + .toDF() + .selectExpr("value AS id", "CAST (value AS STRING) AS data") + .writeStream() + .outputMode("append") + .format("iceberg") + .option("checkpointLocation", checkpoint.toString()) + .option("path", location.toString()) + .start(); + + List batch1 = Lists.newArrayList(1, 2); + send(batch1, inputStream); + + assertThatThrownBy(query::processAllAvailable) + .isInstanceOf(StreamingQueryException.class) + .hasMessageContaining("Cannot write using unsupported transforms: zero"); + } + + @Test + public void testSparkCanReadUnknownTransform() throws IOException { + File parent = temp.resolve("avro").toFile(); + File location = new File(parent, "test"); + File dataFolder = new File(location, "data"); + assertThat(dataFolder.mkdirs()).isTrue(); + + HadoopTables tables = new HadoopTables(CONF); + Table table = tables.create(SCHEMA, UNKNOWN_SPEC, location.toString()); + + // enable snapshot inheritance to avoid rewriting the manifest with an unknown transform + table.updateProperties().set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); + + List expected = RandomData.generateList(table.schema(), 100, 1L); + + File parquetFile = + new File(dataFolder, FileFormat.PARQUET.addExtension(UUID.randomUUID().toString())); + FileAppender writer = + Parquet.write(localOutput(parquetFile)).schema(table.schema()).build(); + try { + writer.addAll(expected); + } finally { + writer.close(); + } + + DataFile file = + DataFiles.builder(FAKE_SPEC) + .withInputFile(localInput(parquetFile)) + .withMetrics(writer.metrics()) + .withPartitionPath("id_zero=0") + .build(); + + OutputFile manifestFile = localOutput(FileFormat.AVRO.addExtension(temp.toFile().toString())); + ManifestWriter manifestWriter = ManifestFiles.write(FAKE_SPEC, manifestFile); + try { + manifestWriter.add(file); + } finally { + manifestWriter.close(); + } + + table.newFastAppend().appendManifest(manifestWriter.toManifestFile()).commit(); + + Dataset df = spark.read().format("iceberg").load(location.toString()); + + List rows = df.collectAsList(); + assertThat(rows).as("Should contain 100 rows").hasSize(100); + + for (int i = 0; i < expected.size(); i += 1) { + TestHelpers.assertEqualsSafe(table.schema().asStruct(), expected.get(i), rows.get(i)); + } + } + + private MemoryStream newMemoryStream(int id, SQLContext sqlContext, Encoder encoder) { + return new MemoryStream<>(id, sqlContext, Option.empty(), encoder); + } + + private void send(List records, MemoryStream stream) { + stream.addData(JavaConverters.asScalaBuffer(records)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSource.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSource.java new file mode 100644 index 000000000000..a850275118db --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSource.java @@ -0,0 +1,42 @@ +/* + * 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.catalog.TableIdentifier; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +public class TestIcebergSource extends IcebergSource { + @Override + public String shortName() { + return "iceberg-test"; + } + + @Override + public Identifier extractIdentifier(CaseInsensitiveStringMap options) { + TableIdentifier ti = TableIdentifier.parse(options.get("iceberg.table.name")); + return Identifier.of(ti.namespace().levels(), ti.name()); + } + + @Override + public String extractCatalog(CaseInsensitiveStringMap options) { + return SparkSession.active().sessionState().catalogManager().currentCatalog().name(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java new file mode 100644 index 000000000000..701c421a253b --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHadoopTables.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.io.File; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopTables; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceHadoopTables extends TestIcebergSourceTablesBase { + + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + @TempDir private File tableDir; + String tableLocation = null; + + @BeforeEach + public void setupTable() { + this.tableLocation = tableDir.toURI().toString(); + } + + @Override + public Table createTable( + TableIdentifier ident, Schema schema, PartitionSpec spec, Map properties) { + return TABLES.create(schema, spec, properties, tableLocation); + } + + @Override + public void dropTable(TableIdentifier ident) { + TABLES.dropTable(tableLocation); + } + + @Override + public Table loadTable(TableIdentifier ident, String entriesSuffix) { + return TABLES.load(loadLocation(ident, entriesSuffix)); + } + + @Override + public String loadLocation(TableIdentifier ident, String entriesSuffix) { + return String.format("%s#%s", loadLocation(ident), entriesSuffix); + } + + @Override + public String loadLocation(TableIdentifier ident) { + return tableLocation; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java new file mode 100644 index 000000000000..9120bbcc35a3 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceHiveTables.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.io.IOException; +import java.util.Map; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; + +public class TestIcebergSourceHiveTables extends TestIcebergSourceTablesBase { + + private static TableIdentifier currentIdentifier; + + @BeforeAll + public static void start() { + Namespace db = Namespace.of("db"); + if (!catalog.namespaceExists(db)) { + catalog.createNamespace(db); + } + } + + @AfterEach + public void dropTable() throws IOException { + if (!catalog.tableExists(currentIdentifier)) { + return; + } + + dropTable(currentIdentifier); + } + + @Override + public Table createTable( + TableIdentifier ident, Schema schema, PartitionSpec spec, Map properties) { + TestIcebergSourceHiveTables.currentIdentifier = ident; + return TestIcebergSourceHiveTables.catalog.createTable(ident, schema, spec, properties); + } + + @Override + public void dropTable(TableIdentifier ident) throws IOException { + Table table = catalog.loadTable(ident); + Path tablePath = new Path(table.location()); + FileSystem fs = tablePath.getFileSystem(spark.sessionState().newHadoopConf()); + fs.delete(tablePath, true); + catalog.dropTable(ident, false); + } + + @Override + public Table loadTable(TableIdentifier ident, String entriesSuffix) { + TableIdentifier identifier = + TableIdentifier.of(ident.namespace().level(0), ident.name(), entriesSuffix); + return TestIcebergSourceHiveTables.catalog.loadTable(identifier); + } + + @Override + public String loadLocation(TableIdentifier ident, String entriesSuffix) { + return String.format("%s.%s", loadLocation(ident), entriesSuffix); + } + + @Override + public String loadLocation(TableIdentifier ident) { + return ident.toString(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java new file mode 100644 index 000000000000..63994726cafb --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -0,0 +1,2459 @@ +/* + * 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.ManifestContent.DATA; +import static org.apache.iceberg.ManifestContent.DELETES; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Path; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.StringJoiner; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecordBuilder; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.DeleteOrphanFiles; +import org.apache.iceberg.actions.RewriteManifests; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.mapping.MappingUtil; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +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.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkSQLProperties; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.spark.actions.SparkActions; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.spark.SparkException; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.functions; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public abstract class TestIcebergSourceTablesBase extends TestBase { + + private static final Schema SCHEMA = + new Schema( + optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); + + private static final Schema SCHEMA2 = + new Schema( + optional(1, "id", Types.IntegerType.get()), + optional(2, "data", Types.StringType.get()), + optional(3, "category", Types.StringType.get())); + + private static final Schema SCHEMA3 = + new Schema( + optional(1, "id", Types.IntegerType.get()), + optional(3, "category", Types.StringType.get())); + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("id").build(); + + @TempDir protected Path temp; + + public abstract Table createTable( + TableIdentifier ident, Schema schema, PartitionSpec spec, Map properties); + + public abstract Table loadTable(TableIdentifier ident, String entriesSuffix); + + public abstract String loadLocation(TableIdentifier ident, String entriesSuffix); + + public abstract String loadLocation(TableIdentifier ident); + + public abstract void dropTable(TableIdentifier ident) throws IOException; + + @AfterEach + public void removeTable() { + spark.sql("DROP TABLE IF EXISTS parquet_table"); + } + + private Table createTable(TableIdentifier ident, Schema schema, PartitionSpec spec) { + return createTable(ident, schema, spec, ImmutableMap.of()); + } + + @Test + public synchronized void testTablesSupport() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "table"); + createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + + List expectedRecords = + Lists.newArrayList( + new SimpleRecord(1, "1"), new SimpleRecord(2, "2"), new SimpleRecord(3, "3")); + + Dataset inputDf = spark.createDataFrame(expectedRecords, SimpleRecord.class); + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + + Dataset resultDf = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); + List actualRecords = + resultDf.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + + assertThat(actualRecords).as("Records should match").isEqualTo(expectedRecords); + } + + @Test + public void testEntriesTable() throws Exception { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "entries_test"); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + Table entriesTable = loadTable(tableIdentifier, "entries"); + + List records = Lists.newArrayList(new SimpleRecord(1, "1")); + + Dataset inputDf = spark.createDataFrame(records, SimpleRecord.class); + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + + Dataset entriesTableDs = + spark.read().format("iceberg").load(loadLocation(tableIdentifier, "entries")); + List actual = TestHelpers.selectNonDerived(entriesTableDs).collectAsList(); + + Snapshot snapshot = table.currentSnapshot(); + + assertThat(snapshot.allManifests(table.io())).as("Should only contain one manifest").hasSize(1); + + InputFile manifest = table.io().newInputFile(snapshot.allManifests(table.io()).get(0).path()); + List expected = Lists.newArrayList(); + try (CloseableIterable rows = + Avro.read(manifest).project(entriesTable.schema()).build()) { + // each row must inherit snapshot_id and sequence_number + rows.forEach( + row -> { + row.put(2, 1L); // data sequence number + row.put(3, 1L); // file sequence number + GenericData.Record file = (GenericData.Record) row.get("data_file"); + TestHelpers.asMetadataRecord(file); + expected.add(row); + }); + } + + assertThat(expected).as("Entries table should have one row").hasSize(1); + assertThat(actual).as("Actual results should have one row").hasSize(1); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(entriesTableDs), expected.get(0), actual.get(0)); + } + + @Test + public void testEntriesTablePartitionedPrune() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "entries_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + + List records = Lists.newArrayList(new SimpleRecord(1, "1")); + + Dataset inputDf = spark.createDataFrame(records, SimpleRecord.class); + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "entries")) + .select("status") + .collectAsList(); + + assertThat(actual).singleElement().satisfies(row -> assertThat(row.getInt(0)).isEqualTo(1)); + } + + @Test + public void testEntriesTableDataFilePrune() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "entries_test"); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + + List records = Lists.newArrayList(new SimpleRecord(1, "1")); + + Dataset inputDf = spark.createDataFrame(records, SimpleRecord.class); + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + + List singleActual = + rowsToJava( + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "entries")) + .select("data_file.file_path") + .collectAsList()); + + List singleExpected = ImmutableList.of(row(file.location())); + + assertEquals( + "Should prune a single element from a nested struct", singleExpected, singleActual); + } + + @Test + public void testEntriesTableDataFilePruneMulti() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "entries_test"); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + + List records = Lists.newArrayList(new SimpleRecord(1, "1")); + + Dataset inputDf = spark.createDataFrame(records, SimpleRecord.class); + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + + List multiActual = + rowsToJava( + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "entries")) + .select( + "data_file.file_path", + "data_file.value_counts", + "data_file.record_count", + "data_file.column_sizes") + .collectAsList()); + + List multiExpected = + ImmutableList.of( + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); + + assertEquals("Should prune a single element from a nested struct", multiExpected, multiActual); + } + + @Test + public void testFilesSelectMap() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "entries_test"); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + + List records = Lists.newArrayList(new SimpleRecord(1, "1")); + + Dataset inputDf = spark.createDataFrame(records, SimpleRecord.class); + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + + List multiActual = + rowsToJava( + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "files")) + .select("file_path", "value_counts", "record_count", "column_sizes") + .collectAsList()); + + List multiExpected = + ImmutableList.of( + row(file.location(), file.valueCounts(), file.recordCount(), file.columnSizes())); + + assertEquals("Should prune a single element from a row", multiExpected, multiActual); + } + + @Test + public void testAllEntriesTable() throws Exception { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "entries_test"); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + Table entriesTable = loadTable(tableIdentifier, "all_entries"); + + Dataset df1 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); + Dataset df2 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "b")), SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + // delete the first file to test that not only live files are listed + table.newDelete().deleteFromRowFilter(Expressions.equal("id", 1)).commit(); + + // add a second file + df2.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + // ensure table data isn't stale + table.refresh(); + + Dataset entriesTableDs = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "all_entries")) + .orderBy("snapshot_id"); + List actual = TestHelpers.selectNonDerived(entriesTableDs).collectAsList(); + + List expected = Lists.newArrayList(); + for (ManifestFile manifest : + Iterables.concat(Iterables.transform(table.snapshots(), s -> s.allManifests(table.io())))) { + InputFile in = table.io().newInputFile(manifest.path()); + try (CloseableIterable rows = + Avro.read(in).project(entriesTable.schema()).build()) { + // each row must inherit snapshot_id and sequence_number + rows.forEach( + row -> { + if (row.get("snapshot_id").equals(table.currentSnapshot().snapshotId())) { + row.put(2, 3L); // data sequence number + row.put(3, 3L); // file sequence number + } else { + row.put(2, 1L); // data sequence number + row.put(3, 1L); // file sequence number + } + GenericData.Record file = (GenericData.Record) row.get("data_file"); + TestHelpers.asMetadataRecord(file); + expected.add(row); + }); + } + } + + expected.sort(Comparator.comparing(o -> (Long) o.get("snapshot_id"))); + + assertThat(expected).as("Entries table should have 3 rows").hasSize(3); + assertThat(actual).as("Actual results should have 3 rows").hasSize(3); + + for (int i = 0; i < expected.size(); i += 1) { + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(entriesTableDs), expected.get(i), actual.get(i)); + } + } + + @Test + public void testCountEntriesTable() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "count_entries_test"); + createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + + // init load + List records = Lists.newArrayList(new SimpleRecord(1, "1")); + Dataset inputDf = spark.createDataFrame(records, SimpleRecord.class); + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + final int expectedEntryCount = 1; + + // count entries + assertThat( + spark.read().format("iceberg").load(loadLocation(tableIdentifier, "entries")).count()) + .as("Count should return " + expectedEntryCount) + .isEqualTo(expectedEntryCount); + + // count all_entries + assertThat( + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "all_entries")) + .count()) + .as("Count should return " + expectedEntryCount) + .isEqualTo(expectedEntryCount); + } + + @Test + public void testFilesTable() throws Exception { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "files_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + Table entriesTable = loadTable(tableIdentifier, "entries"); + + Dataset df1 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); + Dataset df2 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(2, "b")), SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + // add a second file + df2.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + // delete the first file to test that only live files are listed + table.newDelete().deleteFromRowFilter(Expressions.equal("id", 1)).commit(); + + Dataset filesTableDs = + spark.read().format("iceberg").load(loadLocation(tableIdentifier, "files")); + List actual = TestHelpers.selectNonDerived(filesTableDs).collectAsList(); + + List expected = Lists.newArrayList(); + for (ManifestFile manifest : table.currentSnapshot().dataManifests(table.io())) { + InputFile in = table.io().newInputFile(manifest.path()); + try (CloseableIterable rows = + Avro.read(in).project(entriesTable.schema()).build()) { + for (GenericData.Record record : rows) { + if ((Integer) record.get("status") < 2 /* added or existing */) { + GenericData.Record file = (GenericData.Record) record.get("data_file"); + TestHelpers.asMetadataRecord(file); + expected.add(file); + } + } + } + } + + assertThat(expected).as("Files table should have one row").hasSize(1); + assertThat(actual).as("Actual results should have one row").hasSize(1); + + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(filesTableDs), expected.get(0), actual.get(0)); + } + + @Test + public void testFilesTableWithSnapshotIdInheritance() throws Exception { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "files_inheritance_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + table.updateProperties().set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); + Table entriesTable = loadTable(tableIdentifier, "entries"); + + spark.sql( + String.format( + "CREATE TABLE parquet_table (data string, id int) " + + "USING parquet PARTITIONED BY (id) LOCATION '%s'", + temp.toFile())); + + List records = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + + Dataset inputDF = spark.createDataFrame(records, SimpleRecord.class); + inputDF.select("data", "id").write().mode("overwrite").insertInto("parquet_table"); + + NameMapping mapping = MappingUtil.create(table.schema()); + String mappingJson = NameMappingParser.toJson(mapping); + + table.updateProperties().set(TableProperties.DEFAULT_NAME_MAPPING, mappingJson).commit(); + + String stagingLocation = table.location() + "/metadata"; + SparkTableUtil.importSparkTable( + spark, + new org.apache.spark.sql.catalyst.TableIdentifier("parquet_table"), + table, + stagingLocation); + + Dataset filesTableDs = + spark.read().format("iceberg").load(loadLocation(tableIdentifier, "files")); + List actual = TestHelpers.selectNonDerived(filesTableDs).collectAsList(); + + List expected = Lists.newArrayList(); + for (ManifestFile manifest : table.currentSnapshot().dataManifests(table.io())) { + InputFile in = table.io().newInputFile(manifest.path()); + try (CloseableIterable rows = + Avro.read(in).project(entriesTable.schema()).build()) { + for (GenericData.Record record : rows) { + GenericData.Record file = (GenericData.Record) record.get("data_file"); + TestHelpers.asMetadataRecord(file); + expected.add(file); + } + } + } + + Types.StructType struct = TestHelpers.nonDerivedSchema(filesTableDs); + assertThat(expected).as("Files table should have 2 rows").hasSize(2); + assertThat(actual).as("Actual results should have 2 rows").hasSize(2); + TestHelpers.assertEqualsSafe(struct, expected.get(0), actual.get(0)); + TestHelpers.assertEqualsSafe(struct, expected.get(1), actual.get(1)); + } + + @Test + public void testV1EntriesTableWithSnapshotIdInheritance() throws Exception { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "entries_inheritance_test"); + Map properties = ImmutableMap.of(TableProperties.FORMAT_VERSION, "1"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC, properties); + + table.updateProperties().set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); + + spark.sql( + String.format( + "CREATE TABLE parquet_table (data string, id int) " + + "USING parquet PARTITIONED BY (id) LOCATION '%s'", + temp.toFile())); + + List records = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + + Dataset inputDF = spark.createDataFrame(records, SimpleRecord.class); + inputDF.select("data", "id").write().mode("overwrite").insertInto("parquet_table"); + + String stagingLocation = table.location() + "/metadata"; + SparkTableUtil.importSparkTable( + spark, + new org.apache.spark.sql.catalyst.TableIdentifier("parquet_table"), + table, + stagingLocation); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "entries")) + .select("sequence_number", "snapshot_id", "data_file") + .collectAsList(); + + table.refresh(); + + long snapshotId = table.currentSnapshot().snapshotId(); + + assertThat(actual).as("Entries table should have 2 rows").hasSize(2); + assertThat(actual) + .first() + .satisfies( + row -> { + assertThat(row.getLong(0)).isEqualTo(0); + assertThat(row.getLong(1)).isEqualTo(snapshotId); + }); + assertThat(actual) + .element(1) + .satisfies( + row -> { + assertThat(row.getLong(0)).isEqualTo(0); + assertThat(row.getLong(1)).isEqualTo(snapshotId); + }); + } + + @Test + public void testFilesUnpartitionedTable() throws Exception { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "unpartitioned_files_test"); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + Table entriesTable = loadTable(tableIdentifier, "entries"); + + Dataset df1 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); + Dataset df2 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(2, "b")), SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + DataFile toDelete = + Iterables.getOnlyElement(table.currentSnapshot().addedDataFiles(table.io())); + + // add a second file + df2.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + // delete the first file to test that only live files are listed + table.newDelete().deleteFile(toDelete).commit(); + + Dataset filesTableDs = + spark.read().format("iceberg").load(loadLocation(tableIdentifier, "files")); + List actual = TestHelpers.selectNonDerived(filesTableDs).collectAsList(); + + List expected = Lists.newArrayList(); + for (ManifestFile manifest : table.currentSnapshot().dataManifests(table.io())) { + InputFile in = table.io().newInputFile(manifest.path()); + try (CloseableIterable rows = + Avro.read(in).project(entriesTable.schema()).build()) { + for (GenericData.Record record : rows) { + if ((Integer) record.get("status") < 2 /* added or existing */) { + GenericData.Record file = (GenericData.Record) record.get("data_file"); + TestHelpers.asMetadataRecord(file); + expected.add(file); + } + } + } + } + + assertThat(expected).as("Files table should have one row").hasSize(1); + assertThat(actual).as("Actual results should have one row").hasSize(1); + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(filesTableDs), expected.get(0), actual.get(0)); + } + + @Test + public void testAllMetadataTablesWithStagedCommits() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "stage_aggregate_table_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + + table.updateProperties().set(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED, "true").commit(); + spark.conf().set(SparkSQLProperties.WAP_ID, "1234567"); + Dataset df1 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); + Dataset df2 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(2, "b")), SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + // add a second file + df2.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + List actualAllData = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "all_data_files")) + .collectAsList(); + + List actualAllManifests = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "all_manifests")) + .collectAsList(); + + List actualAllEntries = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "all_entries")) + .collectAsList(); + + assertThat(table.snapshots().iterator()).as("Stage table should have some snapshots").hasNext(); + assertThat(table.currentSnapshot()).as("Stage table should have null currentSnapshot").isNull(); + assertThat(actualAllData).as("Actual results should have two rows").hasSize(2); + assertThat(actualAllManifests).as("Actual results should have two rows").hasSize(2); + assertThat(actualAllEntries).as("Actual results should have two rows").hasSize(2); + } + + @Test + public void testAllDataFilesTable() throws Exception { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "files_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + Table entriesTable = loadTable(tableIdentifier, "entries"); + + Dataset df1 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); + Dataset df2 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(2, "b")), SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + // delete the first file to test that not only live files are listed + table.newDelete().deleteFromRowFilter(Expressions.equal("id", 1)).commit(); + + // add a second file + df2.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + // ensure table data isn't stale + table.refresh(); + + Dataset filesTableDs = + spark.read().format("iceberg").load(loadLocation(tableIdentifier, "all_data_files")); + List actual = TestHelpers.selectNonDerived(filesTableDs).collectAsList(); + actual.sort(Comparator.comparing(o -> o.getString(1))); + + List expected = Lists.newArrayList(); + Iterable dataManifests = + Iterables.concat( + Iterables.transform(table.snapshots(), snapshot -> snapshot.dataManifests(table.io()))); + for (ManifestFile manifest : dataManifests) { + InputFile in = table.io().newInputFile(manifest.path()); + try (CloseableIterable rows = + Avro.read(in).project(entriesTable.schema()).build()) { + for (GenericData.Record record : rows) { + if ((Integer) record.get("status") < 2 /* added or existing */) { + GenericData.Record file = (GenericData.Record) record.get("data_file"); + TestHelpers.asMetadataRecord(file); + expected.add(file); + } + } + } + } + + expected.sort(Comparator.comparing(o -> o.get("file_path").toString())); + + assertThat(expected).as("Files table should have two rows").hasSize(2); + assertThat(actual).as("Actual results should have two rows").hasSize(2); + for (int i = 0; i < expected.size(); i += 1) { + TestHelpers.assertEqualsSafe( + TestHelpers.nonDerivedSchema(filesTableDs), expected.get(i), actual.get(i)); + } + } + + @Test + public void testHistoryTable() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "history_test"); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + Table historyTable = loadTable(tableIdentifier, "history"); + + List records = Lists.newArrayList(new SimpleRecord(1, "1")); + Dataset inputDf = spark.createDataFrame(records, SimpleRecord.class); + + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long firstSnapshotTimestamp = table.currentSnapshot().timestampMillis(); + long firstSnapshotId = table.currentSnapshot().snapshotId(); + + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long secondSnapshotTimestamp = table.currentSnapshot().timestampMillis(); + long secondSnapshotId = table.currentSnapshot().snapshotId(); + + // rollback the table state to the first snapshot + table.manageSnapshots().rollbackTo(firstSnapshotId).commit(); + long rollbackTimestamp = Iterables.getLast(table.history()).timestampMillis(); + + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long thirdSnapshotTimestamp = table.currentSnapshot().timestampMillis(); + long thirdSnapshotId = table.currentSnapshot().snapshotId(); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "history")) + .collectAsList(); + + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(historyTable.schema(), "history")); + List expected = + Lists.newArrayList( + builder + .set("made_current_at", firstSnapshotTimestamp * 1000) + .set("snapshot_id", firstSnapshotId) + .set("parent_id", null) + .set("is_current_ancestor", true) + .build(), + builder + .set("made_current_at", secondSnapshotTimestamp * 1000) + .set("snapshot_id", secondSnapshotId) + .set("parent_id", firstSnapshotId) + .set( + "is_current_ancestor", + false) // commit rolled back, not an ancestor of the current table state + .build(), + builder + .set("made_current_at", rollbackTimestamp * 1000) + .set("snapshot_id", firstSnapshotId) + .set("parent_id", null) + .set("is_current_ancestor", true) + .build(), + builder + .set("made_current_at", thirdSnapshotTimestamp * 1000) + .set("snapshot_id", thirdSnapshotId) + .set("parent_id", firstSnapshotId) + .set("is_current_ancestor", true) + .build()); + + assertThat(actual).as("History table should have a row for each commit").hasSize(4); + TestHelpers.assertEqualsSafe(historyTable.schema().asStruct(), expected.get(0), actual.get(0)); + TestHelpers.assertEqualsSafe(historyTable.schema().asStruct(), expected.get(1), actual.get(1)); + TestHelpers.assertEqualsSafe(historyTable.schema().asStruct(), expected.get(2), actual.get(2)); + TestHelpers.assertEqualsSafe(historyTable.schema().asStruct(), expected.get(3), actual.get(3)); + } + + @Test + public void testSnapshotsTable() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "snapshots_test"); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + Table snapTable = loadTable(tableIdentifier, "snapshots"); + + List records = Lists.newArrayList(new SimpleRecord(1, "1")); + Dataset inputDf = spark.createDataFrame(records, SimpleRecord.class); + + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long firstSnapshotTimestamp = table.currentSnapshot().timestampMillis(); + long firstSnapshotId = table.currentSnapshot().snapshotId(); + String firstManifestList = table.currentSnapshot().manifestListLocation(); + + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + + long secondSnapshotTimestamp = table.currentSnapshot().timestampMillis(); + long secondSnapshotId = table.currentSnapshot().snapshotId(); + String secondManifestList = table.currentSnapshot().manifestListLocation(); + + // rollback the table state to the first snapshot + table.manageSnapshots().rollbackTo(firstSnapshotId).commit(); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "snapshots")) + .collectAsList(); + + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(snapTable.schema(), "snapshots")); + List expected = + Lists.newArrayList( + builder + .set("committed_at", firstSnapshotTimestamp * 1000) + .set("snapshot_id", firstSnapshotId) + .set("parent_id", null) + .set("operation", "append") + .set("manifest_list", firstManifestList) + .set( + "summary", + ImmutableMap.of( + "added-records", "1", + "added-data-files", "1", + "changed-partition-count", "1", + "total-data-files", "1", + "total-records", "1")) + .build(), + builder + .set("committed_at", secondSnapshotTimestamp * 1000) + .set("snapshot_id", secondSnapshotId) + .set("parent_id", firstSnapshotId) + .set("operation", "delete") + .set("manifest_list", secondManifestList) + .set( + "summary", + ImmutableMap.of( + "deleted-records", "1", + "deleted-data-files", "1", + "changed-partition-count", "1", + "total-records", "0", + "total-data-files", "0")) + .build()); + + assertThat(actual).as("Snapshots table should have a row for each snapshot").hasSize(2); + TestHelpers.assertEqualsSafe(snapTable.schema().asStruct(), expected.get(0), actual.get(0)); + TestHelpers.assertEqualsSafe(snapTable.schema().asStruct(), expected.get(1), actual.get(1)); + } + + @Test + public void testPrunedSnapshotsTable() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "snapshots_test"); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + + List records = Lists.newArrayList(new SimpleRecord(1, "1")); + Dataset inputDf = spark.createDataFrame(records, SimpleRecord.class); + + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long firstSnapshotTimestamp = table.currentSnapshot().timestampMillis(); + long firstSnapshotId = table.currentSnapshot().snapshotId(); + + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + + long secondSnapshotTimestamp = table.currentSnapshot().timestampMillis(); + + // rollback the table state to the first snapshot + table.manageSnapshots().rollbackTo(firstSnapshotId).commit(); + + Dataset actualDf = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "snapshots")) + .select("operation", "committed_at", "summary", "parent_id"); + + Schema projectedSchema = SparkSchemaUtil.convert(actualDf.schema()); + + List actual = actualDf.collectAsList(); + + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(projectedSchema, "snapshots")); + List expected = + Lists.newArrayList( + builder + .set("committed_at", firstSnapshotTimestamp * 1000) + .set("parent_id", null) + .set("operation", "append") + .set( + "summary", + ImmutableMap.of( + "added-records", "1", + "added-data-files", "1", + "changed-partition-count", "1", + "total-data-files", "1", + "total-records", "1")) + .build(), + builder + .set("committed_at", secondSnapshotTimestamp * 1000) + .set("parent_id", firstSnapshotId) + .set("operation", "delete") + .set( + "summary", + ImmutableMap.of( + "deleted-records", "1", + "deleted-data-files", "1", + "changed-partition-count", "1", + "total-records", "0", + "total-data-files", "0")) + .build()); + + assertThat(actual).as("Snapshots table should have a row for each snapshot").hasSize(2); + TestHelpers.assertEqualsSafe(projectedSchema.asStruct(), expected.get(0), actual.get(0)); + TestHelpers.assertEqualsSafe(projectedSchema.asStruct(), expected.get(1), actual.get(1)); + } + + @Test + public void testManifestsTable() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "manifests_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + Table manifestTable = loadTable(tableIdentifier, "manifests"); + Dataset df1 = + spark.createDataFrame( + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(null, "b")), + SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .option(SparkWriteOptions.DISTRIBUTION_MODE, TableProperties.WRITE_DISTRIBUTION_MODE_NONE) + .save(loadLocation(tableIdentifier)); + + table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); + + DeleteFile deleteFile = writePosDeleteFile(table); + + table.newRowDelta().addDeletes(deleteFile).commit(); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "manifests")) + .collectAsList(); + + table.refresh(); + + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(manifestTable.schema(), "manifests")); + GenericRecordBuilder summaryBuilder = + new GenericRecordBuilder( + AvroSchemaUtil.convert( + manifestTable.schema().findType("partition_summaries.element").asStructType(), + "partition_summary")); + List expected = + Lists.transform( + table.currentSnapshot().allManifests(table.io()), + manifest -> + builder + .set("content", manifest.content().id()) + .set("path", manifest.path()) + .set("length", manifest.length()) + .set("partition_spec_id", manifest.partitionSpecId()) + .set("added_snapshot_id", manifest.snapshotId()) + .set( + "added_data_files_count", + manifest.content() == DATA ? manifest.addedFilesCount() : 0) + .set( + "existing_data_files_count", + manifest.content() == DATA ? manifest.existingFilesCount() : 0) + .set( + "deleted_data_files_count", + manifest.content() == DATA ? manifest.deletedFilesCount() : 0) + .set( + "added_delete_files_count", + manifest.content() == DELETES ? manifest.addedFilesCount() : 0) + .set( + "existing_delete_files_count", + manifest.content() == DELETES ? manifest.existingFilesCount() : 0) + .set( + "deleted_delete_files_count", + manifest.content() == DELETES ? manifest.deletedFilesCount() : 0) + .set( + "partition_summaries", + Lists.transform( + manifest.partitions(), + partition -> + summaryBuilder + .set("contains_null", manifest.content() == DATA) + .set("contains_nan", false) + .set("lower_bound", "1") + .set("upper_bound", "1") + .build())) + .build()); + + assertThat(actual).as("Manifests table should have two manifest rows").hasSize(2); + TestHelpers.assertEqualsSafe(manifestTable.schema().asStruct(), expected.get(0), actual.get(0)); + TestHelpers.assertEqualsSafe(manifestTable.schema().asStruct(), expected.get(1), actual.get(1)); + } + + @Test + public void testPruneManifestsTable() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "manifests_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + Table manifestTable = loadTable(tableIdentifier, "manifests"); + Dataset df1 = + spark.createDataFrame( + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(null, "b")), + SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + if (!spark.version().startsWith("2")) { + // Spark 2 isn't able to actually push down nested struct projections so this will not break + assertThatThrownBy( + () -> + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "manifests")) + .select("partition_spec_id", "path", "partition_summaries.contains_null") + .collectAsList()) + .isInstanceOf(SparkException.class) + .hasMessageContaining("Cannot project a partial list element struct"); + } + + Dataset actualDf = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "manifests")) + .select("partition_spec_id", "path", "partition_summaries"); + + Schema projectedSchema = SparkSchemaUtil.convert(actualDf.schema()); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "manifests")) + .select("partition_spec_id", "path", "partition_summaries") + .collectAsList(); + + table.refresh(); + + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(projectedSchema.asStruct())); + GenericRecordBuilder summaryBuilder = + new GenericRecordBuilder( + AvroSchemaUtil.convert( + projectedSchema.findType("partition_summaries.element").asStructType(), + "partition_summary")); + List expected = + Lists.transform( + table.currentSnapshot().allManifests(table.io()), + manifest -> + builder + .set("partition_spec_id", manifest.partitionSpecId()) + .set("path", manifest.path()) + .set( + "partition_summaries", + Lists.transform( + manifest.partitions(), + partition -> + summaryBuilder + .set("contains_null", true) + .set("contains_nan", false) + .set("lower_bound", "1") + .set("upper_bound", "1") + .build())) + .build()); + + assertThat(actual).as("Manifests table should have one manifest row").hasSize(1); + TestHelpers.assertEqualsSafe(projectedSchema.asStruct(), expected.get(0), actual.get(0)); + } + + @Test + public void testAllManifestsTable() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "manifests_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + Table manifestTable = loadTable(tableIdentifier, "all_manifests"); + Dataset df1 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); + + DeleteFile deleteFile = writePosDeleteFile(table); + + table.newRowDelta().addDeletes(deleteFile).commit(); + + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + + Stream> snapshotIdToManifests = + StreamSupport.stream(table.snapshots().spliterator(), false) + .flatMap( + snapshot -> + snapshot.allManifests(table.io()).stream() + .map(manifest -> Pair.of(snapshot.snapshotId(), manifest))); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "all_manifests")) + .orderBy("path") + .collectAsList(); + + table.refresh(); + + List expected = + snapshotIdToManifests + .map( + snapshotManifest -> + manifestRecord( + manifestTable, snapshotManifest.first(), snapshotManifest.second())) + .sorted(Comparator.comparing(o -> o.get("path").toString())) + .collect(Collectors.toList()); + + assertThat(actual).as("Manifests table should have 5 manifest rows").hasSize(5); + for (int i = 0; i < expected.size(); i += 1) { + TestHelpers.assertEqualsSafe( + manifestTable.schema().asStruct(), expected.get(i), actual.get(i)); + } + } + + @Test + public void testUnpartitionedPartitionsTable() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "unpartitioned_partitions_test"); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + + Dataset df = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + Types.StructType expectedSchema = + Types.StructType.of( + required(2, "record_count", Types.LongType.get(), "Count of records in data files"), + required(3, "file_count", Types.IntegerType.get(), "Count of data files"), + required( + 11, + "total_data_file_size_in_bytes", + Types.LongType.get(), + "Total size in bytes of data files"), + required( + 5, + "position_delete_record_count", + Types.LongType.get(), + "Count of records in position delete files"), + required( + 6, + "position_delete_file_count", + Types.IntegerType.get(), + "Count of position delete files"), + required( + 7, + "equality_delete_record_count", + Types.LongType.get(), + "Count of records in equality delete files"), + required( + 8, + "equality_delete_file_count", + Types.IntegerType.get(), + "Count of equality delete files"), + optional( + 9, + "last_updated_at", + Types.TimestampType.withZone(), + "Commit time of snapshot that last updated this partition"), + optional( + 10, + "last_updated_snapshot_id", + Types.LongType.get(), + "Id of snapshot that last updated this partition")); + + Table partitionsTable = loadTable(tableIdentifier, "partitions"); + + assertThat(expectedSchema) + .as("Schema should not have partition field") + .isEqualTo(partitionsTable.schema().asStruct()); + + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); + GenericData.Record expectedRow = + builder + .set("last_updated_at", table.currentSnapshot().timestampMillis() * 1000) + .set("last_updated_snapshot_id", table.currentSnapshot().snapshotId()) + .set("record_count", 1L) + .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.currentSnapshot().addedDataFiles(table.io()))) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .build(); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .collectAsList(); + + assertThat(actual).as("Unpartitioned partitions table should have one row").hasSize(1); + TestHelpers.assertEqualsSafe(expectedSchema, expectedRow, actual.get(0)); + } + + @Test + public void testPartitionsTable() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "partitions_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + Table partitionsTable = loadTable(tableIdentifier, "partitions"); + Dataset df1 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); + Dataset df2 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(2, "b")), SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long firstCommitId = table.currentSnapshot().snapshotId(); + + // add a second file + df2.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long secondCommitId = table.currentSnapshot().snapshotId(); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .orderBy("partition.id") + .collectAsList(); + + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); + GenericRecordBuilder partitionBuilder = + new GenericRecordBuilder( + AvroSchemaUtil.convert( + partitionsTable.schema().findType("partition").asStructType(), "partition")); + List expected = Lists.newArrayList(); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 1).build()) + .set("record_count", 1L) + .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) + .build()); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 2).build()) + .set("record_count", 1L) + .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(secondCommitId).addedDataFiles(table.io()))) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(secondCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", secondCommitId) + .build()); + + assertThat(expected).as("Partitions table should have two rows").hasSize(2); + assertThat(actual).as("Actual results should have two rows").hasSize(2); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); + } + + // check time travel + List actualAfterFirstCommit = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SNAPSHOT_ID, String.valueOf(firstCommitId)) + .load(loadLocation(tableIdentifier, "partitions")) + .orderBy("partition.id") + .collectAsList(); + + assertThat(actualAfterFirstCommit).as("Actual results should have one row").hasSize(1); + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(0), actualAfterFirstCommit.get(0)); + + // check predicate push down + List filtered = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .filter("partition.id < 2") + .collectAsList(); + + assertThat(filtered).as("Actual results should have one row").hasSize(1); + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(0), filtered.get(0)); + + List nonFiltered = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .filter("partition.id < 2 or record_count=1") + .collectAsList(); + + assertThat(nonFiltered).as("Actual results should have two rows").hasSize(2); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); + } + } + + @Test + public void testPartitionsTableLastUpdatedSnapshot() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "partitions_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + Table partitionsTable = loadTable(tableIdentifier, "partitions"); + Dataset df1 = + spark.createDataFrame( + Lists.newArrayList(new SimpleRecord(1, "1"), new SimpleRecord(2, "2")), + SimpleRecord.class); + Dataset df2 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(2, "20")), SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long firstCommitId = table.currentSnapshot().snapshotId(); + + // add a second file + df2.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long secondCommitId = table.currentSnapshot().snapshotId(); + + // check if rewrite manifest does not override metadata about data file's creating snapshot + RewriteManifests.Result rewriteManifestResult = + SparkActions.get().rewriteManifests(table).execute(); + assertThat(rewriteManifestResult.rewrittenManifests()) + .as("rewrite replaced 2 manifests") + .hasSize(2); + assertThat(rewriteManifestResult.addedManifests()).as("rewrite added 1 manifests").hasSize(1); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .orderBy("partition.id") + .collectAsList(); + + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); + GenericRecordBuilder partitionBuilder = + new GenericRecordBuilder( + AvroSchemaUtil.convert( + partitionsTable.schema().findType("partition").asStructType(), "partition")); + + List dataFiles = TestHelpers.dataFiles(table); + assertDataFilePartitions(dataFiles, Arrays.asList(1, 2, 2)); + + List expected = Lists.newArrayList(); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 1).build()) + .set("record_count", 1L) + .set("file_count", 1) + .set("total_data_file_size_in_bytes", dataFiles.get(0).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) + .build()); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 2).build()) + .set("record_count", 2L) + .set("file_count", 2) + .set( + "total_data_file_size_in_bytes", + dataFiles.get(1).fileSizeInBytes() + dataFiles.get(2).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(secondCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", secondCommitId) + .build()); + + assertThat(expected).as("Partitions table should have two rows").hasSize(2); + assertThat(actual).as("Actual results should have two rows").hasSize(2); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); + } + + // check predicate push down + List filtered = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .filter("partition.id < 2") + .collectAsList(); + assertThat(filtered).as("Actual results should have one row").hasSize(1); + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(0), filtered.get(0)); + + // check for snapshot expiration + // if snapshot with firstCommitId is expired, + // we expect the partition of id=1 will no longer have last updated timestamp and snapshotId + SparkActions.get().expireSnapshots(table).expireSnapshotId(firstCommitId).execute(); + GenericData.Record newPartitionRecord = + builder + .set("partition", partitionBuilder.set("id", 1).build()) + .set("record_count", 1L) + .set("file_count", 1) + .set("total_data_file_size_in_bytes", dataFiles.get(0).fileSizeInBytes()) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", null) + .set("last_updated_snapshot_id", null) + .build(); + expected.remove(0); + expected.add(0, newPartitionRecord); + + List actualAfterSnapshotExpiration = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .collectAsList(); + assertThat(actualAfterSnapshotExpiration).as("Actual results should have two rows").hasSize(2); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), + expected.get(i), + actualAfterSnapshotExpiration.get(i)); + } + } + + @Test + public void testPartitionsTableDeleteStats() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "partitions_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + Table partitionsTable = loadTable(tableIdentifier, "partitions"); + Dataset df1 = + spark.createDataFrame( + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(1, "b"), new SimpleRecord(1, "c")), + SimpleRecord.class); + Dataset df2 = + spark.createDataFrame( + Lists.newArrayList( + new SimpleRecord(2, "d"), new SimpleRecord(2, "e"), new SimpleRecord(2, "f")), + SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + long firstCommitId = table.currentSnapshot().snapshotId(); + + // add a second file + df2.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + // test position deletes + table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); + DeleteFile deleteFile1 = writePosDeleteFile(table, 0); + DeleteFile deleteFile2 = writePosDeleteFile(table, 1); + table.newRowDelta().addDeletes(deleteFile1).addDeletes(deleteFile2).commit(); + table.refresh(); + long posDeleteCommitId = table.currentSnapshot().snapshotId(); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .orderBy("partition.id") + .collectAsList(); + assertThat(actual).as("Actual results should have two rows").hasSize(2); + + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(partitionsTable.schema(), "partitions")); + GenericRecordBuilder partitionBuilder = + new GenericRecordBuilder( + AvroSchemaUtil.convert( + partitionsTable.schema().findType("partition").asStructType(), "partition")); + List expected = Lists.newArrayList(); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 1).build()) + .set("record_count", 3L) + .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(firstCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", firstCommitId) + .build()); + expected.add( + builder + .set("partition", partitionBuilder.set("id", 2).build()) + .set("record_count", 3L) + .set("file_count", 1) + .set( + "total_data_file_size_in_bytes", + totalSizeInBytes(table.snapshot(firstCommitId).addedDataFiles(table.io()))) + .set("position_delete_record_count", 2L) // should be incremented now + .set("position_delete_file_count", 2) // should be incremented now + .set("equality_delete_record_count", 0L) + .set("equality_delete_file_count", 0) + .set("spec_id", 0) + .set("last_updated_at", table.snapshot(posDeleteCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", posDeleteCommitId) + .build()); + + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); + } + + // test equality delete + DeleteFile eqDeleteFile1 = writeEqDeleteFile(table, "d"); + DeleteFile eqDeleteFile2 = writeEqDeleteFile(table, "f"); + table.newRowDelta().addDeletes(eqDeleteFile1).addDeletes(eqDeleteFile2).commit(); + table.refresh(); + long eqDeleteCommitId = table.currentSnapshot().snapshotId(); + actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .orderBy("partition.id") + .collectAsList(); + assertThat(actual).as("Actual results should have two rows").hasSize(2); + expected.remove(0); + expected.add( + 0, + builder + .set("partition", partitionBuilder.set("id", 1).build()) + .set("record_count", 3L) + .set("file_count", 1) + .set("position_delete_record_count", 0L) + .set("position_delete_file_count", 0) + .set("equality_delete_record_count", 2L) // should be incremented now + .set("equality_delete_file_count", 2) // should be incremented now + .set("last_updated_at", table.snapshot(eqDeleteCommitId).timestampMillis() * 1000) + .set("last_updated_snapshot_id", eqDeleteCommitId) + .build()); + for (int i = 0; i < 2; i += 1) { + TestHelpers.assertEqualsSafe( + partitionsTable.schema().asStruct(), expected.get(i), actual.get(i)); + } + } + + @Test + public synchronized void testSnapshotReadAfterAddColumn() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "table"); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + + List originalRecords = + Lists.newArrayList( + RowFactory.create(1, "x"), RowFactory.create(2, "y"), RowFactory.create(3, "z")); + + StructType originalSparkSchema = SparkSchemaUtil.convert(SCHEMA); + Dataset inputDf = spark.createDataFrame(originalRecords, originalSparkSchema); + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + + table.refresh(); + + Dataset resultDf = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); + assertThat(resultDf.orderBy("id").collectAsList()) + .as("Records should match") + .containsExactlyElementsOf(originalRecords); + + Snapshot snapshotBeforeAddColumn = table.currentSnapshot(); + + table.updateSchema().addColumn("category", Types.StringType.get()).commit(); + + List newRecords = + Lists.newArrayList(RowFactory.create(4, "xy", "B"), RowFactory.create(5, "xyz", "C")); + + StructType newSparkSchema = SparkSchemaUtil.convert(SCHEMA2); + Dataset inputDf2 = spark.createDataFrame(newRecords, newSparkSchema); + inputDf2 + .select("id", "data", "category") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + + table.refresh(); + + List updatedRecords = + Lists.newArrayList( + RowFactory.create(1, "x", null), + RowFactory.create(2, "y", null), + RowFactory.create(3, "z", null), + RowFactory.create(4, "xy", "B"), + RowFactory.create(5, "xyz", "C")); + + Dataset resultDf2 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); + assertThat(resultDf2.orderBy("id").collectAsList()) + .as("Records should match") + .containsExactlyElementsOf(updatedRecords); + + Dataset resultDf3 = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) + .load(loadLocation(tableIdentifier)); + assertThat(resultDf3.orderBy("id").collectAsList()) + .as("Records should match") + .containsExactlyElementsOf(originalRecords); + assertThat(resultDf3.schema()).as("Schemas should match").isEqualTo(originalSparkSchema); + } + + @Test + public synchronized void testSnapshotReadAfterDropColumn() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "table"); + Table table = createTable(tableIdentifier, SCHEMA2, PartitionSpec.unpartitioned()); + + List originalRecords = + Lists.newArrayList( + RowFactory.create(1, "x", "A"), + RowFactory.create(2, "y", "A"), + RowFactory.create(3, "z", "B")); + + StructType originalSparkSchema = SparkSchemaUtil.convert(SCHEMA2); + Dataset inputDf = spark.createDataFrame(originalRecords, originalSparkSchema); + inputDf + .select("id", "data", "category") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + + table.refresh(); + + Dataset resultDf = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); + assertThat(resultDf.orderBy("id").collectAsList()) + .as("Records should match") + .containsExactlyElementsOf(originalRecords); + + long tsBeforeDropColumn = waitUntilAfter(System.currentTimeMillis()); + table.updateSchema().deleteColumn("data").commit(); + long tsAfterDropColumn = waitUntilAfter(System.currentTimeMillis()); + + List newRecords = Lists.newArrayList(RowFactory.create(4, "B"), RowFactory.create(5, "C")); + + StructType newSparkSchema = SparkSchemaUtil.convert(SCHEMA3); + Dataset inputDf2 = spark.createDataFrame(newRecords, newSparkSchema); + inputDf2 + .select("id", "category") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + + table.refresh(); + + List updatedRecords = + Lists.newArrayList( + RowFactory.create(1, "A"), + RowFactory.create(2, "A"), + RowFactory.create(3, "B"), + RowFactory.create(4, "B"), + RowFactory.create(5, "C")); + + Dataset resultDf2 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); + assertThat(resultDf2.orderBy("id").collectAsList()) + .as("Records should match") + .containsExactlyElementsOf(updatedRecords); + + Dataset resultDf3 = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.AS_OF_TIMESTAMP, tsBeforeDropColumn) + .load(loadLocation(tableIdentifier)); + assertThat(resultDf3.orderBy("id").collectAsList()) + .as("Records should match") + .containsExactlyElementsOf(originalRecords); + assertThat(resultDf3.schema()).as("Schemas should match").isEqualTo(originalSparkSchema); + + // At tsAfterDropColumn, there has been a schema change, but no new snapshot, + // so the snapshot as of tsAfterDropColumn is the same as that as of tsBeforeDropColumn. + Dataset resultDf4 = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.AS_OF_TIMESTAMP, tsAfterDropColumn) + .load(loadLocation(tableIdentifier)); + assertThat(resultDf4.orderBy("id").collectAsList()) + .as("Records should match") + .containsExactlyElementsOf(originalRecords); + assertThat(resultDf4.schema()).as("Schemas should match").isEqualTo(originalSparkSchema); + } + + @Test + public synchronized void testSnapshotReadAfterAddAndDropColumn() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "table"); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + + List originalRecords = + Lists.newArrayList( + RowFactory.create(1, "x"), RowFactory.create(2, "y"), RowFactory.create(3, "z")); + + StructType originalSparkSchema = SparkSchemaUtil.convert(SCHEMA); + Dataset inputDf = spark.createDataFrame(originalRecords, originalSparkSchema); + inputDf + .select("id", "data") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + + table.refresh(); + + Dataset resultDf = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); + assertThat(resultDf.orderBy("id").collectAsList()) + .as("Records should match") + .containsExactlyElementsOf(originalRecords); + + Snapshot snapshotBeforeAddColumn = table.currentSnapshot(); + + table.updateSchema().addColumn("category", Types.StringType.get()).commit(); + + List newRecords = + Lists.newArrayList(RowFactory.create(4, "xy", "B"), RowFactory.create(5, "xyz", "C")); + + StructType sparkSchemaAfterAddColumn = SparkSchemaUtil.convert(SCHEMA2); + Dataset inputDf2 = spark.createDataFrame(newRecords, sparkSchemaAfterAddColumn); + inputDf2 + .select("id", "data", "category") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + + table.refresh(); + + List updatedRecords = + Lists.newArrayList( + RowFactory.create(1, "x", null), + RowFactory.create(2, "y", null), + RowFactory.create(3, "z", null), + RowFactory.create(4, "xy", "B"), + RowFactory.create(5, "xyz", "C")); + + Dataset resultDf2 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); + assertThat(resultDf2.orderBy("id").collectAsList()) + .as("Records should match") + .containsExactlyElementsOf(updatedRecords); + + table.updateSchema().deleteColumn("data").commit(); + + List recordsAfterDropColumn = + Lists.newArrayList( + RowFactory.create(1, null), + RowFactory.create(2, null), + RowFactory.create(3, null), + RowFactory.create(4, "B"), + RowFactory.create(5, "C")); + + Dataset resultDf3 = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); + assertThat(resultDf3.orderBy("id").collectAsList()) + .as("Records should match") + .containsExactlyElementsOf(recordsAfterDropColumn); + + Dataset resultDf4 = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SNAPSHOT_ID, snapshotBeforeAddColumn.snapshotId()) + .load(loadLocation(tableIdentifier)); + assertThat(resultDf4.orderBy("id").collectAsList()) + .as("Records should match") + .containsExactlyElementsOf(originalRecords); + assertThat(resultDf4.schema()).as("Schemas should match").isEqualTo(originalSparkSchema); + } + + @Test + public void testRemoveOrphanFilesActionSupport() throws InterruptedException { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "table"); + Table table = createTable(tableIdentifier, SCHEMA, PartitionSpec.unpartitioned()); + + List records = Lists.newArrayList(new SimpleRecord(1, "1")); + + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + df.write().mode("append").parquet(table.location() + "/data"); + + // sleep for 1 second to ensure files will be old enough + Thread.sleep(1000); + + SparkActions actions = SparkActions.get(); + + DeleteOrphanFiles.Result result1 = + actions + .deleteOrphanFiles(table) + .location(table.location() + "/metadata") + .olderThan(System.currentTimeMillis()) + .execute(); + assertThat(result1.orphanFileLocations()).as("Should not delete any metadata files").isEmpty(); + + DeleteOrphanFiles.Result result2 = + actions.deleteOrphanFiles(table).olderThan(System.currentTimeMillis()).execute(); + assertThat(result2.orphanFileLocations()).as("Should delete 1 data file").hasSize(1); + + Dataset resultDF = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); + List actualRecords = + resultDF.as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actualRecords).as("Rows must match").containsExactlyInAnyOrderElementsOf(records); + } + + @Test + public void testFilesTablePartitionId() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "files_test"); + Table table = + createTable( + tableIdentifier, SCHEMA, PartitionSpec.builderFor(SCHEMA).identity("id").build()); + int spec0 = table.spec().specId(); + + Dataset df1 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); + Dataset df2 = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(2, "b")), SimpleRecord.class); + + df1.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + // change partition spec + table.refresh(); + table.updateSpec().removeField("id").commit(); + int spec1 = table.spec().specId(); + + // add a second file + df2.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "files")) + .sort(DataFile.SPEC_ID.name()) + .collectAsList() + .stream() + .map(r -> (Integer) r.getAs(DataFile.SPEC_ID.name())) + .collect(Collectors.toList()); + + assertThat(actual).as("Should have two partition specs").containsExactly(spec0, spec1); + } + + @Test + public void testAllManifestTableSnapshotFiltering() { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "all_manifest_snapshot_filtering"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + Table manifestTable = loadTable(tableIdentifier, "all_manifests"); + Dataset df = + spark.createDataFrame(Lists.newArrayList(new SimpleRecord(1, "a")), SimpleRecord.class); + + List> snapshotIdToManifests = Lists.newArrayList(); + + df.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + Snapshot snapshot1 = table.currentSnapshot(); + snapshotIdToManifests.addAll( + snapshot1.allManifests(table.io()).stream() + .map(manifest -> Pair.of(snapshot1.snapshotId(), manifest)) + .collect(Collectors.toList())); + + df.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + table.refresh(); + Snapshot snapshot2 = table.currentSnapshot(); + assertThat(snapshot2.allManifests(table.io())).as("Should have two manifests").hasSize(2); + snapshotIdToManifests.addAll( + snapshot2.allManifests(table.io()).stream() + .map(manifest -> Pair.of(snapshot2.snapshotId(), manifest)) + .collect(Collectors.toList())); + + // Add manifests that will not be selected + df.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + df.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(loadLocation(tableIdentifier)); + + StringJoiner snapshotIds = new StringJoiner(",", "(", ")"); + snapshotIds.add(String.valueOf(snapshot1.snapshotId())); + snapshotIds.add(String.valueOf(snapshot2.snapshotId())); + snapshotIds.toString(); + + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "all_manifests")) + .filter("reference_snapshot_id in " + snapshotIds) + .orderBy("path") + .collectAsList(); + table.refresh(); + + List expected = + snapshotIdToManifests.stream() + .map( + snapshotManifest -> + manifestRecord( + manifestTable, snapshotManifest.first(), snapshotManifest.second())) + .sorted(Comparator.comparing(o -> o.get("path").toString())) + .collect(Collectors.toList()); + + assertThat(actual).as("Manifests table should have 3 manifest rows").hasSize(3); + for (int i = 0; i < expected.size(); i += 1) { + TestHelpers.assertEqualsSafe( + manifestTable.schema().asStruct(), expected.get(i), actual.get(i)); + } + } + + @Test + public void testTableWithInt96Timestamp() throws IOException { + File parquetTableDir = temp.resolve("table_timestamp_int96").toFile(); + String parquetTableLocation = parquetTableDir.toURI().toString(); + Schema schema = + new Schema( + optional(1, "id", Types.LongType.get()), + optional(2, "tmp_col", Types.TimestampType.withZone())); + spark.conf().set(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE().key(), "INT96"); + + LocalDateTime start = LocalDateTime.of(2000, 1, 31, 0, 0, 0); + LocalDateTime end = LocalDateTime.of(2100, 1, 1, 0, 0, 0); + long startSec = start.toEpochSecond(ZoneOffset.UTC); + long endSec = end.toEpochSecond(ZoneOffset.UTC); + Column idColumn = functions.expr("id"); + Column secondsColumn = + functions.expr("(id % " + (endSec - startSec) + " + " + startSec + ")").as("seconds"); + Column timestampColumn = functions.expr("cast( seconds as timestamp) as tmp_col"); + + for (Boolean useDict : new Boolean[] {true, false}) { + for (Boolean useVectorization : new Boolean[] {true, false}) { + spark.sql("DROP TABLE IF EXISTS parquet_table"); + spark + .range(0, 5000, 100, 1) + .select(idColumn, secondsColumn) + .select(idColumn, timestampColumn) + .write() + .format("parquet") + .option("parquet.enable.dictionary", useDict) + .mode("overwrite") + .option("path", parquetTableLocation) + .saveAsTable("parquet_table"); + TableIdentifier tableIdentifier = TableIdentifier.of("db", "table_with_timestamp_int96"); + Table table = createTable(tableIdentifier, schema, PartitionSpec.unpartitioned()); + table + .updateProperties() + .set(TableProperties.PARQUET_VECTORIZATION_ENABLED, useVectorization.toString()) + .commit(); + + String stagingLocation = table.location() + "/metadata"; + SparkTableUtil.importSparkTable( + spark, + new org.apache.spark.sql.catalyst.TableIdentifier("parquet_table"), + table, + stagingLocation); + + // validate we get the expected results back + testWithFilter("tmp_col < to_timestamp('2000-01-31 08:30:00')", tableIdentifier); + testWithFilter("tmp_col <= to_timestamp('2000-01-31 08:30:00')", tableIdentifier); + testWithFilter("tmp_col == to_timestamp('2000-01-31 08:30:00')", tableIdentifier); + testWithFilter("tmp_col > to_timestamp('2000-01-31 08:30:00')", tableIdentifier); + testWithFilter("tmp_col >= to_timestamp('2000-01-31 08:30:00')", tableIdentifier); + dropTable(tableIdentifier); + } + } + } + + @Test + public void testImportSparkTableWithMissingFilesFailure() throws IOException { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "missing_files_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + + File parquetTableDir = temp.resolve("table_missing_files").toFile(); + String parquetTableLocation = parquetTableDir.toURI().toString(); + spark.sql( + String.format( + "CREATE TABLE parquet_table (data string, id int) " + + "USING parquet PARTITIONED BY (id) LOCATION '%s'", + parquetTableLocation)); + + List records = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + + Dataset inputDF = spark.createDataFrame(records, SimpleRecord.class); + inputDF.write().mode("overwrite").insertInto("parquet_table"); + + // Add a Spark partition of which location is missing + spark.sql("ALTER TABLE parquet_table ADD PARTITION (id = 1234)"); + Path partitionLocationPath = parquetTableDir.toPath().resolve("id=1234"); + java.nio.file.Files.delete(partitionLocationPath); + + String stagingLocation = table.location() + "/metadata"; + + assertThatThrownBy( + () -> + SparkTableUtil.importSparkTable( + spark, + new org.apache.spark.sql.catalyst.TableIdentifier("parquet_table"), + table, + stagingLocation)) + .hasMessageContaining( + "Unable to list files in partition: " + partitionLocationPath.toFile().toURI()) + .isInstanceOf(SparkException.class) + .hasRootCauseInstanceOf(FileNotFoundException.class); + } + + @Test + public void testImportSparkTableWithIgnoreMissingFilesEnabled() throws IOException { + TableIdentifier tableIdentifier = TableIdentifier.of("db", "missing_files_test"); + Table table = createTable(tableIdentifier, SCHEMA, SPEC); + + File parquetTableDir = temp.resolve("table_missing_files").toFile(); + String parquetTableLocation = parquetTableDir.toURI().toString(); + spark.sql( + String.format( + "CREATE TABLE parquet_table (data string, id int) " + + "USING parquet PARTITIONED BY (id) LOCATION '%s'", + parquetTableLocation)); + + List records = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + + Dataset inputDF = spark.createDataFrame(records, SimpleRecord.class); + inputDF.write().mode("overwrite").insertInto("parquet_table"); + + // Add a Spark partition of which location is missing + spark.sql("ALTER TABLE parquet_table ADD PARTITION (id = 1234)"); + Path partitionLocationPath = parquetTableDir.toPath().resolve("id=1234"); + java.nio.file.Files.delete(partitionLocationPath); + + String stagingLocation = table.location() + "/metadata"; + + SparkTableUtil.importSparkTable( + spark, + new org.apache.spark.sql.catalyst.TableIdentifier("parquet_table"), + table, + stagingLocation, + Collections.emptyMap(), + false, + true, + SparkTableUtil.migrationService(1)); + + List partitionsTableRows = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "partitions")) + .collectAsList(); + assertThat(partitionsTableRows).as("Partitions table should have 2 rows").hasSize(2); + } + + private void testWithFilter(String filterExpr, TableIdentifier tableIdentifier) { + List expected = + spark.table("parquet_table").select("tmp_col").filter(filterExpr).collectAsList(); + List actual = + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier)) + .select("tmp_col") + .filter(filterExpr) + .collectAsList(); + assertThat(actual).as("Rows must match").containsExactlyInAnyOrderElementsOf(expected); + } + + @Test + public void testSessionConfigSupport() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("id").build(); + TableIdentifier tableIdentifier = TableIdentifier.of("db", "session_config_table"); + Table table = createTable(tableIdentifier, SCHEMA, spec); + + List initialRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + Dataset df = spark.createDataFrame(initialRecords, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + + long s1 = table.currentSnapshot().snapshotId(); + + withSQLConf( + // set write option through session configuration + ImmutableMap.of("spark.datasource.iceberg.snapshot-property.foo", "bar"), + () -> { + df.select("id", "data") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(loadLocation(tableIdentifier)); + }); + + table.refresh(); + assertThat(table.currentSnapshot().summary()).containsEntry("foo", "bar"); + + withSQLConf( + // set read option through session configuration + ImmutableMap.of("spark.datasource.iceberg.snapshot-id", String.valueOf(s1)), + () -> { + Dataset result = spark.read().format("iceberg").load(loadLocation(tableIdentifier)); + List actual = result.as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual) + .as("Rows must match") + .containsExactlyInAnyOrderElementsOf(initialRecords); + }); + } + + private GenericData.Record manifestRecord( + Table manifestTable, Long referenceSnapshotId, ManifestFile manifest) { + GenericRecordBuilder builder = + new GenericRecordBuilder(AvroSchemaUtil.convert(manifestTable.schema(), "manifests")); + GenericRecordBuilder summaryBuilder = + new GenericRecordBuilder( + AvroSchemaUtil.convert( + manifestTable.schema().findType("partition_summaries.element").asStructType(), + "partition_summary")); + return builder + .set("content", manifest.content().id()) + .set("path", manifest.path()) + .set("length", manifest.length()) + .set("partition_spec_id", manifest.partitionSpecId()) + .set("added_snapshot_id", manifest.snapshotId()) + .set("added_data_files_count", manifest.content() == DATA ? manifest.addedFilesCount() : 0) + .set( + "existing_data_files_count", + manifest.content() == DATA ? manifest.existingFilesCount() : 0) + .set( + "deleted_data_files_count", + manifest.content() == DATA ? manifest.deletedFilesCount() : 0) + .set( + "added_delete_files_count", + manifest.content() == DELETES ? manifest.addedFilesCount() : 0) + .set( + "existing_delete_files_count", + manifest.content() == DELETES ? manifest.existingFilesCount() : 0) + .set( + "deleted_delete_files_count", + manifest.content() == DELETES ? manifest.deletedFilesCount() : 0) + .set( + "partition_summaries", + Lists.transform( + manifest.partitions(), + partition -> + summaryBuilder + .set("contains_null", false) + .set("contains_nan", false) + .set("lower_bound", "1") + .set("upper_bound", "1") + .build())) + .set("reference_snapshot_id", referenceSnapshotId) + .build(); + } + + private PositionDeleteWriter newPositionDeleteWriter( + Table table, PartitionSpec spec, StructLike partition) { + OutputFileFactory fileFactory = OutputFileFactory.builderFor(table, 0, 0).build(); + EncryptedOutputFile outputFile = fileFactory.newOutputFile(spec, partition); + + SparkFileWriterFactory fileWriterFactory = SparkFileWriterFactory.builderFor(table).build(); + return fileWriterFactory.newPositionDeleteWriter(outputFile, spec, partition); + } + + private DeleteFile writePositionDeletes( + Table table, + PartitionSpec spec, + StructLike partition, + Iterable> deletes) { + PositionDeleteWriter positionDeleteWriter = + newPositionDeleteWriter(table, spec, partition); + + try (PositionDeleteWriter writer = positionDeleteWriter) { + for (PositionDelete delete : deletes) { + writer.write(delete); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + return positionDeleteWriter.toDeleteFile(); + } + + private DeleteFile writePosDeleteFile(Table table) { + return writePosDeleteFile(table, 0L); + } + + private DeleteFile writePosDeleteFile(Table table, long pos) { + DataFile dataFile = + Iterables.getFirst(table.currentSnapshot().addedDataFiles(table.io()), null); + PartitionSpec dataFileSpec = table.specs().get(dataFile.specId()); + StructLike dataFilePartition = dataFile.partition(); + + PositionDelete delete = PositionDelete.create(); + delete.set(dataFile.location(), pos, null); + + return writePositionDeletes(table, dataFileSpec, dataFilePartition, ImmutableList.of(delete)); + } + + private DeleteFile writeEqDeleteFile(Table table, String dataValue) { + List deletes = Lists.newArrayList(); + Schema deleteRowSchema = SCHEMA.select("data"); + Record delete = GenericRecord.create(deleteRowSchema); + deletes.add(delete.copy("data", dataValue)); + try { + return FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + org.apache.iceberg.TestHelpers.Row.of(1), + deletes, + deleteRowSchema); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private long totalSizeInBytes(Iterable dataFiles) { + return Lists.newArrayList(dataFiles).stream().mapToLong(DataFile::fileSizeInBytes).sum(); + } + + private void assertDataFilePartitions( + List dataFiles, List expectedPartitionIds) { + assertThat(dataFiles) + .as("Table should have " + expectedPartitionIds.size() + " data files") + .hasSameSizeAs(expectedPartitionIds); + + for (int i = 0; i < dataFiles.size(); ++i) { + assertThat(dataFiles.get(i).partition().get(0, Integer.class).intValue()) + .as("Data file should have partition of id " + expectedPartitionIds.get(i)) + .isEqualTo(expectedPartitionIds.get(i).intValue()); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java new file mode 100644 index 000000000000..f4f57157e479 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java @@ -0,0 +1,295 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.math.BigDecimal; +import java.net.InetAddress; +import java.nio.ByteBuffer; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.List; +import org.apache.iceberg.spark.IcebergSpark; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.apache.spark.sql.types.CharType; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.DecimalType; +import org.apache.spark.sql.types.VarcharType; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +public class TestIcebergSpark { + + private static SparkSession spark = null; + + @BeforeAll + public static void startSpark() { + TestIcebergSpark.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestIcebergSpark.spark; + TestIcebergSpark.spark = null; + currentSpark.stop(); + } + + @Test + public void testRegisterIntegerBucketUDF() { + IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_int_16", DataTypes.IntegerType, 16); + List results = spark.sql("SELECT iceberg_bucket_int_16(1)").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getInt(0)) + .isEqualTo(Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1))); + } + + @Test + public void testRegisterShortBucketUDF() { + IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_short_16", DataTypes.ShortType, 16); + List results = spark.sql("SELECT iceberg_bucket_short_16(1S)").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getInt(0)) + .isEqualTo(Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1))); + } + + @Test + public void testRegisterByteBucketUDF() { + IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_byte_16", DataTypes.ByteType, 16); + List results = spark.sql("SELECT iceberg_bucket_byte_16(1Y)").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getInt(0)) + .isEqualTo(Transforms.bucket(16).bind(Types.IntegerType.get()).apply(1))); + } + + @Test + public void testRegisterLongBucketUDF() { + IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_long_16", DataTypes.LongType, 16); + List results = spark.sql("SELECT iceberg_bucket_long_16(1L)").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getInt(0)) + .isEqualTo(Transforms.bucket(16).bind(Types.LongType.get()).apply(1L))); + } + + @Test + public void testRegisterStringBucketUDF() { + IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_string_16", DataTypes.StringType, 16); + List results = spark.sql("SELECT iceberg_bucket_string_16('hello')").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getInt(0)) + .isEqualTo(Transforms.bucket(16).bind(Types.StringType.get()).apply("hello"))); + } + + @Test + public void testRegisterCharBucketUDF() { + IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_char_16", new CharType(5), 16); + List results = spark.sql("SELECT iceberg_bucket_char_16('hello')").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getInt(0)) + .isEqualTo(Transforms.bucket(16).bind(Types.StringType.get()).apply("hello"))); + } + + @Test + public void testRegisterVarCharBucketUDF() { + IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_varchar_16", new VarcharType(5), 16); + List results = spark.sql("SELECT iceberg_bucket_varchar_16('hello')").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getInt(0)) + .isEqualTo(Transforms.bucket(16).bind(Types.StringType.get()).apply("hello"))); + } + + @Test + public void testRegisterDateBucketUDF() { + IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_date_16", DataTypes.DateType, 16); + List results = + spark.sql("SELECT iceberg_bucket_date_16(DATE '2021-06-30')").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getInt(0)) + .isEqualTo( + Transforms.bucket(16) + .bind(Types.DateType.get()) + .apply(DateTimeUtils.fromJavaDate(Date.valueOf("2021-06-30"))))); + } + + @Test + public void testRegisterTimestampBucketUDF() { + IcebergSpark.registerBucketUDF( + spark, "iceberg_bucket_timestamp_16", DataTypes.TimestampType, 16); + List results = + spark + .sql("SELECT iceberg_bucket_timestamp_16(TIMESTAMP '2021-06-30 00:00:00.000')") + .collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getInt(0)) + .isEqualTo( + Transforms.bucket(16) + .bind(Types.TimestampType.withZone()) + .apply( + DateTimeUtils.fromJavaTimestamp( + Timestamp.valueOf("2021-06-30 00:00:00.000"))))); + } + + @Test + public void testRegisterBinaryBucketUDF() { + IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_binary_16", DataTypes.BinaryType, 16); + List results = spark.sql("SELECT iceberg_bucket_binary_16(X'0020001F')").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getInt(0)) + .isEqualTo( + Transforms.bucket(16) + .bind(Types.BinaryType.get()) + .apply(ByteBuffer.wrap(new byte[] {0x00, 0x20, 0x00, 0x1F})))); + } + + @Test + public void testRegisterDecimalBucketUDF() { + IcebergSpark.registerBucketUDF(spark, "iceberg_bucket_decimal_16", new DecimalType(4, 2), 16); + List results = spark.sql("SELECT iceberg_bucket_decimal_16(11.11)").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getInt(0)) + .isEqualTo( + Transforms.bucket(16) + .bind(Types.DecimalType.of(4, 2)) + .apply(new BigDecimal("11.11")))); + } + + @Test + public void testRegisterBooleanBucketUDF() { + assertThatThrownBy( + () -> + IcebergSpark.registerBucketUDF( + spark, "iceberg_bucket_boolean_16", DataTypes.BooleanType, 16)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot bucket by type: boolean"); + } + + @Test + public void testRegisterDoubleBucketUDF() { + assertThatThrownBy( + () -> + IcebergSpark.registerBucketUDF( + spark, "iceberg_bucket_double_16", DataTypes.DoubleType, 16)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot bucket by type: double"); + } + + @Test + public void testRegisterFloatBucketUDF() { + assertThatThrownBy( + () -> + IcebergSpark.registerBucketUDF( + spark, "iceberg_bucket_float_16", DataTypes.FloatType, 16)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot bucket by type: float"); + } + + @Test + public void testRegisterIntegerTruncateUDF() { + IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_int_4", DataTypes.IntegerType, 4); + List results = spark.sql("SELECT iceberg_truncate_int_4(1)").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getInt(0)) + .isEqualTo(Transforms.truncate(4).bind(Types.IntegerType.get()).apply(1))); + } + + @Test + public void testRegisterLongTruncateUDF() { + IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_long_4", DataTypes.LongType, 4); + List results = spark.sql("SELECT iceberg_truncate_long_4(1L)").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getLong(0)) + .isEqualTo(Transforms.truncate(4).bind(Types.LongType.get()).apply(1L))); + } + + @Test + public void testRegisterDecimalTruncateUDF() { + IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_decimal_4", new DecimalType(4, 2), 4); + List results = spark.sql("SELECT iceberg_truncate_decimal_4(11.11)").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getDecimal(0)) + .isEqualTo( + Transforms.truncate(4) + .bind(Types.DecimalType.of(4, 2)) + .apply(new BigDecimal("11.11")))); + } + + @Test + public void testRegisterStringTruncateUDF() { + IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_string_4", DataTypes.StringType, 4); + List results = spark.sql("SELECT iceberg_truncate_string_4('hello')").collectAsList(); + assertThat(results) + .singleElement() + .satisfies( + row -> + assertThat(row.getString(0)) + .isEqualTo(Transforms.truncate(4).bind(Types.StringType.get()).apply("hello"))); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java new file mode 100644 index 000000000000..bf3bcacbfbe9 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java @@ -0,0 +1,251 @@ +/* + * 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.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hadoop.HadoopTables; +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.SparkReadOptions; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkTableUtil; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.TableIdentifier; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIdentityPartitionData extends TestBase { + private static final Configuration CONF = new Configuration(); + private static final HadoopTables TABLES = new HadoopTables(CONF); + + @Parameters(name = "format = {0}, vectorized = {1}, properties = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + FileFormat.PARQUET, + false, + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, "parquet", + TableProperties.DATA_PLANNING_MODE, LOCAL.modeName(), + TableProperties.DELETE_PLANNING_MODE, LOCAL.modeName()) + }, + { + FileFormat.PARQUET, + true, + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, "parquet", + TableProperties.DATA_PLANNING_MODE, DISTRIBUTED.modeName(), + TableProperties.DELETE_PLANNING_MODE, DISTRIBUTED.modeName()) + }, + { + FileFormat.AVRO, + false, + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, "avro", + TableProperties.DATA_PLANNING_MODE, LOCAL.modeName(), + TableProperties.DELETE_PLANNING_MODE, LOCAL.modeName()) + }, + { + FileFormat.ORC, + false, + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, "orc", + TableProperties.DATA_PLANNING_MODE, DISTRIBUTED.modeName(), + TableProperties.DELETE_PLANNING_MODE, DISTRIBUTED.modeName()) + }, + { + FileFormat.ORC, + true, + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, "orc", + TableProperties.DATA_PLANNING_MODE, LOCAL.modeName(), + TableProperties.DELETE_PLANNING_MODE, LOCAL.modeName()) + }, + }; + } + + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private boolean vectorized; + + @Parameter(index = 2) + private Map properties; + + private static final Schema LOG_SCHEMA = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "date", Types.StringType.get()), + Types.NestedField.optional(3, "level", Types.StringType.get()), + Types.NestedField.optional(4, "message", Types.StringType.get())); + + private static final List LOGS = + ImmutableList.of( + LogMessage.debug("2020-02-02", "debug event 1"), + LogMessage.info("2020-02-02", "info event 1"), + LogMessage.debug("2020-02-02", "debug event 2"), + LogMessage.info("2020-02-03", "info event 2"), + LogMessage.debug("2020-02-03", "debug event 3"), + LogMessage.info("2020-02-03", "info event 3"), + LogMessage.error("2020-02-03", "error event 1"), + LogMessage.debug("2020-02-04", "debug event 4"), + LogMessage.warn("2020-02-04", "warn event 1"), + LogMessage.debug("2020-02-04", "debug event 5")); + + @TempDir private Path temp; + + private final PartitionSpec spec = + PartitionSpec.builderFor(LOG_SCHEMA).identity("date").identity("level").build(); + private Table table = null; + private Dataset logs = null; + + /** + * Use the Hive Based table to make Identity Partition Columns with no duplication of the data in + * the underlying parquet files. This makes sure that if the identity mapping fails, the test will + * also fail. + */ + private void setupParquet() throws Exception { + File location = Files.createTempDirectory(temp, "logs").toFile(); + File hiveLocation = Files.createTempDirectory(temp, "hive").toFile(); + String hiveTable = "hivetable"; + assertThat(location).as("Temp folder should exist").exists(); + + this.logs = + spark.createDataFrame(LOGS, LogMessage.class).select("id", "date", "level", "message"); + spark.sql(String.format("DROP TABLE IF EXISTS %s", hiveTable)); + logs.orderBy("date", "level", "id") + .write() + .partitionBy("date", "level") + .format("parquet") + .option("path", hiveLocation.toString()) + .saveAsTable(hiveTable); + + this.table = + TABLES.create( + SparkSchemaUtil.schemaForTable(spark, hiveTable), + SparkSchemaUtil.specForTable(spark, hiveTable), + properties, + location.toString()); + + SparkTableUtil.importSparkTable( + spark, new TableIdentifier(hiveTable), table, location.toString()); + } + + @BeforeEach + public void setupTable() throws Exception { + if (format.equals(FileFormat.PARQUET)) { + setupParquet(); + } else { + File location = Files.createTempDirectory(temp, "logs").toFile(); + assertThat(location).as("Temp folder should exist").exists(); + + this.table = TABLES.create(LOG_SCHEMA, spec, properties, location.toString()); + this.logs = + spark.createDataFrame(LOGS, LogMessage.class).select("id", "date", "level", "message"); + + logs.orderBy("date", "level", "id") + .write() + .format("iceberg") + .mode("append") + .save(location.toString()); + } + } + + @TestTemplate + public void testFullProjection() { + List expected = logs.orderBy("id").collectAsList(); + List actual = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(table.location()) + .orderBy("id") + .select("id", "date", "level", "message") + .collectAsList(); + assertThat(actual).as("Rows should match").isEqualTo(expected); + } + + @TestTemplate + public void testProjections() { + String[][] cases = + new String[][] { + // individual fields + new String[] {"date"}, + new String[] {"level"}, + new String[] {"message"}, + // field pairs + new String[] {"date", "message"}, + new String[] {"level", "message"}, + new String[] {"date", "level"}, + // out-of-order pairs + new String[] {"message", "date"}, + new String[] {"message", "level"}, + new String[] {"level", "date"}, + // full projection, different orderings + new String[] {"date", "level", "message"}, + new String[] {"level", "date", "message"}, + new String[] {"date", "message", "level"}, + new String[] {"level", "message", "date"}, + new String[] {"message", "date", "level"}, + new String[] {"message", "level", "date"} + }; + + for (String[] ordering : cases) { + List expected = logs.select("id", ordering).orderBy("id").collectAsList(); + List actual = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(table.location()) + .select("id", ordering) + .orderBy("id") + .collectAsList(); + assertThat(actual) + .as("Rows should match for ordering: " + Arrays.toString(ordering)) + .isEqualTo(expected); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java new file mode 100644 index 000000000000..05e1cae51666 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java @@ -0,0 +1,81 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import java.util.Iterator; +import org.apache.iceberg.RecordWrapperTestBase; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.util.StructLikeWrapper; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.jupiter.api.Disabled; + +public class TestInternalRowWrapper extends RecordWrapperTestBase { + + @Disabled + @Override + public void testTimestampWithoutZone() { + // Spark does not support timestamp without zone. + } + + @Disabled + @Override + public void testTime() { + // Spark does not support time fields. + } + + @Override + protected void generateAndValidate(Schema schema, AssertMethod assertMethod) { + int numRecords = 100; + Iterable recordList = RandomGenericData.generate(schema, numRecords, 101L); + Iterable rowList = RandomData.generateSpark(schema, numRecords, 101L); + + InternalRecordWrapper recordWrapper = new InternalRecordWrapper(schema.asStruct()); + InternalRowWrapper rowWrapper = + new InternalRowWrapper(SparkSchemaUtil.convert(schema), schema.asStruct()); + + Iterator actual = recordList.iterator(); + Iterator expected = rowList.iterator(); + + StructLikeWrapper actualWrapper = StructLikeWrapper.forType(schema.asStruct()); + StructLikeWrapper expectedWrapper = StructLikeWrapper.forType(schema.asStruct()); + for (int i = 0; i < numRecords; i++) { + assertThat(actual).as("Should have more records").hasNext(); + assertThat(expected).as("Should have more InternalRow").hasNext(); + + StructLike recordStructLike = recordWrapper.wrap(actual.next()); + StructLike rowStructLike = rowWrapper.wrap(expected.next()); + + assertMethod.assertEquals( + "Should have expected StructLike values", + actualWrapper.set(recordStructLike), + expectedWrapper.set(rowStructLike)); + } + + assertThat(actual).as("Shouldn't have more record").isExhausted(); + assertThat(expected).as("Shouldn't have more InternalRow").isExhausted(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java new file mode 100644 index 000000000000..c21ccd0100db --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTableReadableMetrics.java @@ -0,0 +1,379 @@ +/* + * 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.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.Base64; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +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.Lists; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMetadataTableReadableMetrics extends TestBaseWithCatalog { + + private static final Types.StructType LEAF_STRUCT_TYPE = + Types.StructType.of( + optional(1, "leafLongCol", Types.LongType.get()), + optional(2, "leafDoubleCol", Types.DoubleType.get())); + + private static final Types.StructType NESTED_STRUCT_TYPE = + Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE)); + + private static final Schema NESTED_SCHEMA = + new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE)); + + private static final Schema PRIMITIVE_SCHEMA = + new Schema( + required(1, "booleanCol", Types.BooleanType.get()), + required(2, "intCol", Types.IntegerType.get()), + required(3, "longCol", Types.LongType.get()), + required(4, "floatCol", Types.FloatType.get()), + required(5, "doubleCol", Types.DoubleType.get()), + optional(6, "decimalCol", Types.DecimalType.of(10, 2)), + optional(7, "stringCol", Types.StringType.get()), + optional(8, "fixedCol", Types.FixedType.ofLength(3)), + optional(9, "binaryCol", Types.BinaryType.get())); + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + protected static Object[][] parameters() { + return new Object[][] { + { + // only SparkCatalog supports metadata table sql queries + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties() + }, + }; + } + + protected String tableName() { + return tableName.split("\\.")[2]; + } + + protected String database() { + return tableName.split("\\.")[1]; + } + + private Table createPrimitiveTable() throws IOException { + Table table = + catalog.createTable( + TableIdentifier.of(Namespace.of(database()), tableName()), + PRIMITIVE_SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + List records = + Lists.newArrayList( + createPrimitiveRecord( + false, + 1, + 1L, + 0, + 1.0D, + new BigDecimal("1.00"), + "1", + Base64.getDecoder().decode("1111"), + ByteBuffer.wrap(Base64.getDecoder().decode("1111"))), + createPrimitiveRecord( + true, + 2, + 2L, + 0, + 2.0D, + new BigDecimal("2.00"), + "2", + Base64.getDecoder().decode("2222"), + ByteBuffer.wrap(Base64.getDecoder().decode("2222"))), + createPrimitiveRecord(false, 1, 1, Float.NaN, Double.NaN, null, "1", null, null), + createPrimitiveRecord( + false, 2, 2L, Float.NaN, 2.0D, new BigDecimal("2.00"), "2", null, null)); + + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); + table.newAppend().appendFile(dataFile).commit(); + return table; + } + + private Pair createNestedTable() throws IOException { + Table table = + catalog.createTable( + TableIdentifier.of(Namespace.of(database()), tableName()), + NESTED_SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + + List records = + Lists.newArrayList( + createNestedRecord(0L, 0.0), + createNestedRecord(1L, Double.NaN), + createNestedRecord(null, null)); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); + table.newAppend().appendFile(dataFile).commit(); + return Pair.of(table, dataFile); + } + + @AfterEach + public void dropTable() { + sql("DROP TABLE %s", tableName); + } + + private Dataset filesDf() { + return spark.read().format("iceberg").load(database() + "." + tableName() + ".files"); + } + + protected GenericRecord createPrimitiveRecord( + boolean booleanCol, + int intCol, + long longCol, + float floatCol, + double doubleCol, + BigDecimal decimalCol, + String stringCol, + byte[] fixedCol, + ByteBuffer binaryCol) { + GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA); + record.set(0, booleanCol); + record.set(1, intCol); + record.set(2, longCol); + record.set(3, floatCol); + record.set(4, doubleCol); + record.set(5, decimalCol); + record.set(6, stringCol); + record.set(7, fixedCol); + record.set(8, binaryCol); + return record; + } + + private GenericRecord createNestedRecord(Long longCol, Double doubleCol) { + GenericRecord record = GenericRecord.create(NESTED_SCHEMA); + GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE); + GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE); + leaf.set(0, longCol); + leaf.set(1, doubleCol); + nested.set(0, leaf); + record.set(0, nested); + return record; + } + + @TestTemplate + public void testPrimitiveColumns() throws Exception { + Table table = createPrimitiveTable(); + DataFile dataFile = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + Map columnSizeStats = dataFile.columnSizes(); + + Object[] binaryCol = + row( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("binaryCol").fieldId()), + 4L, + 2L, + null, + Base64.getDecoder().decode("1111"), + Base64.getDecoder().decode("2222")); + Object[] booleanCol = + row( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("booleanCol").fieldId()), + 4L, + 0L, + null, + false, + true); + Object[] decimalCol = + row( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("decimalCol").fieldId()), + 4L, + 1L, + null, + new BigDecimal("1.00"), + new BigDecimal("2.00")); + Object[] doubleCol = + row( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("doubleCol").fieldId()), + 4L, + 0L, + 1L, + 1.0D, + 2.0D); + Object[] fixedCol = + row( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("fixedCol").fieldId()), + 4L, + 2L, + null, + Base64.getDecoder().decode("1111"), + Base64.getDecoder().decode("2222")); + Object[] floatCol = + row( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("floatCol").fieldId()), + 4L, + 0L, + 2L, + 0f, + 0f); + Object[] intCol = + row( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("intCol").fieldId()), + 4L, + 0L, + null, + 1, + 2); + Object[] longCol = + row( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("longCol").fieldId()), + 4L, + 0L, + null, + 1L, + 2L); + Object[] stringCol = + row( + columnSizeStats.get(PRIMITIVE_SCHEMA.findField("stringCol").fieldId()), + 4L, + 0L, + null, + "1", + "2"); + + Object[] metrics = + row( + binaryCol, + booleanCol, + decimalCol, + doubleCol, + fixedCol, + floatCol, + intCol, + longCol, + stringCol); + + List expected = ImmutableList.of(new Object[] {metrics}); + String sql = "SELECT readable_metrics FROM %s.%s"; + List filesReadableMetrics = sql(String.format(sql, tableName, "files")); + List entriesReadableMetrics = sql(String.format(sql, tableName, "entries")); + assertEquals("Row should match for files table", expected, filesReadableMetrics); + assertEquals("Row should match for entries table", expected, entriesReadableMetrics); + } + + @TestTemplate + public void testSelectPrimitiveValues() throws Exception { + createPrimitiveTable(); + + List expected = ImmutableList.of(row(1, true)); + String sql = + "SELECT readable_metrics.intCol.lower_bound, readable_metrics.booleanCol.upper_bound FROM %s.%s"; + List filesReadableMetrics = sql(String.format(sql, tableName, "files")); + List entriesReadableMetrics = sql(String.format(sql, tableName, "entries")); + assertEquals( + "select of primitive readable_metrics fields should work for files table", + expected, + filesReadableMetrics); + assertEquals( + "select of primitive readable_metrics fields should work for entries table", + expected, + entriesReadableMetrics); + + assertEquals( + "mixed select of readable_metrics and other field should work", + ImmutableList.of(row(0, 4L)), + sql("SELECT content, readable_metrics.longCol.value_count FROM %s.files", tableName)); + + assertEquals( + "mixed select of readable_metrics and other field should work, in the other order", + ImmutableList.of(row(4L, 0)), + sql("SELECT readable_metrics.longCol.value_count, content FROM %s.files", tableName)); + + assertEquals( + "mixed select of readable_metrics and other field should work for entries table", + ImmutableList.of(row(1, 4L)), + sql("SELECT status, readable_metrics.longCol.value_count FROM %s.entries", tableName)); + + assertEquals( + "mixed select of readable_metrics and other field should work, in the other order for entries table", + ImmutableList.of(row(4L, 1)), + sql("SELECT readable_metrics.longCol.value_count, status FROM %s.entries", tableName)); + } + + @TestTemplate + public void testSelectNestedValues() throws Exception { + createNestedTable(); + + List expected = ImmutableList.of(row(0L, 3L)); + String sql = + "SELECT readable_metrics.`nestedStructCol.leafStructCol.leafLongCol`.lower_bound, " + + "readable_metrics.`nestedStructCol.leafStructCol.leafDoubleCol`.value_count FROM %s.%s"; + List filesReadableMetrics = sql(String.format(sql, tableName, "files")); + List entriesReadableMetrics = sql(String.format(sql, tableName, "entries")); + + assertEquals( + "select of nested readable_metrics fields should work for files table", + expected, + filesReadableMetrics); + assertEquals( + "select of nested readable_metrics fields should work for entries table", + expected, + entriesReadableMetrics); + } + + @TestTemplate + public void testNestedValues() throws Exception { + Pair table = createNestedTable(); + int longColId = + table.first().schema().findField("nestedStructCol.leafStructCol.leafLongCol").fieldId(); + int doubleColId = + table.first().schema().findField("nestedStructCol.leafStructCol.leafDoubleCol").fieldId(); + + Object[] leafDoubleCol = + row(table.second().columnSizes().get(doubleColId), 3L, 1L, 1L, 0.0D, 0.0D); + Object[] leafLongCol = row(table.second().columnSizes().get(longColId), 3L, 1L, null, 0L, 1L); + Object[] metrics = row(leafDoubleCol, leafLongCol); + + List expected = ImmutableList.of(new Object[] {metrics}); + String sql = "SELECT readable_metrics FROM %s.%s"; + List filesReadableMetrics = sql(String.format(sql, tableName, "files")); + List entriesReadableMetrics = sql(String.format(sql, tableName, "entries")); + assertEquals("Row should match for files table", expected, filesReadableMetrics); + assertEquals("Row should match for entries table", expected, entriesReadableMetrics); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java new file mode 100644 index 000000000000..a417454b45dc --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java @@ -0,0 +1,725 @@ +/* + * 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.FileFormat.AVRO; +import static org.apache.iceberg.FileFormat.ORC; +import static org.apache.iceberg.FileFormat.PARQUET; +import static org.apache.iceberg.MetadataTableType.ALL_DATA_FILES; +import static org.apache.iceberg.MetadataTableType.ALL_ENTRIES; +import static org.apache.iceberg.MetadataTableType.ENTRIES; +import static org.apache.iceberg.MetadataTableType.FILES; +import static org.apache.iceberg.MetadataTableType.PARTITIONS; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.FORMAT_VERSION; +import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; +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.Arrays; +import java.util.List; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expressions; +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.CatalogTestBase; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMetadataTablesWithPartitionEvolution extends CatalogTestBase { + + @Parameters(name = "catalog = {0}, impl = {1}, conf = {2}, fileFormat = {3}, formatVersion = {4}") + public static Object[][] parameters() { + return new Object[][] { + { + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default"), + ORC, + 1 + }, + { + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default"), + ORC, + 2 + }, + {"testhadoop", SparkCatalog.class.getName(), ImmutableMap.of("type", "hadoop"), PARQUET, 1}, + {"testhadoop", SparkCatalog.class.getName(), ImmutableMap.of("type", "hadoop"), PARQUET, 2}, + { + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "clients", "1", + "parquet-enabled", "false", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + ), + AVRO, + 1 + }, + { + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "clients", "1", + "parquet-enabled", "false", + "cache-enabled", + "false" // Spark will delete tables using v1, leaving the cache out of sync + ), + AVRO, + 2 + } + }; + } + + @Parameter(index = 3) + private FileFormat fileFormat; + + @Parameter(index = 4) + private int formatVersion; + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testFilesMetadataTable() throws ParseException { + createTable("id bigint NOT NULL, category string, data string"); + + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", tableName); + + // verify the metadata tables while the current spec is still unpartitioned + for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { + Dataset df = loadMetadataTable(tableType); + assertThat(df.schema().getFieldIndex("partition").isEmpty()) + .as("Partition must be skipped") + .isTrue(); + } + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateSpec().addField("data").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", tableName); + + // verify the metadata tables after adding the first partition column + for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { + assertPartitions( + ImmutableList.of(row(new Object[] {null}), row("b1")), "STRUCT", tableType); + } + + table.updateSpec().addField(Expressions.bucket("category", 8)).commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", tableName); + + // verify the metadata tables after adding the second partition column + for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { + assertPartitions( + ImmutableList.of(row(null, null), row("b1", null), row("b1", 2)), + "STRUCT", + tableType); + } + + table.updateSpec().removeField("data").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", tableName); + + // verify the metadata tables after dropping the first partition column + for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { + assertPartitions( + ImmutableList.of(row(null, null), row(null, 2), row("b1", null), row("b1", 2)), + "STRUCT", + tableType); + } + + table.updateSpec().renameField("category_bucket_8", "category_bucket_8_another_name").commit(); + sql("REFRESH TABLE %s", tableName); + + // verify the metadata tables after renaming the second partition column + for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { + assertPartitions( + ImmutableList.of(row(null, null), row(null, 2), row("b1", null), row("b1", 2)), + "STRUCT", + tableType); + } + } + + @TestTemplate + public void testFilesMetadataTableFilter() throws ParseException { + createTable("id bigint NOT NULL, category string, data string"); + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, MANIFEST_MERGE_ENABLED); + + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + // verify the metadata tables while the current spec is still unpartitioned + for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { + Dataset df = loadMetadataTable(tableType); + assertThat(df.schema().getFieldIndex("partition").isEmpty()) + .as("Partition must be skipped") + .isTrue(); + } + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateSpec().addField("data").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + // verify the metadata tables after adding the first partition column + for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { + assertPartitions( + ImmutableList.of(row("d2")), "STRUCT", tableType, "partition.data = 'd2'"); + } + + table.updateSpec().addField("category").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + // verify the metadata tables after adding the second partition column + for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { + assertPartitions( + ImmutableList.of(row("d2", null), row("d2", "c2")), + "STRUCT", + tableType, + "partition.data = 'd2'"); + } + for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { + assertPartitions( + ImmutableList.of(row("d2", "c2")), + "STRUCT", + tableType, + "partition.category = 'c2'"); + } + + table.updateSpec().removeField("data").commit(); + sql("REFRESH TABLE %s", tableName); + + // Verify new partitions do not show up for removed 'partition.data=d2' query + sql("INSERT INTO TABLE %s VALUES (3, 'c3', 'd2')", tableName); + sql("INSERT INTO TABLE %s VALUES (4, 'c4', 'd2')", tableName); + + // Verify new partitions do show up for 'partition.category=c2' query + sql("INSERT INTO TABLE %s VALUES (5, 'c2', 'd5')", tableName); + + // no new partition should show up for 'data' partition query as partition field has been + // removed + for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { + assertPartitions( + ImmutableList.of(row("d2", null), row("d2", "c2")), + "STRUCT", + tableType, + "partition.data = 'd2'"); + } + // new partition shows up from 'category' partition field query + for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { + assertPartitions( + ImmutableList.of(row(null, "c2"), row("d2", "c2")), + "STRUCT", + tableType, + "partition.category = 'c2'"); + } + + table.updateSpec().renameField("category", "category_another_name").commit(); + sql("REFRESH TABLE %s", tableName); + + // Verify new partitions do show up for 'category=c2' query + sql("INSERT INTO TABLE %s VALUES (6, 'c2', 'd6')", tableName); + for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES)) { + assertPartitions( + ImmutableList.of(row(null, "c2"), row(null, "c2"), row("d2", "c2")), + "STRUCT", + tableType, + "partition.category_another_name = 'c2'"); + } + } + + @TestTemplate + public void testEntriesMetadataTable() throws ParseException { + createTable("id bigint NOT NULL, category string, data string"); + + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", tableName); + + // verify the metadata tables while the current spec is still unpartitioned + for (MetadataTableType tableType : Arrays.asList(ENTRIES, ALL_ENTRIES)) { + Dataset df = loadMetadataTable(tableType); + StructType dataFileType = (StructType) df.schema().apply("data_file").dataType(); + assertThat(dataFileType.getFieldIndex("").isEmpty()).as("Partition must be skipped").isTrue(); + } + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateSpec().addField("data").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", tableName); + + // verify the metadata tables after adding the first partition column + for (MetadataTableType tableType : Arrays.asList(ENTRIES, ALL_ENTRIES)) { + assertPartitions( + ImmutableList.of(row(new Object[] {null}), row("b1")), "STRUCT", tableType); + } + + table.updateSpec().addField(Expressions.bucket("category", 8)).commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", tableName); + + // verify the metadata tables after adding the second partition column + for (MetadataTableType tableType : Arrays.asList(ENTRIES, ALL_ENTRIES)) { + assertPartitions( + ImmutableList.of(row(null, null), row("b1", null), row("b1", 2)), + "STRUCT", + tableType); + } + + table.updateSpec().removeField("data").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", tableName); + + // verify the metadata tables after dropping the first partition column + for (MetadataTableType tableType : Arrays.asList(ENTRIES, ALL_ENTRIES)) { + assertPartitions( + ImmutableList.of(row(null, null), row(null, 2), row("b1", null), row("b1", 2)), + "STRUCT", + tableType); + } + + table.updateSpec().renameField("category_bucket_8", "category_bucket_8_another_name").commit(); + sql("REFRESH TABLE %s", tableName); + + // verify the metadata tables after renaming the second partition column + for (MetadataTableType tableType : Arrays.asList(ENTRIES, ALL_ENTRIES)) { + assertPartitions( + ImmutableList.of(row(null, null), row(null, 2), row("b1", null), row("b1", 2)), + "STRUCT", + tableType); + } + } + + @TestTemplate + public void testPartitionsTableAddRemoveFields() throws ParseException { + createTable("id bigint NOT NULL, category string, data string"); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + // verify the metadata tables while the current spec is still unpartitioned + Dataset df = loadMetadataTable(PARTITIONS); + assertThat(df.schema().getFieldIndex("partition").isEmpty()) + .as("Partition must be skipped") + .isTrue(); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateSpec().addField("data").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + // verify the metadata tables after adding the first partition column + assertPartitions( + ImmutableList.of(row(new Object[] {null}), row("d1"), row("d2")), + "STRUCT", + PARTITIONS); + + table.updateSpec().addField("category").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + // verify the metadata tables after adding the second partition column + assertPartitions( + ImmutableList.of( + row(null, null), row("d1", null), row("d1", "c1"), row("d2", null), row("d2", "c2")), + "STRUCT", + PARTITIONS); + + // verify the metadata tables after removing the first partition column + table.updateSpec().removeField("data").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + assertPartitions( + ImmutableList.of( + row(null, null), + row(null, "c1"), + row(null, "c2"), + row("d1", null), + row("d1", "c1"), + row("d2", null), + row("d2", "c2")), + "STRUCT", + PARTITIONS); + } + + @TestTemplate + public void testPartitionsTableRenameFields() throws ParseException { + createTable("id bigint NOT NULL, category string, data string"); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateSpec().addField("data").addField("category").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + assertPartitions( + ImmutableList.of(row("d1", "c1"), row("d2", "c2")), + "STRUCT", + PARTITIONS); + + table.updateSpec().renameField("category", "category_another_name").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + assertPartitions( + ImmutableList.of(row("d1", "c1"), row("d2", "c2")), + "STRUCT", + PARTITIONS); + } + + @TestTemplate + public void testPartitionsTableSwitchFields() throws Exception { + createTable("id bigint NOT NULL, category string, data string"); + + Table table = validationCatalog.loadTable(tableIdent); + + // verify the metadata tables after re-adding the first dropped column in the second location + table.updateSpec().addField("data").addField("category").commit(); + sql("REFRESH TABLE %s", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + assertPartitions( + ImmutableList.of(row("d1", "c1"), row("d2", "c2")), + "STRUCT", + PARTITIONS); + + table.updateSpec().removeField("data").commit(); + sql("REFRESH TABLE %s", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + assertPartitions( + ImmutableList.of(row(null, "c1"), row(null, "c2"), row("d1", "c1"), row("d2", "c2")), + "STRUCT", + PARTITIONS); + + table.updateSpec().addField("data").commit(); + sql("REFRESH TABLE %s", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + sql("INSERT INTO TABLE %s VALUES (3, 'c3', 'd3')", tableName); + + if (formatVersion == 1) { + assertPartitions( + ImmutableList.of( + row(null, "c1", null), + row(null, "c1", "d1"), + row(null, "c2", null), + row(null, "c2", "d2"), + row(null, "c3", "d3"), + row("d1", "c1", null), + row("d2", "c2", null)), + "STRUCT", + PARTITIONS); + } else { + // In V2 re-adding a former partition field that was part of an older spec will not change its + // name or its + // field ID either, thus values will be collapsed into a single common column (as opposed to + // V1 where any new + // partition field addition will result in a new column in this metadata table) + assertPartitions( + ImmutableList.of( + row(null, "c1"), row(null, "c2"), row("d1", "c1"), row("d2", "c2"), row("d3", "c3")), + "STRUCT", + PARTITIONS); + } + } + + @TestTemplate + public void testPartitionTableFilterAddRemoveFields() throws ParseException { + // Create un-partitioned table + createTable("id bigint NOT NULL, category string, data string"); + + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + // Partition Table with one partition column + Table table = validationCatalog.loadTable(tableIdent); + table.updateSpec().addField("data").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + assertPartitions( + ImmutableList.of(row("d2")), "STRUCT", PARTITIONS, "partition.data = 'd2'"); + + // Partition Table with two partition column + table.updateSpec().addField("category").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + assertPartitions( + ImmutableList.of(row("d2", null), row("d2", "c2")), + "STRUCT", + PARTITIONS, + "partition.data = 'd2'"); + assertPartitions( + ImmutableList.of(row("d2", "c2")), + "STRUCT", + PARTITIONS, + "partition.category = 'c2'"); + + // Partition Table with first partition column removed + table.updateSpec().removeField("data").commit(); + sql("REFRESH TABLE %s", tableName); + + sql("INSERT INTO TABLE %s VALUES (3, 'c3', 'd2')", tableName); + sql("INSERT INTO TABLE %s VALUES (4, 'c4', 'd2')", tableName); + sql("INSERT INTO TABLE %s VALUES (5, 'c2', 'd5')", tableName); + assertPartitions( + ImmutableList.of(row("d2", null), row("d2", "c2")), + "STRUCT", + PARTITIONS, + "partition.data = 'd2'"); + assertPartitions( + ImmutableList.of(row(null, "c2"), row("d2", "c2")), + "STRUCT", + PARTITIONS, + "partition.category = 'c2'"); + } + + @TestTemplate + public void testPartitionTableFilterSwitchFields() throws Exception { + // Re-added partition fields currently not re-associated: + // https://github.com/apache/iceberg/issues/4292 + // In V1, dropped partition fields show separately when field is re-added + // In V2, re-added field currently conflicts with its deleted form + assumeThat(formatVersion).isEqualTo(1); + + createTable("id bigint NOT NULL, category string, data string"); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + + // Two partition columns + table.updateSpec().addField("data").addField("category").commit(); + sql("REFRESH TABLE %s", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + // Drop first partition column + table.updateSpec().removeField("data").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + // Re-add first partition column at the end + table.updateSpec().addField("data").commit(); + sql("REFRESH TABLE %s", tableName); + + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + assertPartitions( + ImmutableList.of(row(null, "c2", null), row(null, "c2", "d2"), row("d2", "c2", null)), + "STRUCT", + PARTITIONS, + "partition.category = 'c2'"); + + assertPartitions( + ImmutableList.of(row(null, "c1", "d1")), + "STRUCT", + PARTITIONS, + "partition.data = 'd1'"); + } + + @TestTemplate + public void testPartitionsTableFilterRenameFields() throws ParseException { + createTable("id bigint NOT NULL, category string, data string"); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateSpec().addField("data").addField("category").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + table.updateSpec().renameField("category", "category_another_name").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'c1', 'd1')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'c2', 'd2')", tableName); + + assertPartitions( + ImmutableList.of(row("d1", "c1")), + "STRUCT", + PARTITIONS, + "partition.category_another_name = 'c1'"); + } + + @TestTemplate + public void testMetadataTablesWithUnknownTransforms() { + createTable("id bigint NOT NULL, category string, data string"); + + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + PartitionSpec unknownSpec = + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(1) + .addField("zero", 1, "id_zero") + .build(); + + // replace the table spec to include an unknown transform + TableOperations ops = ((HasTableOperations) table).operations(); + TableMetadata base = ops.current(); + ops.commit(base, base.updatePartitionSpec(unknownSpec)); + + sql("REFRESH TABLE %s", tableName); + + for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES, ENTRIES, ALL_ENTRIES)) { + assertThatThrownBy(() -> loadMetadataTable(tableType)) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot build table partition type, unknown transforms: [zero]"); + } + } + + @TestTemplate + public void testPartitionColumnNamedPartition() { + sql( + "CREATE TABLE %s (id int, partition int) USING iceberg PARTITIONED BY (partition)", + tableName); + sql("INSERT INTO %s VALUES (1, 1), (2, 1), (3, 2), (2, 2)", tableName); + List expected = ImmutableList.of(row(1, 1), row(2, 1), row(3, 2), row(2, 2)); + assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); + assertThat(sql("SELECT * FROM %s.files", tableName)).hasSize(2); + } + + private void assertPartitions( + List expectedPartitions, String expectedTypeAsString, MetadataTableType tableType) + throws ParseException { + assertPartitions(expectedPartitions, expectedTypeAsString, tableType, null); + } + + private void assertPartitions( + List expectedPartitions, + String expectedTypeAsString, + MetadataTableType tableType, + String filter) + throws ParseException { + Dataset df = loadMetadataTable(tableType); + if (filter != null) { + df = df.filter(filter); + } + + DataType expectedType = spark.sessionState().sqlParser().parseDataType(expectedTypeAsString); + switch (tableType) { + case PARTITIONS: + case FILES: + case ALL_DATA_FILES: + DataType actualFilesType = df.schema().apply("partition").dataType(); + assertThat(actualFilesType).as("Partition type must match").isEqualTo(expectedType); + break; + + case ENTRIES: + case ALL_ENTRIES: + StructType dataFileType = (StructType) df.schema().apply("data_file").dataType(); + DataType actualEntriesType = dataFileType.apply("partition").dataType(); + assertThat(actualEntriesType).as("Partition type must match").isEqualTo(expectedType); + break; + + default: + throw new UnsupportedOperationException("Unsupported metadata table type: " + tableType); + } + + switch (tableType) { + case PARTITIONS: + case FILES: + case ALL_DATA_FILES: + List actualFilesPartitions = + df.orderBy("partition").select("partition.*").collectAsList(); + assertEquals( + "Partitions must match", expectedPartitions, rowsToJava(actualFilesPartitions)); + break; + + case ENTRIES: + case ALL_ENTRIES: + List actualEntriesPartitions = + df.orderBy("data_file.partition").select("data_file.partition.*").collectAsList(); + assertEquals( + "Partitions must match", expectedPartitions, rowsToJava(actualEntriesPartitions)); + break; + + default: + throw new UnsupportedOperationException("Unsupported metadata table type: " + tableType); + } + } + + private Dataset loadMetadataTable(MetadataTableType tableType) { + return spark.read().format("iceberg").load(tableName + "." + tableType.name()); + } + + private void createTable(String schema) { + sql( + "CREATE TABLE %s (%s) USING iceberg TBLPROPERTIES ('%s' '%s', '%s' '%d')", + tableName, schema, DEFAULT_FILE_FORMAT, fileFormat.name(), FORMAT_VERSION, formatVersion); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java new file mode 100644 index 000000000000..892e260f66f0 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestORCDataFrameWrite.java @@ -0,0 +1,57 @@ +/* + * 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.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.spark.SparkException; +import org.junit.jupiter.api.Test; + +public class TestORCDataFrameWrite extends DataFrameWriteTestBase { + @Override + protected void configureTable(Table table) { + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.ORC.toString()) + .commit(); + } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create ListType with unknown element type"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create MapType with unknown value type"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java new file mode 100644 index 000000000000..5e1d88f8bcd4 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java @@ -0,0 +1,33 @@ +/* + * 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.junit.jupiter.api.BeforeAll; + +public class TestParquetCometVectorizedScan extends TestParquetScan { + @BeforeAll + public static void setComet() { + ScanTestBase.spark.conf().set("spark.sql.iceberg.parquet.reader-type", "COMET"); + } + + @Override + protected boolean vectorized() { + return true; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java new file mode 100644 index 000000000000..7a8531278314 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetDataFrameWrite.java @@ -0,0 +1,62 @@ +/* + * 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.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.spark.SparkException; +import org.junit.jupiter.api.Test; + +public class TestParquetDataFrameWrite extends DataFrameWriteTestBase { + @Override + protected boolean supportsVariant() { + return true; + } + + @Override + protected void configureTable(Table table) { + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.PARQUET.toString()) + .commit(); + } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert element Parquet: unknown"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(SparkException.class) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert value Parquet: unknown"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java new file mode 100644 index 000000000000..0ea54821a46a --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetScan.java @@ -0,0 +1,113 @@ +/* + * 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.Files.localOutput; +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.List; +import java.util.UUID; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestParquetScan extends ScanTestBase { + protected boolean vectorized() { + return false; + } + + @Override + protected boolean supportsVariant() { + return true; + } + + @Override + protected void configureTable(Table table) { + table + .updateProperties() + .set(TableProperties.PARQUET_VECTORIZATION_ENABLED, String.valueOf(vectorized())) + .commit(); + } + + @Override + protected void writeRecords(Table table, List records) throws IOException { + File dataFolder = new File(table.location(), "data"); + File parquetFile = + new File(dataFolder, FileFormat.PARQUET.addExtension(UUID.randomUUID().toString())); + + try (FileAppender writer = + Parquet.write(localOutput(parquetFile)) + .schema(table.schema()) + .createWriterFunc(GenericParquetWriter::create) + .build()) { + writer.addAll(records); + } + + DataFile file = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withFileSizeInBytes(parquetFile.length()) + .withPath(parquetFile.toString()) + .withRecordCount(records.size()) + .build(); + + table.newAppend().appendFile(file).commit(); + } + + @Override + protected void writeAndValidate(Schema writeSchema, Schema expectedSchema) throws IOException { + assumeThat( + TypeUtil.find( + writeSchema, + type -> type.isMapType() && type.asMapType().keyType() != Types.StringType.get())) + .as("Cannot handle non-string map keys in parquet-avro") + .isNull(); + + super.writeAndValidate(writeSchema, expectedSchema); + } + + @Test + @Override + public void testUnknownListType() { + assertThatThrownBy(super::testUnknownListType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert element Parquet: unknown"); + } + + @Test + @Override + public void testUnknownMapType() { + assertThatThrownBy(super::testUnknownMapType) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot convert value Parquet: unknown"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetVectorizedScan.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetVectorizedScan.java new file mode 100644 index 000000000000..a6b5166b3a4e --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetVectorizedScan.java @@ -0,0 +1,26 @@ +/* + * 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; + +public class TestParquetVectorizedScan extends TestParquetScan { + @Override + protected boolean vectorized() { + return true; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java new file mode 100644 index 000000000000..b31880e8e958 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java @@ -0,0 +1,489 @@ +/* + * 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.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.net.URI; +import java.nio.file.Files; +import java.sql.Timestamp; +import java.time.Instant; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RawLocalFileSystem; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.hadoop.HadoopTables; +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.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Types; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestPartitionPruning { + + private static final Configuration CONF = new Configuration(); + private static final HadoopTables TABLES = new HadoopTables(CONF); + + @Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.PARQUET, false, DISTRIBUTED}, + {FileFormat.PARQUET, true, LOCAL}, + {FileFormat.AVRO, false, DISTRIBUTED}, + {FileFormat.ORC, false, LOCAL}, + {FileFormat.ORC, true, DISTRIBUTED} + }; + } + + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private boolean vectorized; + + @Parameter(index = 2) + private PlanningMode planningMode; + + private static SparkSession spark = null; + private static JavaSparkContext sparkContext = null; + + private static final Function BUCKET_FUNC = + Transforms.bucket(3).bind(Types.IntegerType.get()); + private static final Function TRUNCATE_FUNC = + Transforms.truncate(5).bind(Types.StringType.get()); + private static final Function HOUR_FUNC = + Transforms.hour().bind(Types.TimestampType.withoutZone()); + + @BeforeAll + public static void startSpark() { + TestPartitionPruning.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .getOrCreate(); + TestPartitionPruning.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + String optionKey = String.format("fs.%s.impl", CountOpenLocalFileSystem.scheme); + CONF.set(optionKey, CountOpenLocalFileSystem.class.getName()); + spark.conf().set(optionKey, CountOpenLocalFileSystem.class.getName()); + spark.conf().set("spark.sql.session.timeZone", "UTC"); + spark.udf().register("bucket3", (Integer num) -> BUCKET_FUNC.apply(num), DataTypes.IntegerType); + spark + .udf() + .register("truncate5", (String str) -> TRUNCATE_FUNC.apply(str), DataTypes.StringType); + // NOTE: date transforms take the type long, not Timestamp + spark + .udf() + .register( + "hour", + (Timestamp ts) -> + HOUR_FUNC.apply( + org.apache.spark.sql.catalyst.util.DateTimeUtils.fromJavaTimestamp(ts)), + DataTypes.IntegerType); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestPartitionPruning.spark; + TestPartitionPruning.spark = null; + currentSpark.stop(); + } + + private static final Schema LOG_SCHEMA = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "date", Types.StringType.get()), + Types.NestedField.optional(3, "level", Types.StringType.get()), + Types.NestedField.optional(4, "message", Types.StringType.get()), + Types.NestedField.optional(5, "timestamp", Types.TimestampType.withZone())); + + private static final List LOGS = + ImmutableList.of( + LogMessage.debug("2020-02-02", "debug event 1", getInstant("2020-02-02T00:00:00")), + LogMessage.info("2020-02-02", "info event 1", getInstant("2020-02-02T01:00:00")), + LogMessage.debug("2020-02-02", "debug event 2", getInstant("2020-02-02T02:00:00")), + LogMessage.info("2020-02-03", "info event 2", getInstant("2020-02-03T00:00:00")), + LogMessage.debug("2020-02-03", "debug event 3", getInstant("2020-02-03T01:00:00")), + LogMessage.info("2020-02-03", "info event 3", getInstant("2020-02-03T02:00:00")), + LogMessage.error("2020-02-03", "error event 1", getInstant("2020-02-03T03:00:00")), + LogMessage.debug("2020-02-04", "debug event 4", getInstant("2020-02-04T01:00:00")), + LogMessage.warn("2020-02-04", "warn event 1", getInstant("2020-02-04T02:00:00")), + LogMessage.debug("2020-02-04", "debug event 5", getInstant("2020-02-04T03:00:00"))); + + private static Instant getInstant(String timestampWithoutZone) { + Long epochMicros = + (Long) Literal.of(timestampWithoutZone).to(Types.TimestampType.withoutZone()).value(); + return Instant.ofEpochMilli(TimeUnit.MICROSECONDS.toMillis(epochMicros)); + } + + @TempDir private java.nio.file.Path temp; + + private final PartitionSpec spec = + PartitionSpec.builderFor(LOG_SCHEMA) + .identity("date") + .identity("level") + .bucket("id", 3) + .truncate("message", 5) + .hour("timestamp") + .build(); + + @TestTemplate + public void testPartitionPruningIdentityString() { + String filterCond = "date >= '2020-02-03' AND level = 'DEBUG'"; + Predicate partCondition = + (Row r) -> { + String date = r.getString(0); + String level = r.getString(1); + return date.compareTo("2020-02-03") >= 0 && level.equals("DEBUG"); + }; + + runTest(filterCond, partCondition); + } + + @TestTemplate + public void testPartitionPruningBucketingInteger() { + final int[] ids = new int[] {LOGS.get(3).getId(), LOGS.get(7).getId()}; + String condForIds = + Arrays.stream(ids).mapToObj(String::valueOf).collect(Collectors.joining(",", "(", ")")); + String filterCond = "id in " + condForIds; + Predicate partCondition = + (Row r) -> { + int bucketId = r.getInt(2); + Set buckets = + Arrays.stream(ids).map(BUCKET_FUNC::apply).boxed().collect(Collectors.toSet()); + return buckets.contains(bucketId); + }; + + runTest(filterCond, partCondition); + } + + @TestTemplate + public void testPartitionPruningTruncatedString() { + String filterCond = "message like 'info event%'"; + Predicate partCondition = + (Row r) -> { + String truncatedMessage = r.getString(3); + return truncatedMessage.equals("info "); + }; + + runTest(filterCond, partCondition); + } + + @TestTemplate + public void testPartitionPruningTruncatedStringComparingValueShorterThanPartitionValue() { + String filterCond = "message like 'inf%'"; + Predicate partCondition = + (Row r) -> { + String truncatedMessage = r.getString(3); + return truncatedMessage.startsWith("inf"); + }; + + runTest(filterCond, partCondition); + } + + @TestTemplate + public void testPartitionPruningHourlyPartition() { + String filterCond; + if (spark.version().startsWith("2")) { + // Looks like from Spark 2 we need to compare timestamp with timestamp to push down the + // filter. + filterCond = "timestamp >= to_timestamp('2020-02-03T01:00:00')"; + } else { + filterCond = "timestamp >= '2020-02-03T01:00:00'"; + } + Predicate partCondition = + (Row r) -> { + int hourValue = r.getInt(4); + Instant instant = getInstant("2020-02-03T01:00:00"); + Integer hourValueToFilter = + HOUR_FUNC.apply(TimeUnit.MILLISECONDS.toMicros(instant.toEpochMilli())); + return hourValue >= hourValueToFilter; + }; + + runTest(filterCond, partCondition); + } + + private void runTest(String filterCond, Predicate partCondition) { + File originTableLocation = createTempDir(); + assertThat(originTableLocation).as("Temp folder should exist").exists(); + + Table table = createTable(originTableLocation); + Dataset logs = createTestDataset(); + saveTestDatasetToTable(logs, table); + + List expected = + logs.select("id", "date", "level", "message", "timestamp") + .filter(filterCond) + .orderBy("id") + .collectAsList(); + assertThat(expected).as("Expected rows should not be empty").isNotEmpty(); + + // remove records which may be recorded during storing to table + CountOpenLocalFileSystem.resetRecordsInPathPrefix(originTableLocation.getAbsolutePath()); + + List actual = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(table.location()) + .select("id", "date", "level", "message", "timestamp") + .filter(filterCond) + .orderBy("id") + .collectAsList(); + assertThat(actual).isNotEmpty().isEqualTo(expected); + + assertAccessOnDataFiles(originTableLocation, table, partCondition); + } + + private File createTempDir() { + try { + return Files.createTempDirectory(temp, "junit").toFile(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private Table createTable(File originTableLocation) { + String trackedTableLocation = CountOpenLocalFileSystem.convertPath(originTableLocation); + Map properties = + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, format.toString(), + TableProperties.DATA_PLANNING_MODE, planningMode.modeName(), + TableProperties.DELETE_PLANNING_MODE, planningMode.modeName()); + return TABLES.create(LOG_SCHEMA, spec, properties, trackedTableLocation); + } + + private Dataset createTestDataset() { + List rows = + LOGS.stream() + .map( + logMessage -> { + Object[] underlying = + new Object[] { + logMessage.getId(), + UTF8String.fromString(logMessage.getDate()), + UTF8String.fromString(logMessage.getLevel()), + UTF8String.fromString(logMessage.getMessage()), + // discard the nanoseconds part to simplify + TimeUnit.MILLISECONDS.toMicros(logMessage.getTimestamp().toEpochMilli()) + }; + return new GenericInternalRow(underlying); + }) + .collect(Collectors.toList()); + + JavaRDD rdd = sparkContext.parallelize(rows); + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + + Dataset df = + ((org.apache.spark.sql.classic.SparkSession) spark) + .internalCreateDataFrame( + JavaRDD.toRDD(rdd), SparkSchemaUtil.convert(LOG_SCHEMA), false); + + return df.selectExpr("id", "date", "level", "message", "timestamp") + .selectExpr( + "id", + "date", + "level", + "message", + "timestamp", + "bucket3(id) AS bucket_id", + "truncate5(message) AS truncated_message", + "hour(timestamp) AS ts_hour"); + } + + private void saveTestDatasetToTable(Dataset logs, Table table) { + logs.orderBy("date", "level", "bucket_id", "truncated_message", "ts_hour") + .select("id", "date", "level", "message", "timestamp") + .write() + .format("iceberg") + .mode("append") + .save(table.location()); + } + + private void assertAccessOnDataFiles( + File originTableLocation, Table table, Predicate partCondition) { + // only use files in current table location to avoid side-effects on concurrent test runs + Set readFilesInQuery = + CountOpenLocalFileSystem.pathToNumOpenCalled.keySet().stream() + .filter(path -> path.startsWith(originTableLocation.getAbsolutePath())) + .collect(Collectors.toSet()); + + List files = + spark.read().format("iceberg").load(table.location() + "#files").collectAsList(); + + Set filesToRead = extractFilePathsMatchingConditionOnPartition(files, partCondition); + Set filesToNotRead = extractFilePathsNotIn(files, filesToRead); + + // Just to be sure, they should be mutually exclusive. + assertThat(filesToRead).doesNotContainAnyElementsOf(filesToNotRead); + + assertThat(filesToNotRead).as("The query should prune some data files.").isNotEmpty(); + + // We don't check "all" data files bound to the condition are being read, as data files can be + // pruned on + // other conditions like lower/upper bound of columns. + assertThat(filesToRead) + .as( + "Some of data files in partition range should be read. " + + "Read files in query: " + + readFilesInQuery + + " / data files in partition range: " + + filesToRead) + .containsAnyElementsOf(readFilesInQuery); + + // Data files which aren't bound to the condition shouldn't be read. + assertThat(filesToNotRead) + .as( + "Data files outside of partition range should not be read. " + + "Read files in query: " + + readFilesInQuery + + " / data files outside of partition range: " + + filesToNotRead) + .doesNotContainAnyElementsOf(readFilesInQuery); + } + + private Set extractFilePathsMatchingConditionOnPartition( + List files, Predicate condition) { + // idx 1: file_path, idx 3: partition + return files.stream() + .filter( + r -> { + Row partition = r.getStruct(4); + return condition.test(partition); + }) + .map(r -> CountOpenLocalFileSystem.stripScheme(r.getString(1))) + .collect(Collectors.toSet()); + } + + private Set extractFilePathsNotIn(List files, Set filePaths) { + Set allFilePaths = + files.stream() + .map(r -> CountOpenLocalFileSystem.stripScheme(r.getString(1))) + .collect(Collectors.toSet()); + return Sets.newHashSet(Sets.symmetricDifference(allFilePaths, filePaths)); + } + + public static class CountOpenLocalFileSystem extends RawLocalFileSystem { + public static String scheme = + String.format("TestIdentityPartitionData%dfs", new Random().nextInt()); + public static Map pathToNumOpenCalled = Maps.newConcurrentMap(); + + public static String convertPath(String absPath) { + return scheme + "://" + absPath; + } + + public static String convertPath(File file) { + return convertPath(file.getAbsolutePath()); + } + + public static String stripScheme(String pathWithScheme) { + if (!pathWithScheme.startsWith(scheme + ":")) { + throw new IllegalArgumentException("Received unexpected path: " + pathWithScheme); + } + + int idxToCut = scheme.length() + 1; + while (pathWithScheme.charAt(idxToCut) == '/') { + idxToCut++; + } + + // leave the last '/' + idxToCut--; + + return pathWithScheme.substring(idxToCut); + } + + public static void resetRecordsInPathPrefix(String pathPrefix) { + pathToNumOpenCalled.keySet().stream() + .filter(p -> p.startsWith(pathPrefix)) + .forEach(key -> pathToNumOpenCalled.remove(key)); + } + + @Override + public URI getUri() { + return URI.create(scheme + ":///"); + } + + @Override + public String getScheme() { + return scheme; + } + + @Override + public FSDataInputStream open(Path f, int bufferSize) throws IOException { + String path = f.toUri().getPath(); + pathToNumOpenCalled.compute( + path, + (ignored, v) -> { + if (v == null) { + return 1L; + } else { + return v + 1; + } + }); + return super.open(f, bufferSize); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java new file mode 100644 index 000000000000..0b6ab2052b66 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionValues.java @@ -0,0 +1,511 @@ +/* + * 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.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.File; +import java.net.InetAddress; +import java.nio.file.Path; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.avro.generic.GenericData; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.TestHelpers; +import org.apache.iceberg.types.Types; +import org.apache.spark.api.java.function.MapFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestPartitionValues { + @Parameters(name = "format = {0}, vectorized = {1}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.PARQUET, false}, + {FileFormat.PARQUET, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, false}, + {FileFormat.ORC, true} + }; + } + + private static final Schema SUPPORTED_PRIMITIVES = + new Schema( + required(100, "id", Types.LongType.get()), + required(101, "data", Types.StringType.get()), + required(102, "b", Types.BooleanType.get()), + required(103, "i", Types.IntegerType.get()), + required(104, "l", Types.LongType.get()), + required(105, "f", Types.FloatType.get()), + required(106, "d", Types.DoubleType.get()), + required(107, "date", Types.DateType.get()), + required(108, "ts", Types.TimestampType.withZone()), + required(110, "s", Types.StringType.get()), + required(113, "bytes", Types.BinaryType.get()), + required(114, "dec_9_0", Types.DecimalType.of(9, 0)), + required(115, "dec_11_2", Types.DecimalType.of(11, 2)), + required(116, "dec_38_10", Types.DecimalType.of(38, 10)) // spark's maximum precision + ); + + private static final Schema SIMPLE_SCHEMA = + new Schema( + optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); + + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SIMPLE_SCHEMA).identity("data").build(); + + private static SparkSession spark = null; + + @BeforeAll + public static void startSpark() { + TestPartitionValues.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestPartitionValues.spark; + TestPartitionValues.spark = null; + currentSpark.stop(); + } + + @TempDir private Path temp; + + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private boolean vectorized; + + @TestTemplate + public void testNullPartitionValue() throws Exception { + String desc = "null_part"; + File parent = new File(temp.toFile(), desc); + File location = new File(parent, "test"); + File dataFolder = new File(location, "data"); + assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue(); + + HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf()); + Table table = tables.create(SIMPLE_SCHEMA, SPEC, location.toString()); + table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format.toString()).commit(); + + List expected = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, null)); + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(location.toString()); + + Dataset result = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(location.toString()); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + + @TestTemplate + public void testReorderedColumns() throws Exception { + String desc = "reorder_columns"; + File parent = new File(temp.toFile(), desc); + File location = new File(parent, "test"); + File dataFolder = new File(location, "data"); + assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue(); + + HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf()); + Table table = tables.create(SIMPLE_SCHEMA, SPEC, location.toString()); + table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format.toString()).commit(); + + List expected = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + df.select("data", "id") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .option(SparkWriteOptions.CHECK_ORDERING, "false") + .save(location.toString()); + + Dataset result = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(location.toString()); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + + @TestTemplate + public void testReorderedColumnsNoNullability() throws Exception { + String desc = "reorder_columns_no_nullability"; + File parent = new File(temp.toFile(), desc); + File location = new File(parent, "test"); + File dataFolder = new File(location, "data"); + assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue(); + + HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf()); + Table table = tables.create(SIMPLE_SCHEMA, SPEC, location.toString()); + table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format.toString()).commit(); + + List expected = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + df.select("data", "id") + .write() + .format("iceberg") + .mode(SaveMode.Append) + .option(SparkWriteOptions.CHECK_ORDERING, "false") + .option(SparkWriteOptions.CHECK_NULLABILITY, "false") + .save(location.toString()); + + Dataset result = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(location.toString()); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + + @TestTemplate + public void testPartitionValueTypes() throws Exception { + String[] columnNames = + new String[] { + "b", "i", "l", "f", "d", "date", "ts", "s", "bytes", "dec_9_0", "dec_11_2", "dec_38_10" + }; + + HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf()); + + // create a table around the source data + String sourceLocation = temp.resolve("source_table").toString(); + Table source = tables.create(SUPPORTED_PRIMITIVES, sourceLocation); + + // write out an Avro data file with all of the data types for source data + List expected = RandomData.generateList(source.schema(), 2, 128735L); + File avroData = File.createTempFile("data", ".avro", temp.toFile()); + assertThat(avroData.delete()).isTrue(); + try (FileAppender appender = + Avro.write(Files.localOutput(avroData)).schema(source.schema()).build()) { + appender.addAll(expected); + } + + // add the Avro data file to the source table + source + .newAppend() + .appendFile( + DataFiles.builder(PartitionSpec.unpartitioned()) + .withRecordCount(10) + .withInputFile(Files.localInput(avroData)) + .build()) + .commit(); + + Dataset sourceDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(sourceLocation); + + for (String column : columnNames) { + String desc = "partition_by_" + SUPPORTED_PRIMITIVES.findType(column).toString(); + + File parent = new File(temp.toFile(), desc); + File location = new File(parent, "test"); + File dataFolder = new File(location, "data"); + assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue(); + + PartitionSpec spec = PartitionSpec.builderFor(SUPPORTED_PRIMITIVES).identity(column).build(); + + Table table = tables.create(SUPPORTED_PRIMITIVES, spec, location.toString()); + table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format.toString()).commit(); + + // disable distribution/ordering and fanout writers to preserve the original ordering + sourceDF + .write() + .format("iceberg") + .mode(SaveMode.Append) + .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") + .option(SparkWriteOptions.FANOUT_ENABLED, "false") + .save(location.toString()); + + List actual = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(location.toString()) + .collectAsList(); + + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + + for (int i = 0; i < expected.size(); i += 1) { + TestHelpers.assertEqualsSafe( + SUPPORTED_PRIMITIVES.asStruct(), expected.get(i), actual.get(i)); + } + } + } + + @TestTemplate + public void testNestedPartitionValues() throws Exception { + String[] columnNames = + new String[] { + "b", "i", "l", "f", "d", "date", "ts", "s", "bytes", "dec_9_0", "dec_11_2", "dec_38_10" + }; + + HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf()); + Schema nestedSchema = new Schema(optional(1, "nested", SUPPORTED_PRIMITIVES.asStruct())); + + // create a table around the source data + String sourceLocation = temp.resolve("source_table").toString(); + Table source = tables.create(nestedSchema, sourceLocation); + + // write out an Avro data file with all of the data types for source data + List expected = RandomData.generateList(source.schema(), 2, 128735L); + File avroData = File.createTempFile("data", ".avro", temp.toFile()); + assertThat(avroData.delete()).isTrue(); + try (FileAppender appender = + Avro.write(Files.localOutput(avroData)).schema(source.schema()).build()) { + appender.addAll(expected); + } + + // add the Avro data file to the source table + source + .newAppend() + .appendFile( + DataFiles.builder(PartitionSpec.unpartitioned()) + .withRecordCount(10) + .withInputFile(Files.localInput(avroData)) + .build()) + .commit(); + + Dataset sourceDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(sourceLocation); + + for (String column : columnNames) { + String desc = "partition_by_" + SUPPORTED_PRIMITIVES.findType(column).toString(); + + File parent = new File(temp.toFile(), desc); + File location = new File(parent, "test"); + File dataFolder = new File(location, "data"); + assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue(); + + PartitionSpec spec = + PartitionSpec.builderFor(nestedSchema).identity("nested." + column).build(); + + Table table = tables.create(nestedSchema, spec, location.toString()); + table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format.toString()).commit(); + + // disable distribution/ordering and fanout writers to preserve the original ordering + sourceDF + .write() + .format("iceberg") + .mode(SaveMode.Append) + .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") + .option(SparkWriteOptions.FANOUT_ENABLED, "false") + .save(location.toString()); + + List actual = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(location.toString()) + .collectAsList(); + + assertThat(actual).as("Number of rows should match").hasSameSizeAs(expected); + + for (int i = 0; i < expected.size(); i += 1) { + TestHelpers.assertEqualsSafe(nestedSchema.asStruct(), expected.get(i), actual.get(i)); + } + } + } + + /** + * To verify if WrappedPositionAccessor is generated against a string field within a nested field, + * rather than a Position2Accessor. Or when building the partition path, a ClassCastException is + * thrown with the message like: Cannot cast org.apache.spark.unsafe.types.UTF8String to + * java.lang.CharSequence + */ + @TestTemplate + public void testPartitionedByNestedString() throws Exception { + // schema and partition spec + Schema nestedSchema = + new Schema( + Types.NestedField.required( + 1, + "struct", + Types.StructType.of( + Types.NestedField.required(2, "string", Types.StringType.get())))); + PartitionSpec spec = PartitionSpec.builderFor(nestedSchema).identity("struct.string").build(); + + // create table + HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf()); + String baseLocation = temp.resolve("partition_by_nested_string").toString(); + tables.create(nestedSchema, spec, baseLocation); + + // input data frame + StructField[] structFields = { + new StructField( + "struct", + DataTypes.createStructType( + new StructField[] { + new StructField("string", DataTypes.StringType, false, Metadata.empty()) + }), + false, + Metadata.empty()) + }; + + List rows = Lists.newArrayList(); + rows.add(RowFactory.create(RowFactory.create("nested_string_value"))); + Dataset sourceDF = spark.createDataFrame(rows, new StructType(structFields)); + + // write into iceberg + sourceDF.write().format("iceberg").mode(SaveMode.Append).save(baseLocation); + + // verify + List actual = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(baseLocation) + .collectAsList(); + + assertThat(actual).as("Number of rows should match").hasSameSizeAs(rows); + } + + @TestTemplate + public void testReadPartitionColumn() throws Exception { + assumeThat(format).as("Temporary skip ORC").isNotEqualTo(FileFormat.ORC); + + Schema nestedSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional( + 2, + "struct", + Types.StructType.of( + Types.NestedField.optional(3, "innerId", Types.LongType.get()), + Types.NestedField.optional(4, "innerName", Types.StringType.get())))); + PartitionSpec spec = + PartitionSpec.builderFor(nestedSchema).identity("struct.innerName").build(); + + // create table + HadoopTables tables = new HadoopTables(spark.sessionState().newHadoopConf()); + String baseLocation = temp.resolve("partition_by_nested_string").toString(); + Table table = tables.create(nestedSchema, spec, baseLocation); + table.updateProperties().set(TableProperties.DEFAULT_FILE_FORMAT, format.toString()).commit(); + + // write into iceberg + MapFunction func = + value -> new ComplexRecord(value, new NestedRecord(value, "name_" + value)); + spark + .range(0, 10, 1, 1) + .map(func, Encoders.bean(ComplexRecord.class)) + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(baseLocation); + + List actual = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(baseLocation) + .select("struct.innerName") + .orderBy("struct.innerName") + .as(Encoders.STRING()) + .collectAsList(); + + assertThat(actual).as("Number of rows should match").hasSize(10); + + List inputRecords = + IntStream.range(0, 10).mapToObj(i -> "name_" + i).collect(Collectors.toList()); + assertThat(actual).as("Read object should be matched").isEqualTo(inputRecords); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java new file mode 100644 index 000000000000..aa8fe047b3f7 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java @@ -0,0 +1,83 @@ +/* + * 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.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.hadoop.HadoopTableOperations; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.PathIdentifier; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.connector.expressions.Transform; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestPathIdentifier extends TestBase { + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); + + @TempDir private Path temp; + private File tableLocation; + private PathIdentifier identifier; + private SparkCatalog sparkCatalog; + + @BeforeEach + public void before() throws IOException { + tableLocation = temp.toFile(); + identifier = new PathIdentifier(tableLocation.getAbsolutePath()); + sparkCatalog = new SparkCatalog(); + sparkCatalog.initialize("test", new CaseInsensitiveStringMap(ImmutableMap.of())); + } + + @AfterEach + public void after() { + sparkCatalog = null; + } + + @Test + public void testPathIdentifier() throws TableAlreadyExistsException, NoSuchTableException { + SparkTable table = + (SparkTable) + sparkCatalog.createTable( + identifier, SparkSchemaUtil.convert(SCHEMA), new Transform[0], ImmutableMap.of()); + + assertThat(tableLocation.getAbsolutePath()).isEqualTo(table.table().location()); + assertThat(table.table()).isInstanceOf(BaseTable.class); + assertThat(((BaseTable) table.table()).operations()).isInstanceOf(HadoopTableOperations.class); + + assertThat(table).isEqualTo(sparkCatalog.loadTable(identifier)); + assertThat(sparkCatalog.dropTable(identifier)).isTrue(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesReader.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesReader.java new file mode 100644 index 000000000000..c0db51ec8c7e --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesReader.java @@ -0,0 +1,332 @@ +/* + * 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.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.BaseScanTaskGroup; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.PositionDeletesTable; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +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.Lists; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.Pair; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.unsafe.types.UTF8String; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestPositionDeletesReader extends TestBase { + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).bucket("data", 16).build(); + + private Table table; + private DataFile dataFile1; + private DataFile dataFile2; + + @TempDir private Path temp; + + @Parameter(index = 0) + private int formatVersion; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return org.apache.iceberg.TestHelpers.V2_AND_ABOVE; + } + + @BeforeEach + public void before() throws IOException { + table = + catalog.createTable( + TableIdentifier.of("default", "test"), + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion))); + + GenericRecord record = GenericRecord.create(table.schema()); + List records1 = Lists.newArrayList(); + records1.add(record.copy("id", 29, "data", "a")); + records1.add(record.copy("id", 43, "data", "b")); + records1.add(record.copy("id", 61, "data", "c")); + records1.add(record.copy("id", 89, "data", "d")); + + List records2 = Lists.newArrayList(); + records2.add(record.copy("id", 100, "data", "e")); + records2.add(record.copy("id", 121, "data", "f")); + records2.add(record.copy("id", 122, "data", "g")); + + dataFile1 = writeDataFile(records1); + dataFile2 = writeDataFile(records2); + table.newAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); + } + + @AfterEach + public void after() { + catalog.dropTable(TableIdentifier.of("default", "test")); + } + + @TestTemplate + public void readPositionDeletesTableWithNoDeleteFiles() { + Table positionDeletesTable = + catalog.loadTable(TableIdentifier.of("default", "test", "position_deletes")); + + assertThat(positionDeletesTable.newBatchScan().planFiles()).isEmpty(); + } + + @TestTemplate + public void readPositionDeletesTableWithMultipleDeleteFiles() throws IOException { + Pair posDeletes1 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + Lists.newArrayList( + Pair.of(dataFile1.location(), 0L), Pair.of(dataFile1.location(), 1L)), + formatVersion); + + Pair posDeletes2 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + Lists.newArrayList( + Pair.of(dataFile2.location(), 2L), Pair.of(dataFile2.location(), 3L)), + formatVersion); + + DeleteFile deleteFile1 = posDeletes1.first(); + DeleteFile deleteFile2 = posDeletes2.first(); + table + .newRowDelta() + .addDeletes(deleteFile1) + .addDeletes(deleteFile2) + .validateDataFilesExist(posDeletes1.second()) + .validateDataFilesExist(posDeletes2.second()) + .commit(); + + Table positionDeletesTable = + catalog.loadTable(TableIdentifier.of("default", "test", "position_deletes")); + + List columns = + Lists.newArrayList( + MetadataColumns.DELETE_FILE_PATH.name(), + MetadataColumns.DELETE_FILE_POS.name(), + PositionDeletesTable.DELETE_FILE_PATH); + if (formatVersion >= 3) { + columns.add(PositionDeletesTable.CONTENT_OFFSET); + columns.add(PositionDeletesTable.CONTENT_SIZE_IN_BYTES); + } + + Schema projectedSchema = positionDeletesTable.schema().select(columns); + + List scanTasks = + Lists.newArrayList( + positionDeletesTable.newBatchScan().project(projectedSchema).planFiles()); + assertThat(scanTasks).hasSize(2); + + assertThat(scanTasks.get(0)).isInstanceOf(PositionDeletesScanTask.class); + PositionDeletesScanTask scanTask1 = (PositionDeletesScanTask) scanTasks.get(0); + + try (PositionDeletesRowReader reader = + new PositionDeletesRowReader( + table, + new BaseScanTaskGroup<>(null, ImmutableList.of(scanTask1)), + positionDeletesTable.schema(), + projectedSchema, + false, + true)) { + List actualRows = Lists.newArrayList(); + while (reader.next()) { + actualRows.add(reader.get().copy()); + } + + String dataFileLocation = + formatVersion >= 3 ? deleteFile1.referencedDataFile() : dataFile1.location(); + List first = + Lists.newArrayList( + UTF8String.fromString(dataFileLocation), + 0L, + UTF8String.fromString(deleteFile1.location())); + List second = + Lists.newArrayList( + UTF8String.fromString(dataFileLocation), + 1L, + UTF8String.fromString(deleteFile1.location())); + + if (formatVersion >= 3) { + first.add(deleteFile1.contentOffset()); + first.add(deleteFile1.contentSizeInBytes()); + second.add(deleteFile1.contentOffset()); + second.add(deleteFile1.contentSizeInBytes()); + } + + assertThat(internalRowsToJava(actualRows, projectedSchema)) + .hasSize(2) + .containsExactly(first.toArray(), second.toArray()); + } + + assertThat(scanTasks.get(1)).isInstanceOf(PositionDeletesScanTask.class); + PositionDeletesScanTask scanTask2 = (PositionDeletesScanTask) scanTasks.get(1); + try (PositionDeletesRowReader reader = + new PositionDeletesRowReader( + table, + new BaseScanTaskGroup<>(null, ImmutableList.of(scanTask2)), + positionDeletesTable.schema(), + projectedSchema, + false, + true)) { + List actualRows = Lists.newArrayList(); + while (reader.next()) { + actualRows.add(reader.get().copy()); + } + + String dataFileLocation = + formatVersion >= 3 ? deleteFile2.referencedDataFile() : dataFile2.location(); + List first = + Lists.newArrayList( + UTF8String.fromString(dataFileLocation), + 2L, + UTF8String.fromString(deleteFile2.location())); + List second = + Lists.newArrayList( + UTF8String.fromString(dataFileLocation), + 3L, + UTF8String.fromString(deleteFile2.location())); + + if (formatVersion >= 3) { + first.add(deleteFile2.contentOffset()); + first.add(deleteFile2.contentSizeInBytes()); + second.add(deleteFile2.contentOffset()); + second.add(deleteFile2.contentSizeInBytes()); + } + + assertThat(internalRowsToJava(actualRows, projectedSchema)) + .hasSize(2) + .containsExactly(first.toArray(), second.toArray()); + } + } + + @TestTemplate + public void readPositionDeletesTableWithDifferentColumnOrdering() throws IOException { + Pair posDeletes1 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + Lists.newArrayList( + Pair.of(dataFile1.location(), 0L), Pair.of(dataFile1.location(), 1L)), + formatVersion); + + DeleteFile deleteFile1 = posDeletes1.first(); + table + .newRowDelta() + .addDeletes(deleteFile1) + .validateDataFilesExist(posDeletes1.second()) + .commit(); + + Table positionDeletesTable = + catalog.loadTable(TableIdentifier.of("default", "test", "position_deletes")); + + // select a few fields in backwards order + Schema projectedSchema = + new Schema(MetadataColumns.DELETE_FILE_POS, MetadataColumns.DELETE_FILE_PATH); + + List scanTasks = + Lists.newArrayList( + positionDeletesTable.newBatchScan().project(projectedSchema).planFiles()); + assertThat(scanTasks).hasSize(1); + + assertThat(scanTasks.get(0)).isInstanceOf(PositionDeletesScanTask.class); + PositionDeletesScanTask scanTask1 = (PositionDeletesScanTask) scanTasks.get(0); + + try (PositionDeletesRowReader reader = + new PositionDeletesRowReader( + table, + new BaseScanTaskGroup<>(null, ImmutableList.of(scanTask1)), + positionDeletesTable.schema(), + projectedSchema, + false, + true)) { + List actualRows = Lists.newArrayList(); + while (reader.next()) { + actualRows.add(reader.get().copy()); + } + + assertThat(internalRowsToJava(actualRows, projectedSchema)) + .hasSize(2) + .containsExactly( + new Object[] {0L, UTF8String.fromString(dataFile1.location())}, + new Object[] {1L, UTF8String.fromString(dataFile1.location())}); + } + } + + private DataFile writeDataFile(List records) throws IOException { + return FileHelpers.writeDataFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + records); + } + + private List internalRowsToJava(List rows, Schema projection) { + return rows.stream().map(row -> toJava(row, projection)).collect(Collectors.toList()); + } + + private Object[] toJava(InternalRow row, Schema projection) { + Object[] values = new Object[row.numFields()]; + for (int i = 0; i < projection.columns().size(); i++) { + values[i] = row.get(i, SparkSchemaUtil.convert(projection.columns().get(i).type())); + } + return values; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java new file mode 100644 index 000000000000..7892fd65b405 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestPositionDeletesTable.java @@ -0,0 +1,1779 @@ +/* + * 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.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.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.function.BiFunction; +import java.util.stream.Collectors; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; +import org.apache.iceberg.PositionDeletesScanTask; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +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.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator; +import org.apache.iceberg.spark.ScanTaskSetManager; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkStructLike; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.DeleteFileSet; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.ScanTaskUtil; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.functions; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestPositionDeletesTable extends CatalogTestBase { + + public static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + private static final Map CATALOG_PROPS = + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "cache-enabled", "false"); + private static final List NON_PATH_COLS = + ImmutableList.of("file_path", "pos", "row", "partition", "spec_id"); + private static final List NON_PATH_V3_COLS = + ImmutableList.builder() + .addAll(NON_PATH_COLS) + .add("content_offset") + .add("content_size_in_bytes") + .build(); + + @Parameter(index = 3) + private FileFormat format; + + @Parameter(index = 4) + private int formatVersion; + + @Parameters( + name = + "catalogName = {1}, implementation = {2}, config = {3}, fileFormat = {4}, formatVersion = {5}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + CATALOG_PROPS, + FileFormat.PARQUET, + 2 + }, + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + CATALOG_PROPS, + FileFormat.AVRO, + 2 + }, + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + CATALOG_PROPS, + FileFormat.ORC, + 2 + }, + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + CATALOG_PROPS, + FileFormat.PARQUET, + 3 + }, + }; + } + + @TestTemplate + public void testNullRows() throws IOException { + String tableName = "null_rows"; + Table tab = createTable(tableName, SCHEMA, PartitionSpec.unpartitioned()); + + DataFile dFile = dataFile(tab); + tab.newAppend().appendFile(dFile).commit(); + + List> deletes = Lists.newArrayList(); + deletes.add(Pair.of(dFile.location(), 0L)); + deletes.add(Pair.of(dFile.location(), 1L)); + Pair posDeletes = + FileHelpers.writeDeleteFile( + tab, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + deletes, + formatVersion); + tab.newRowDelta().addDeletes(posDeletes.first()).commit(); + + StructLikeSet actual = actual(tableName, tab); + + List> expectedDeletes = + Lists.newArrayList( + positionDelete(dFile.location(), 0L), positionDelete(dFile.location(), 1L)); + StructLikeSet expected = expected(tab, expectedDeletes, null, posDeletes.first()); + + assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); + dropTable(tableName); + } + + @TestTemplate + public void testPartitionedTable() throws IOException { + assumeThat(formatVersion).as("DVs don't have row info in PositionDeletesTable").isEqualTo(2); + // Create table with two partitions + String tableName = "partitioned_table"; + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table tab = createTable(tableName, SCHEMA, spec); + + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + // Add position deletes for both partitions + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileB, "b"); + + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + // Select deletes from one partition + StructLikeSet actual = actual(tableName, tab, "row.data='b'"); + GenericRecord partitionB = GenericRecord.create(tab.spec().partitionType()); + partitionB.setField("data", "b"); + StructLikeSet expected = expected(tab, deletesB.first(), partitionB, deletesB.second()); + + assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); + dropTable(tableName); + } + + @TestTemplate + public void testSelect() throws IOException { + assumeThat(formatVersion).as("DVs don't have row info in PositionDeletesTable").isEqualTo(2); + // Create table with two partitions + String tableName = "select"; + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table tab = createTable(tableName, SCHEMA, spec); + + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + // Add position deletes for both partitions + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileB, "b"); + + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + // Select certain columns + Dataset df = + spark + .read() + .format("iceberg") + .load("default." + tableName + ".position_deletes") + .withColumn("input_file", functions.input_file_name()) + .select("row.id", "pos", "delete_file_path", "input_file"); + List actual = rowsToJava(df.collectAsList()); + + // Select cols from expected delete values + List expected = Lists.newArrayList(); + BiFunction, DeleteFile, Object[]> toRow = + (delete, file) -> { + int rowData = delete.get(2, GenericRecord.class).get(0, Integer.class); + long pos = delete.get(1, Long.class); + return row(rowData, pos, file.location(), file.location()); + }; + expected.addAll( + deletesA.first().stream() + .map(d -> toRow.apply(d, deletesA.second())) + .collect(Collectors.toList())); + expected.addAll( + deletesB.first().stream() + .map(d -> toRow.apply(d, deletesB.second())) + .collect(Collectors.toList())); + + // Sort and compare + Comparator comp = + (o1, o2) -> { + int result = Integer.compare((int) o1[0], (int) o2[0]); + if (result != 0) { + return result; + } else { + return ((String) o1[2]).compareTo((String) o2[2]); + } + }; + actual.sort(comp); + expected.sort(comp); + assertThat(actual) + .as("Position Delete table should contain expected rows") + .usingRecursiveComparison() + .isEqualTo(expected); + dropTable(tableName); + } + + @TestTemplate + public void testSelectWithDVs() throws IOException { + assumeThat(formatVersion).isGreaterThanOrEqualTo(3); + String tableName = "select_with_dvs"; + Table tab = createTable(tableName, SCHEMA, PartitionSpec.unpartitioned()); + + DataFile dataFileA = dataFile(tab); + DataFile dataFileB = dataFile(tab); + + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileB); + + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + // Select certain columns + Dataset df = + spark + .read() + .format("iceberg") + .load("default." + tableName + ".position_deletes") + .select( + "pos", "file_path", "delete_file_path", "content_offset", "content_size_in_bytes"); + List actual = rowsToJava(df.collectAsList()); + + // Select cols from expected delete values + List expected = Lists.newArrayList(); + BiFunction, DeleteFile, Object[]> toRow = + (delete, file) -> + row( + delete.get(1, Long.class), + file.referencedDataFile(), + file.location(), + file.contentOffset(), + ScanTaskUtil.contentSizeInBytes(file)); + + expected.addAll( + deletesA.first().stream() + .map(d -> toRow.apply(d, deletesA.second())) + .collect(Collectors.toList())); + expected.addAll( + deletesB.first().stream() + .map(d -> toRow.apply(d, deletesB.second())) + .collect(Collectors.toList())); + + // Sort by pos and file_path + Comparator comp = + (o1, o2) -> { + int result = Long.compare((long) o1[0], (long) o2[0]); + if (result != 0) { + return result; + } else { + return ((String) o1[1]).compareTo((String) o2[1]); + } + }; + + actual.sort(comp); + expected.sort(comp); + + assertThat(actual) + .as("Position Delete table should contain expected rows") + .usingRecursiveComparison() + .isEqualTo(expected); + dropTable(tableName); + } + + @TestTemplate + public void testSplitTasks() throws IOException { + String tableName = "big_table"; + Table tab = createTable(tableName, SCHEMA, PartitionSpec.unpartitioned()); + tab.updateProperties().set("read.split.target-size", "100").commit(); + int records = 500; + + GenericRecord record = GenericRecord.create(tab.schema()); + List dataRecords = Lists.newArrayList(); + for (int i = 0; i < records; i++) { + dataRecords.add(record.copy("id", i, "data", String.valueOf(i))); + } + DataFile dFile = + FileHelpers.writeDataFile( + tab, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + org.apache.iceberg.TestHelpers.Row.of(), + dataRecords); + tab.newAppend().appendFile(dFile).commit(); + + List> deletes = Lists.newArrayList(); + for (long i = 0; i < records; i++) { + deletes.add(positionDelete(tab.schema(), dFile.location(), i, (int) i, String.valueOf(i))); + } + DeleteFile posDeletes = + FileHelpers.writePosDeleteFile( + tab, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + deletes, + formatVersion); + tab.newRowDelta().addDeletes(posDeletes).commit(); + + Table deleteTable = + MetadataTableUtils.createMetadataTableInstance(tab, MetadataTableType.POSITION_DELETES); + + if (format.equals(FileFormat.AVRO)) { + assertThat(deleteTable.newBatchScan().planTasks()) + .as("Position delete scan should produce more than one split") + .hasSizeGreaterThan(1); + } else { + assertThat(deleteTable.newBatchScan().planTasks()) + .as("Position delete scan should produce one split") + .hasSize(1); + } + + StructLikeSet actual = actual(tableName, tab); + StructLikeSet expected = expected(tab, deletes, null, posDeletes); + + assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); + dropTable(tableName); + } + + @TestTemplate + public void testPartitionFilter() throws IOException { + // Create table with two partitions + String tableName = "partition_filter"; + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table tab = createTable(tableName, SCHEMA, spec); + Table deletesTab = + MetadataTableUtils.createMetadataTableInstance(tab, MetadataTableType.POSITION_DELETES); + + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + // Add position deletes for both partitions + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileA, "b"); + + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + // Prepare expected values + GenericRecord partitionRecordTemplate = GenericRecord.create(tab.spec().partitionType()); + Record partitionA = partitionRecordTemplate.copy("data", "a"); + Record partitionB = partitionRecordTemplate.copy("data", "b"); + StructLikeSet expectedA = expected(tab, deletesA.first(), partitionA, deletesA.second()); + StructLikeSet expectedB = expected(tab, deletesB.first(), partitionB, deletesB.second()); + StructLikeSet allExpected = StructLikeSet.create(deletesTab.schema().asStruct()); + allExpected.addAll(expectedA); + allExpected.addAll(expectedB); + + // Select deletes from all partitions + StructLikeSet actual = actual(tableName, tab); + assertThat(actual) + .as("Position Delete table should contain expected rows") + .isEqualTo(allExpected); + + // Select deletes from one partition + StructLikeSet actual2 = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); + + assertThat(actual2) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); + dropTable(tableName); + } + + @TestTemplate + public void testPartitionTransformFilter() throws IOException { + // Create table with two partitions + String tableName = "partition_filter"; + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).truncate("data", 1).build(); + Table tab = createTable(tableName, SCHEMA, spec); + Table deletesTable = + MetadataTableUtils.createMetadataTableInstance(tab, MetadataTableType.POSITION_DELETES); + + DataFile dataFileA = dataFile(tab, new Object[] {"aa"}, new Object[] {"a"}); + DataFile dataFileB = dataFile(tab, new Object[] {"bb"}, new Object[] {"b"}); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + // Add position deletes for both partitions + Pair>, DeleteFile> deletesA = + deleteFile(tab, dataFileA, new Object[] {"aa"}, new Object[] {"a"}); + Pair>, DeleteFile> deletesB = + deleteFile(tab, dataFileA, new Object[] {"bb"}, new Object[] {"b"}); + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + // Prepare expected values + GenericRecord partitionRecordTemplate = GenericRecord.create(tab.spec().partitionType()); + Record partitionA = partitionRecordTemplate.copy("data_trunc", "a"); + Record partitionB = partitionRecordTemplate.copy("data_trunc", "b"); + StructLikeSet expectedA = expected(tab, deletesA.first(), partitionA, deletesA.second()); + StructLikeSet expectedB = expected(tab, deletesB.first(), partitionB, deletesB.second()); + StructLikeSet allExpected = StructLikeSet.create(deletesTable.schema().asStruct()); + allExpected.addAll(expectedA); + allExpected.addAll(expectedB); + + // Select deletes from all partitions + StructLikeSet actual = actual(tableName, tab); + assertThat(actual) + .as("Position Delete table should contain expected rows") + .isEqualTo(allExpected); + + // Select deletes from one partition + StructLikeSet actual2 = actual(tableName, tab, "partition.data_trunc = 'a' AND pos >= 0"); + + assertThat(actual2) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); + dropTable(tableName); + } + + @TestTemplate + public void testPartitionEvolutionReplace() throws Exception { + // Create table with spec (data) + String tableName = "partition_evolution"; + PartitionSpec originalSpec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table tab = createTable(tableName, SCHEMA, originalSpec); + int dataSpec = tab.spec().specId(); + + // Add files with old spec (data) + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileA, "b"); + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + // Switch partition spec from (data) to (id) + tab.updateSpec().removeField("data").addField("id").commit(); + + // Add data and delete files with new spec (id) + DataFile dataFile10 = dataFile(tab, 10); + DataFile dataFile99 = dataFile(tab, 99); + tab.newAppend().appendFile(dataFile10).appendFile(dataFile99).commit(); + + Pair>, DeleteFile> deletes10 = deleteFile(tab, dataFile10, 10); + Pair>, DeleteFile> deletes99 = deleteFile(tab, dataFile10, 99); + tab.newRowDelta().addDeletes(deletes10.second()).addDeletes(deletes99.second()).commit(); + + // Query partition of old spec + GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); + Record partitionA = partitionRecordTemplate.copy("data", "a"); + StructLikeSet expectedA = + expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second()); + StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); + + // Query partition of new spec + Record partition10 = partitionRecordTemplate.copy("id", 10); + StructLikeSet expected10 = + expected(tab, deletes10.first(), partition10, tab.spec().specId(), deletes10.second()); + StructLikeSet actual10 = actual(tableName, tab, "partition.id = 10 AND pos >= 0"); + + assertThat(actual10) + .as("Position Delete table should contain expected rows") + .isEqualTo(expected10); + dropTable(tableName); + } + + @TestTemplate + public void testPartitionEvolutionAdd() throws Exception { + // Create unpartitioned table + String tableName = "partition_evolution_add"; + Table tab = createTable(tableName, SCHEMA, PartitionSpec.unpartitioned()); + int specId0 = tab.spec().specId(); + + // Add files with unpartitioned spec + DataFile dataFileUnpartitioned = dataFile(tab); + tab.newAppend().appendFile(dataFileUnpartitioned).commit(); + Pair>, DeleteFile> deletesUnpartitioned = + deleteFile(tab, dataFileUnpartitioned); + tab.newRowDelta().addDeletes(deletesUnpartitioned.second()).commit(); + + // Switch partition spec to (data) + tab.updateSpec().addField("data").commit(); + int specId1 = tab.spec().specId(); + + // Add files with new spec (data) + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileB, "b"); + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + // Select deletes from new spec (data) + GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); + Record partitionA = partitionRecordTemplate.copy("data", "a"); + StructLikeSet expectedA = + expected(tab, deletesA.first(), partitionA, specId1, deletesA.second()); + StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); + + // Select deletes from 'unpartitioned' data + Record unpartitionedRecord = partitionRecordTemplate.copy("data", null); + StructLikeSet expectedUnpartitioned = + expected( + tab, + deletesUnpartitioned.first(), + unpartitionedRecord, + specId0, + deletesUnpartitioned.second()); + StructLikeSet actualUnpartitioned = + actual(tableName, tab, "partition.data IS NULL and pos >= 0"); + + assertThat(actualUnpartitioned) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedUnpartitioned); + dropTable(tableName); + } + + @TestTemplate + public void testPartitionEvolutionRemove() throws Exception { + // Create table with spec (data) + String tableName = "partition_evolution_remove"; + PartitionSpec originalSpec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table tab = createTable(tableName, SCHEMA, originalSpec); + int specId0 = tab.spec().specId(); + + // Add files with spec (data) + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileB, "b"); + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + // Remove partition field + tab.updateSpec().removeField("data").commit(); + int specId1 = tab.spec().specId(); + + // Add unpartitioned files + DataFile dataFileUnpartitioned = dataFile(tab); + tab.newAppend().appendFile(dataFileUnpartitioned).commit(); + Pair>, DeleteFile> deletesUnpartitioned = + deleteFile(tab, dataFileUnpartitioned); + tab.newRowDelta().addDeletes(deletesUnpartitioned.second()).commit(); + + // Select deletes from (data) spec + GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); + Record partitionA = partitionRecordTemplate.copy("data", "a"); + StructLikeSet expectedA = + expected(tab, deletesA.first(), partitionA, specId0, deletesA.second()); + StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); + + // Select deletes from 'unpartitioned' spec + Record unpartitionedRecord = partitionRecordTemplate.copy("data", null); + StructLikeSet expectedUnpartitioned = + expected( + tab, + deletesUnpartitioned.first(), + unpartitionedRecord, + specId1, + deletesUnpartitioned.second()); + StructLikeSet actualUnpartitioned = + actual(tableName, tab, "partition.data IS NULL and pos >= 0"); + + assertThat(actualUnpartitioned) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedUnpartitioned); + dropTable(tableName); + } + + @TestTemplate + public void testSpecIdFilter() throws Exception { + // Create table with spec (data) + String tableName = "spec_id_filter"; + Table tab = createTable(tableName, SCHEMA, PartitionSpec.unpartitioned()); + int unpartitionedSpec = tab.spec().specId(); + + // Add data file and delete + DataFile dataFileUnpartitioned = dataFile(tab); + tab.newAppend().appendFile(dataFileUnpartitioned).commit(); + Pair>, DeleteFile> deletesUnpartitioned = + deleteFile(tab, dataFileUnpartitioned); + tab.newRowDelta().addDeletes(deletesUnpartitioned.second()).commit(); + + // Switch partition spec to (data) and add files + tab.updateSpec().addField("data").commit(); + int dataSpec = tab.spec().specId(); + + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileB, "b"); + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + // Select deletes from 'unpartitioned' + GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); + StructLikeSet expectedUnpartitioned = + expected( + tab, + deletesUnpartitioned.first(), + partitionRecordTemplate, + unpartitionedSpec, + deletesUnpartitioned.second()); + StructLikeSet actualUnpartitioned = + actual(tableName, tab, String.format("spec_id = %d", unpartitionedSpec)); + assertThat(actualUnpartitioned) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedUnpartitioned); + + // Select deletes from 'data' partition spec + StructLike partitionA = partitionRecordTemplate.copy("data", "a"); + StructLike partitionB = partitionRecordTemplate.copy("data", "b"); + StructLikeSet expected = + expected(tab, deletesA.first(), partitionA, dataSpec, deletesA.second()); + expected.addAll(expected(tab, deletesB.first(), partitionB, dataSpec, deletesB.second())); + + StructLikeSet actual = actual(tableName, tab, String.format("spec_id = %d", dataSpec)); + assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); + dropTable(tableName); + } + + @TestTemplate + public void testSchemaEvolutionAdd() throws Exception { + // Create table with original schema + String tableName = "schema_evolution_add"; + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table tab = createTable(tableName, SCHEMA, spec); + + // Add files with original schema + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileB, "b"); + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + // Add files with new schema + tab.updateSchema() + .addColumn("new_col_1", Types.IntegerType.get()) + .addColumn("new_col_2", Types.IntegerType.get()) + .commit(); + + // Add files with new schema + DataFile dataFileC = dataFile(tab, "c"); + DataFile dataFileD = dataFile(tab, "d"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + Pair>, DeleteFile> deletesC = deleteFile(tab, dataFileC, "c"); + Pair>, DeleteFile> deletesD = deleteFile(tab, dataFileD, "d"); + tab.newRowDelta().addDeletes(deletesC.second()).addDeletes(deletesD.second()).commit(); + + // Select deletes from old schema + GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); + Record partitionA = partitionRecordTemplate.copy("data", "a"); + // pad expected delete rows with null values for new columns + List> expectedDeletesA = deletesA.first(); + expectedDeletesA.forEach( + d -> { + GenericRecord nested = d.get(2, GenericRecord.class); + GenericRecord padded = GenericRecord.create(tab.schema().asStruct()); + padded.set(0, nested.get(0)); + padded.set(1, nested.get(1)); + padded.set(2, null); + padded.set(3, null); + d.set(2, padded); + }); + StructLikeSet expectedA = expected(tab, expectedDeletesA, partitionA, deletesA.second()); + StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); + + // Select deletes from new schema + Record partitionC = partitionRecordTemplate.copy("data", "c"); + StructLikeSet expectedC = expected(tab, deletesC.first(), partitionC, deletesC.second()); + StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); + + assertThat(actualC) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedC); + dropTable(tableName); + } + + @TestTemplate + public void testSchemaEvolutionRemove() throws Exception { + // Create table with original schema + String tableName = "schema_evolution_remove"; + Schema oldSchema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "new_col_1", Types.IntegerType.get()), + Types.NestedField.optional(4, "new_col_2", Types.IntegerType.get())); + PartitionSpec spec = PartitionSpec.builderFor(oldSchema).identity("data").build(); + Table tab = createTable(tableName, oldSchema, spec); + + // Add files with original schema + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileB, "b"); + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + // Add files with new schema + tab.updateSchema().deleteColumn("new_col_1").deleteColumn("new_col_2").commit(); + + // Add files with new schema + DataFile dataFileC = dataFile(tab, "c"); + DataFile dataFileD = dataFile(tab, "d"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + Pair>, DeleteFile> deletesC = deleteFile(tab, dataFileC, "c"); + Pair>, DeleteFile> deletesD = deleteFile(tab, dataFileD, "d"); + tab.newRowDelta().addDeletes(deletesC.second()).addDeletes(deletesD.second()).commit(); + + // Select deletes from old schema + GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); + Record partitionA = partitionRecordTemplate.copy("data", "a"); + // remove deleted columns from expected result + List> expectedDeletesA = deletesA.first(); + expectedDeletesA.forEach( + d -> { + GenericRecord nested = d.get(2, GenericRecord.class); + GenericRecord padded = GenericRecord.create(tab.schema().asStruct()); + padded.set(0, nested.get(0)); + padded.set(1, nested.get(1)); + d.set(2, padded); + }); + StructLikeSet expectedA = expected(tab, expectedDeletesA, partitionA, deletesA.second()); + StructLikeSet actualA = actual(tableName, tab, "partition.data = 'a' AND pos >= 0"); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); + + // Select deletes from new schema + Record partitionC = partitionRecordTemplate.copy("data", "c"); + StructLikeSet expectedC = expected(tab, deletesC.first(), partitionC, deletesC.second()); + StructLikeSet actualC = actual(tableName, tab, "partition.data = 'c' and pos >= 0"); + + assertThat(actualC) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedC); + dropTable(tableName); + } + + @TestTemplate + public void testWrite() throws IOException, NoSuchTableException { + String tableName = "test_write"; + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table tab = createTable(tableName, SCHEMA, spec); + + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + // Add position deletes for both partitions + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileB, "b"); + + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + Table posDeletesTable = + MetadataTableUtils.createMetadataTableInstance(tab, MetadataTableType.POSITION_DELETES); + String posDeletesTableName = catalogName + ".default." + tableName + ".position_deletes"; + for (String partValue : ImmutableList.of("a", "b")) { + try (CloseableIterable tasks = tasks(posDeletesTable, "data", partValue)) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(tab, fileSetID, tasks); + + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) + .load(posDeletesTableName); + + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); + scanDF + .writeTo(posDeletesTableName) + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) + .append(); + + commit(tab, posDeletesTable, fileSetID, 1); + } + } + + // Prepare expected values (without 'delete_file_path' as these have been rewritten) + GenericRecord partitionRecordTemplate = GenericRecord.create(tab.spec().partitionType()); + Record partitionA = partitionRecordTemplate.copy("data", "a"); + Record partitionB = partitionRecordTemplate.copy("data", "b"); + StructLikeSet expectedA = expected(tab, deletesA.first(), partitionA, deletesA.second(), false); + StructLikeSet expectedB = expected(tab, deletesB.first(), partitionB, deletesB.second(), false); + StructLikeSet allExpected = + StructLikeSet.create( + TypeUtil.selectNot( + posDeletesTable.schema(), ImmutableSet.of(MetadataColumns.FILE_PATH_COLUMN_ID)) + .asStruct()); + allExpected.addAll(expectedA); + allExpected.addAll(expectedB); + + // Compare values without 'delete_file_path' as these have been rewritten + StructLikeSet actual = + actual(tableName, tab, null, formatVersion >= 3 ? NON_PATH_V3_COLS : NON_PATH_COLS); + assertThat(actual) + .as("Position Delete table should contain expected rows") + .isEqualTo(allExpected); + dropTable(tableName); + } + + @TestTemplate + public void testWriteUnpartitionedNullRows() throws Exception { + String tableName = "write_null_rows"; + Table tab = createTable(tableName, SCHEMA, PartitionSpec.unpartitioned()); + + DataFile dFile = dataFile(tab); + tab.newAppend().appendFile(dFile).commit(); + + List> deletes = Lists.newArrayList(); + deletes.add(Pair.of(dFile.location(), 0L)); + deletes.add(Pair.of(dFile.location(), 1L)); + Pair posDeletes = + FileHelpers.writeDeleteFile( + tab, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + deletes, + formatVersion); + tab.newRowDelta().addDeletes(posDeletes.first()).commit(); + + Table posDeletesTable = + MetadataTableUtils.createMetadataTableInstance(tab, MetadataTableType.POSITION_DELETES); + String posDeletesTableName = catalogName + ".default." + tableName + ".position_deletes"; + try (CloseableIterable tasks = posDeletesTable.newBatchScan().planFiles()) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(tab, fileSetID, tasks); + + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) + .load(posDeletesTableName); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); + scanDF + .writeTo(posDeletesTableName) + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) + .append(); + + commit(tab, posDeletesTable, fileSetID, 1); + } + + List columns = ImmutableList.of("file_path", "pos", "row", "spec_id"); + if (formatVersion >= 3) { + columns = + ImmutableList.builder() + .addAll(columns) + .add("content_offset") + .add("content_size_in_bytes") + .build(); + } + + // Compare values without 'delete_file_path' as these have been rewritten + StructLikeSet actual = actual(tableName, tab, null, columns); + + List> expectedDeletes = + Lists.newArrayList( + positionDelete(dFile.location(), 0L), positionDelete(dFile.location(), 1L)); + StructLikeSet expected = expected(tab, expectedDeletes, null, posDeletes.first(), false); + + assertThat(actual).as("Position Delete table should contain expected rows").isEqualTo(expected); + dropTable(tableName); + } + + @TestTemplate + public void testWriteMixedRows() throws Exception { + String tableName = "write_mixed_rows"; + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table tab = createTable(tableName, SCHEMA, spec); + + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + // Add a delete file with row and without row + List> deletes = Lists.newArrayList(); + deletes.add(Pair.of(dataFileA.location(), 0L)); + deletes.add(Pair.of(dataFileA.location(), 1L)); + Pair deletesWithoutRow = + FileHelpers.writeDeleteFile( + tab, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of("a"), + deletes, + formatVersion); + + Pair>, DeleteFile> deletesWithRow = deleteFile(tab, dataFileB, "b"); + + tab.newRowDelta() + .addDeletes(deletesWithoutRow.first()) + .addDeletes(deletesWithRow.second()) + .commit(); + + // rewrite delete files + Table posDeletesTable = + MetadataTableUtils.createMetadataTableInstance(tab, MetadataTableType.POSITION_DELETES); + String posDeletesTableName = catalogName + ".default." + tableName + ".position_deletes"; + for (String partValue : ImmutableList.of("a", "b")) { + try (CloseableIterable tasks = tasks(posDeletesTable, "data", partValue)) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(tab, fileSetID, tasks); + + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .load(posDeletesTableName); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); + scanDF + .writeTo(posDeletesTableName) + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) + .append(); + + commit(tab, posDeletesTable, fileSetID, 1); + } + } + + // Compare values without 'delete_file_path' as these have been rewritten + StructLikeSet actual = + actual(tableName, tab, null, formatVersion >= 3 ? NON_PATH_V3_COLS : NON_PATH_COLS); + + // Prepare expected values + GenericRecord partitionRecordTemplate = GenericRecord.create(tab.spec().partitionType()); + Record partitionA = partitionRecordTemplate.copy("data", "a"); + Record partitionB = partitionRecordTemplate.copy("data", "b"); + StructLikeSet allExpected = + StructLikeSet.create( + TypeUtil.selectNot( + posDeletesTable.schema(), ImmutableSet.of(MetadataColumns.FILE_PATH_COLUMN_ID)) + .asStruct()); + allExpected.addAll( + expected( + tab, + Lists.newArrayList( + positionDelete(dataFileA.location(), 0L), positionDelete(dataFileA.location(), 1L)), + partitionA, + deletesWithoutRow.first(), + false)); + allExpected.addAll( + expected(tab, deletesWithRow.first(), partitionB, deletesWithRow.second(), false)); + + assertThat(actual) + .as("Position Delete table should contain expected rows") + .isEqualTo(allExpected); + dropTable(tableName); + } + + @TestTemplate + public void testWritePartitionEvolutionAdd() throws Exception { + // Create unpartitioned table + String tableName = "write_partition_evolution_add"; + Table tab = createTable(tableName, SCHEMA, PartitionSpec.unpartitioned()); + int specId0 = tab.spec().specId(); + + // Add files with unpartitioned spec + DataFile dataFileUnpartitioned = dataFile(tab); + tab.newAppend().appendFile(dataFileUnpartitioned).commit(); + Pair>, DeleteFile> deletesUnpartitioned = + deleteFile(tab, dataFileUnpartitioned); + tab.newRowDelta().addDeletes(deletesUnpartitioned.second()).commit(); + + // Switch partition spec to (data) + tab.updateSpec().addField("data").commit(); + int specId1 = tab.spec().specId(); + + // Add files with new spec (data) + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileB, "b"); + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + Table posDeletesTable = + MetadataTableUtils.createMetadataTableInstance(tab, MetadataTableType.POSITION_DELETES); + String posDeletesTableName = catalogName + ".default." + tableName + ".position_deletes"; + + // Read/write back unpartitioned data + try (CloseableIterable tasks = + posDeletesTable.newBatchScan().filter(Expressions.isNull("partition.data")).planFiles()) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(tab, fileSetID, tasks); + + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) + .load(posDeletesTableName); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); + scanDF + .writeTo(posDeletesTableName) + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) + .append(); + + commit(tab, posDeletesTable, fileSetID, 1); + } + + // Select deletes from unpartitioned data + // Compare values without 'delete_file_path' as these have been rewritten + GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); + Record unpartitionedRecord = partitionRecordTemplate.copy("data", null); + StructLikeSet expectedUnpartitioned = + expected( + tab, + deletesUnpartitioned.first(), + unpartitionedRecord, + specId0, + deletesUnpartitioned.second(), + false); + StructLikeSet actualUnpartitioned = + actual( + tableName, + tab, + "partition.data IS NULL", + formatVersion >= 3 ? NON_PATH_V3_COLS : NON_PATH_COLS); + assertThat(actualUnpartitioned) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedUnpartitioned); + + // Read/write back new partition spec (data) + for (String partValue : ImmutableList.of("a", "b")) { + try (CloseableIterable tasks = tasks(posDeletesTable, "data", partValue)) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(tab, fileSetID, tasks); + + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) + .load(posDeletesTableName); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); + scanDF + .writeTo(posDeletesTableName) + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) + .append(); + + // commit the rewrite + commit(tab, posDeletesTable, fileSetID, 1); + } + } + + // Select deletes from new spec (data) + Record partitionA = partitionRecordTemplate.copy("data", "a"); + Record partitionB = partitionRecordTemplate.copy("data", "b"); + StructLikeSet expectedAll = + StructLikeSet.create( + TypeUtil.selectNot( + posDeletesTable.schema(), ImmutableSet.of(MetadataColumns.FILE_PATH_COLUMN_ID)) + .asStruct()); + expectedAll.addAll( + expected(tab, deletesA.first(), partitionA, specId1, deletesA.second(), false)); + expectedAll.addAll( + expected(tab, deletesB.first(), partitionB, specId1, deletesB.second(), false)); + StructLikeSet actualAll = + actual( + tableName, + tab, + "partition.data = 'a' OR partition.data = 'b'", + formatVersion >= 3 ? NON_PATH_V3_COLS : NON_PATH_COLS); + assertThat(actualAll) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedAll); + dropTable(tableName); + } + + @TestTemplate + public void testWritePartitionEvolutionDisallowed() throws Exception { + // Create unpartitioned table + String tableName = "write_partition_evolution_write"; + Table tab = createTable(tableName, SCHEMA, PartitionSpec.unpartitioned()); + + // Add files with unpartitioned spec + DataFile dataFileUnpartitioned = dataFile(tab); + tab.newAppend().appendFile(dataFileUnpartitioned).commit(); + Pair>, DeleteFile> deletesUnpartitioned = + deleteFile(tab, dataFileUnpartitioned); + tab.newRowDelta().addDeletes(deletesUnpartitioned.second()).commit(); + + Table posDeletesTable = + MetadataTableUtils.createMetadataTableInstance(tab, MetadataTableType.POSITION_DELETES); + String posDeletesTableName = catalogName + ".default." + tableName + ".position_deletes"; + + Dataset scanDF; + String fileSetID = UUID.randomUUID().toString(); + try (CloseableIterable tasks = posDeletesTable.newBatchScan().planFiles()) { + stageTask(tab, fileSetID, tasks); + + scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) + .load(posDeletesTableName); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); + + // Add partition field to render the original un-partitioned dataset un-commitable + tab.updateSpec().addField("data").commit(); + } + + scanDF + .writeTo(posDeletesTableName) + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) + .append(); + + scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) + .load(posDeletesTableName); + assertThat(Arrays.asList(scanDF.columns()).contains("partition")); + + dropTable(tableName); + dropTable(posDeletesTableName); + } + + @TestTemplate + public void testWriteSchemaEvolutionAdd() throws Exception { + // Create table with original schema + String tableName = "write_schema_evolution_add"; + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table tab = createTable(tableName, SCHEMA, spec); + + // Add files with original schema + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileB, "b"); + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + // Add files with new schema + tab.updateSchema() + .addColumn("new_col_1", Types.IntegerType.get()) + .addColumn("new_col_2", Types.IntegerType.get()) + .commit(); + + // Add files with new schema + DataFile dataFileC = dataFile(tab, "c"); + DataFile dataFileD = dataFile(tab, "d"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + Pair>, DeleteFile> deletesC = deleteFile(tab, dataFileC, "c"); + Pair>, DeleteFile> deletesD = deleteFile(tab, dataFileD, "d"); + tab.newRowDelta().addDeletes(deletesC.second()).addDeletes(deletesD.second()).commit(); + + Table posDeletesTable = + MetadataTableUtils.createMetadataTableInstance(tab, MetadataTableType.POSITION_DELETES); + String posDeletesTableName = catalogName + ".default." + tableName + ".position_deletes"; + + // rewrite files of old schema + try (CloseableIterable tasks = tasks(posDeletesTable, "data", "a")) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(tab, fileSetID, tasks); + + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) + .load(posDeletesTableName); + + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); + scanDF + .writeTo(posDeletesTableName) + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) + .append(); + + commit(tab, posDeletesTable, fileSetID, 1); + } + + // Select deletes from old schema + GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); + Record partitionA = partitionRecordTemplate.copy("data", "a"); + // pad expected delete rows with null values for new columns + List> expectedDeletesA = deletesA.first(); + expectedDeletesA.forEach( + d -> { + GenericRecord nested = d.get(2, GenericRecord.class); + GenericRecord padded = GenericRecord.create(tab.schema().asStruct()); + padded.set(0, nested.get(0)); + padded.set(1, nested.get(1)); + padded.set(2, null); + padded.set(3, null); + d.set(2, padded); + }); + StructLikeSet expectedA = expected(tab, expectedDeletesA, partitionA, deletesA.second(), false); + StructLikeSet actualA = + actual( + tableName, + tab, + "partition.data = 'a'", + formatVersion >= 3 ? NON_PATH_V3_COLS : NON_PATH_COLS); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); + + // rewrite files of new schema + try (CloseableIterable tasks = tasks(posDeletesTable, "data", "c")) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(tab, fileSetID, tasks); + + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) + .load(posDeletesTableName); + + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); + scanDF + .writeTo(posDeletesTableName) + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) + .append(); + + commit(tab, posDeletesTable, fileSetID, 1); + } + + // Select deletes from new schema + Record partitionC = partitionRecordTemplate.copy("data", "c"); + StructLikeSet expectedC = expected(tab, deletesC.first(), partitionC, deletesC.second(), false); + StructLikeSet actualC = + actual( + tableName, + tab, + "partition.data = 'c'", + formatVersion >= 3 ? NON_PATH_V3_COLS : NON_PATH_COLS); + + assertThat(actualC) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedC); + dropTable(tableName); + } + + @TestTemplate + public void testWriteSchemaEvolutionRemove() throws Exception { + // Create table with original schema + String tableName = "write_schema_evolution_remove"; + Schema oldSchema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "new_col_1", Types.IntegerType.get()), + Types.NestedField.optional(4, "new_col_2", Types.IntegerType.get())); + PartitionSpec spec = PartitionSpec.builderFor(oldSchema).identity("data").build(); + Table tab = createTable(tableName, oldSchema, spec); + + // Add files with original schema + DataFile dataFileA = dataFile(tab, "a"); + DataFile dataFileB = dataFile(tab, "b"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + Pair>, DeleteFile> deletesB = deleteFile(tab, dataFileB, "b"); + tab.newRowDelta().addDeletes(deletesA.second()).addDeletes(deletesB.second()).commit(); + + // Add files with new schema + tab.updateSchema().deleteColumn("new_col_1").deleteColumn("new_col_2").commit(); + + // Add files with new schema + DataFile dataFileC = dataFile(tab, "c"); + DataFile dataFileD = dataFile(tab, "d"); + tab.newAppend().appendFile(dataFileA).appendFile(dataFileB).commit(); + + Pair>, DeleteFile> deletesC = deleteFile(tab, dataFileC, "c"); + Pair>, DeleteFile> deletesD = deleteFile(tab, dataFileD, "d"); + tab.newRowDelta().addDeletes(deletesC.second()).addDeletes(deletesD.second()).commit(); + + Table posDeletesTable = + MetadataTableUtils.createMetadataTableInstance(tab, MetadataTableType.POSITION_DELETES); + String posDeletesTableName = catalogName + ".default." + tableName + ".position_deletes"; + + // rewrite files + for (String partValue : ImmutableList.of("a", "b", "c", "d")) { + try (CloseableIterable tasks = tasks(posDeletesTable, "data", partValue)) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(tab, fileSetID, tasks); + + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .option(SparkReadOptions.FILE_OPEN_COST, Integer.MAX_VALUE) + .load(posDeletesTableName); + assertThat(scanDF.javaRDD().getNumPartitions()).isEqualTo(1); + scanDF + .writeTo(posDeletesTableName) + .option(SparkWriteOptions.REWRITTEN_FILE_SCAN_TASK_SET_ID, fileSetID) + .append(); + + commit(tab, posDeletesTable, fileSetID, 1); + } + } + + // Select deletes from old schema + GenericRecord partitionRecordTemplate = GenericRecord.create(Partitioning.partitionType(tab)); + Record partitionA = partitionRecordTemplate.copy("data", "a"); + // remove deleted columns from expected result + List> expectedDeletesA = deletesA.first(); + expectedDeletesA.forEach( + d -> { + GenericRecord nested = d.get(2, GenericRecord.class); + GenericRecord padded = GenericRecord.create(tab.schema().asStruct()); + padded.set(0, nested.get(0)); + padded.set(1, nested.get(1)); + d.set(2, padded); + }); + StructLikeSet expectedA = expected(tab, expectedDeletesA, partitionA, deletesA.second(), false); + StructLikeSet actualA = + actual( + tableName, + tab, + "partition.data = 'a'", + formatVersion >= 3 ? NON_PATH_V3_COLS : NON_PATH_COLS); + assertThat(actualA) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedA); + + // Select deletes from new schema + Record partitionC = partitionRecordTemplate.copy("data", "c"); + StructLikeSet expectedC = expected(tab, deletesC.first(), partitionC, deletesC.second(), false); + StructLikeSet actualC = + actual( + tableName, + tab, + "partition.data = 'c'", + formatVersion >= 3 ? NON_PATH_V3_COLS : NON_PATH_COLS); + + assertThat(actualC) + .as("Position Delete table should contain expected rows") + .isEqualTo(expectedC); + dropTable(tableName); + } + + @TestTemplate + public void testNormalWritesNotAllowed() throws IOException { + String tableName = "test_normal_write_not_allowed"; + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table tab = createTable(tableName, SCHEMA, spec); + + DataFile dataFileA = dataFile(tab, "a"); + tab.newAppend().appendFile(dataFileA).commit(); + + Pair>, DeleteFile> deletesA = deleteFile(tab, dataFileA, "a"); + tab.newRowDelta().addDeletes(deletesA.second()).commit(); + + String posDeletesTableName = catalogName + ".default." + tableName + ".position_deletes"; + + Dataset scanDF = spark.read().format("iceberg").load(posDeletesTableName); + + assertThatThrownBy(() -> scanDF.writeTo(posDeletesTableName).append()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Can only write to " + posDeletesTableName + " via actions"); + + dropTable(tableName); + } + + private StructLikeSet actual(String tableName, Table table) { + return actual(tableName, table, null, null); + } + + private StructLikeSet actual(String tableName, Table table, String filter) { + return actual(tableName, table, filter, null); + } + + private StructLikeSet actual(String tableName, Table table, String filter, List cols) { + Dataset df = + spark + .read() + .format("iceberg") + .load(catalogName + ".default." + tableName + ".position_deletes"); + if (filter != null) { + df = df.filter(filter); + } + if (cols != null) { + df = df.select(cols.get(0), cols.subList(1, cols.size()).toArray(new String[0])); + } + Table deletesTable = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); + Types.StructType projection = deletesTable.schema().asStruct(); + if (cols != null) { + projection = + Types.StructType.of( + projection.fields().stream() + .filter(f -> cols.contains(f.name())) + .collect(Collectors.toList())); + } + + Types.StructType finalProjection = projection; + StructLikeSet set = StructLikeSet.create(projection); + df.collectAsList() + .forEach( + row -> { + SparkStructLike rowWrapper = new SparkStructLike(finalProjection); + set.add(rowWrapper.wrap(row)); + }); + + return set; + } + + protected Table createTable(String name, Schema schema, PartitionSpec spec) { + Map properties = + ImmutableMap.of( + TableProperties.FORMAT_VERSION, + String.valueOf(formatVersion), + TableProperties.DEFAULT_FILE_FORMAT, + format.toString()); + return validationCatalog.createTable( + TableIdentifier.of("default", name), schema, spec, properties); + } + + protected void dropTable(String name) { + validationCatalog.dropTable(TableIdentifier.of("default", name), false); + } + + private PositionDelete positionDelete(CharSequence path, Long position) { + PositionDelete posDelete = PositionDelete.create(); + posDelete.set(path, position, null); + return posDelete; + } + + private PositionDelete positionDelete( + Schema tableSchema, CharSequence path, Long position, Object... values) { + PositionDelete posDelete = PositionDelete.create(); + GenericRecord nested = GenericRecord.create(tableSchema); + for (int i = 0; i < values.length; i++) { + nested.set(i, values[i]); + } + posDelete.set(path, position, nested); + return posDelete; + } + + private StructLikeSet expected( + Table testTable, + List> deletes, + StructLike partitionStruct, + int specId, + DeleteFile deleteFile) { + return expected(testTable, deletes, partitionStruct, specId, deleteFile, true); + } + + private StructLikeSet expected( + Table testTable, + List> deletes, + StructLike partitionStruct, + int specId, + DeleteFile deleteFile, + boolean includeDeleteFilePath) { + Table deletesTable = + MetadataTableUtils.createMetadataTableInstance( + testTable, MetadataTableType.POSITION_DELETES); + Types.StructType posDeleteSchema = deletesTable.schema().asStruct(); + // Do not compare file paths + if (!includeDeleteFilePath) { + posDeleteSchema = + TypeUtil.selectNot( + deletesTable.schema(), ImmutableSet.of(MetadataColumns.FILE_PATH_COLUMN_ID)) + .asStruct(); + } + final Types.StructType finalSchema = posDeleteSchema; + StructLikeSet set = StructLikeSet.create(posDeleteSchema); + deletes.stream() + .map( + p -> { + GenericRecord record = GenericRecord.create(finalSchema); + record.setField("file_path", p.path()); + record.setField("pos", p.pos()); + record.setField("row", formatVersion >= 3 ? null : p.row()); + if (partitionStruct != null) { + record.setField("partition", partitionStruct); + } + record.setField("spec_id", specId); + if (includeDeleteFilePath) { + record.setField("delete_file_path", deleteFile.location()); + } + if (formatVersion >= 3) { + record.setField("content_offset", deleteFile.contentOffset()); + record.setField( + "content_size_in_bytes", ScanTaskUtil.contentSizeInBytes(deleteFile)); + } + return record; + }) + .forEach(set::add); + return set; + } + + private StructLikeSet expected( + Table testTable, + List> deletes, + StructLike partitionStruct, + DeleteFile deleteFile) { + return expected(testTable, deletes, partitionStruct, testTable.spec().specId(), deleteFile); + } + + private StructLikeSet expected( + Table testTable, + List> deletes, + StructLike partitionStruct, + DeleteFile deleteFile, + boolean includeDeleteFilePath) { + return expected( + testTable, + deletes, + partitionStruct, + testTable.spec().specId(), + deleteFile, + includeDeleteFilePath); + } + + private DataFile dataFile(Table tab, Object... partValues) throws IOException { + return dataFile(tab, partValues, partValues); + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + private DataFile dataFile(Table tab, Object[] partDataValues, Object[] partFieldValues) + throws IOException { + GenericRecord record = GenericRecord.create(tab.schema()); + List partitionFieldNames = + tab.spec().fields().stream().map(PartitionField::name).collect(Collectors.toList()); + int idIndex = partitionFieldNames.indexOf("id"); + int dataIndex = partitionFieldNames.indexOf("data"); + Integer idPartition = idIndex != -1 ? (Integer) partDataValues[idIndex] : null; + String dataPartition = dataIndex != -1 ? (String) partDataValues[dataIndex] : null; + + // fill columns with partition source fields, or preset values + List records = + Lists.newArrayList( + record.copy( + "id", + idPartition != null ? idPartition : 29, + "data", + dataPartition != null ? dataPartition : "c"), + record.copy( + "id", + idPartition != null ? idPartition : 43, + "data", + dataPartition != null ? dataPartition : "k"), + record.copy( + "id", + idPartition != null ? idPartition : 61, + "data", + dataPartition != null ? dataPartition : "r"), + record.copy( + "id", + idPartition != null ? idPartition : 89, + "data", + dataPartition != null ? dataPartition : "t")); + + // fill remaining columns with incremental values + List cols = tab.schema().columns(); + if (cols.size() > 2) { + for (int i = 2; i < cols.size(); i++) { + final int pos = i; + records.forEach(r -> r.set(pos, pos)); + } + } + + TestHelpers.Row partitionInfo = TestHelpers.Row.of(partFieldValues); + return FileHelpers.writeDataFile( + tab, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + partitionInfo, + records); + } + + private Pair>, DeleteFile> deleteFile( + Table tab, DataFile dataFile, Object... partValues) throws IOException { + return deleteFile(tab, dataFile, partValues, partValues); + } + + private Pair>, DeleteFile> deleteFile( + Table tab, DataFile dataFile, Object[] partDataValues, Object[] partFieldValues) + throws IOException { + List partFields = tab.spec().fields(); + List partitionFieldNames = + partFields.stream().map(PartitionField::name).collect(Collectors.toList()); + int idIndex = partitionFieldNames.indexOf("id"); + int dataIndex = partitionFieldNames.indexOf("data"); + Integer idPartition = idIndex != -1 ? (Integer) partDataValues[idIndex] : null; + String dataPartition = dataIndex != -1 ? (String) partDataValues[dataIndex] : null; + + // fill columns with partition source fields, or preset values + List> deletes = + Lists.newArrayList( + positionDelete( + tab.schema(), + dataFile.location(), + 0L, + idPartition != null ? idPartition : 29, + dataPartition != null ? dataPartition : "c"), + positionDelete( + tab.schema(), + dataFile.location(), + 1L, + idPartition != null ? idPartition : 61, + dataPartition != null ? dataPartition : "r")); + + // fill remaining columns with incremental values + List cols = tab.schema().columns(); + if (cols.size() > 2) { + for (int i = 2; i < cols.size(); i++) { + final int pos = i; + deletes.forEach(d -> d.get(2, GenericRecord.class).set(pos, pos)); + } + } + + TestHelpers.Row partitionInfo = TestHelpers.Row.of(partFieldValues); + + DeleteFile deleteFile = + FileHelpers.writePosDeleteFile( + tab, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + partitionInfo, + deletes, + formatVersion); + return Pair.of(deletes, deleteFile); + } + + private void stageTask( + Table tab, String fileSetID, CloseableIterable tasks) { + ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + taskSetManager.stageTasks(tab, fileSetID, Lists.newArrayList(tasks)); + } + + private void commit( + Table baseTab, + Table posDeletesTable, + String fileSetID, + int expectedSourceFiles, + int expectedTargetFiles) { + PositionDeletesRewriteCoordinator rewriteCoordinator = PositionDeletesRewriteCoordinator.get(); + Set rewrittenFiles = + ScanTaskSetManager.get().fetchTasks(posDeletesTable, fileSetID).stream() + .map(t -> ((PositionDeletesScanTask) t).file()) + .collect(Collectors.toCollection(DeleteFileSet::create)); + Set addedFiles = rewriteCoordinator.fetchNewFiles(posDeletesTable, fileSetID); + + // Assert new files and old files are equal in number but different in paths + assertThat(rewrittenFiles).hasSize(expectedSourceFiles); + assertThat(addedFiles).hasSize(expectedTargetFiles); + + List sortedAddedFiles = + addedFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); + List sortedRewrittenFiles = + rewrittenFiles.stream().map(ContentFile::location).sorted().collect(Collectors.toList()); + assertThat(sortedRewrittenFiles) + .as("Lists should not be the same") + .isNotEqualTo(sortedAddedFiles); + + baseTab + .newRewrite() + .rewriteFiles(ImmutableSet.of(), rewrittenFiles, ImmutableSet.of(), addedFiles) + .commit(); + } + + private void commit(Table baseTab, Table posDeletesTable, String fileSetID, int expectedFiles) { + commit(baseTab, posDeletesTable, fileSetID, expectedFiles, expectedFiles); + } + + private CloseableIterable tasks( + Table posDeletesTable, String partitionColumn, String partitionValue) { + + Expression filter = Expressions.equal("partition." + partitionColumn, partitionValue); + CloseableIterable files = posDeletesTable.newBatchScan().filter(filter).planFiles(); + + // take care of fail to filter in some partition evolution cases + return CloseableIterable.filter( + files, + t -> { + StructLike filePartition = ((PositionDeletesScanTask) t).partition(); + String filePartitionValue = filePartition.get(0, String.class); + return filePartitionValue != null && filePartitionValue.equals(partitionValue); + }); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java new file mode 100644 index 000000000000..9cf8e435270c --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java @@ -0,0 +1,654 @@ +/* + * 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.avro.Schema.Type.UNION; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.within; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +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.Maps; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TestReadProjection { + @Parameter(index = 0) + protected FileFormat format; + + protected abstract Record writeAndRead( + String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException; + + @TempDir protected Path temp; + + @TestTemplate + public void testFullProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + Record record = GenericRecord.create(schema); + record.setField("id", 34L); + record.setField("data", "test"); + + Record projected = writeAndRead("full_projection", schema, schema, record); + + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + + int cmp = + Comparators.charSequences().compare("test", (CharSequence) projected.getField("data")); + assertThat(cmp).as("Should contain the correct data value").isEqualTo(0); + } + + @TestTemplate + public void testReorderedFullProjection() throws Exception { + // Assume.assumeTrue( + // "Spark's Parquet read support does not support reordered columns", + // !format.equalsIgnoreCase("parquet")); + + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + Record record = GenericRecord.create(schema); + record.setField("id", 34L); + record.setField("data", "test"); + + Schema reordered = + new Schema( + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.required(0, "id", Types.LongType.get())); + + Record projected = writeAndRead("reordered_full_projection", schema, reordered, record); + + assertThat(projected.get(0)) + .asString() + .as("Should contain the correct 0 value") + .isEqualTo("test"); + assertThat(projected.get(1)).as("Should contain the correct 1 value").isEqualTo(34L); + } + + @TestTemplate + public void testReorderedProjection() throws Exception { + // Assume.assumeTrue( + // "Spark's Parquet read support does not support reordered columns", + // !format.equalsIgnoreCase("parquet")); + + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + Record record = GenericRecord.create(schema); + record.setField("id", 34L); + record.setField("data", "test"); + + Schema reordered = + new Schema( + Types.NestedField.optional(2, "missing_1", Types.StringType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.optional(3, "missing_2", Types.LongType.get())); + + Record projected = writeAndRead("reordered_projection", schema, reordered, record); + + assertThat(projected.get(0)).as("Should contain the correct 0 value").isNull(); + assertThat(projected.get(1).toString()) + .as("Should contain the correct 1 value") + .isEqualTo("test"); + assertThat(projected.get(2)).as("Should contain the correct 2 value").isNull(); + } + + @TestTemplate + @SuppressWarnings("checkstyle:AssertThatThrownByWithMessageCheck") + public void testEmptyProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + Record record = GenericRecord.create(schema); + record.setField("id", 34L); + record.setField("data", "test"); + + Record projected = writeAndRead("empty_projection", schema, schema.select(), record); + + assertThat(projected).as("Should read a non-null record").isNotNull(); + // this is expected because there are no values + // no check on the underlying error msg as it might be missing based on the JDK version + assertThatThrownBy(() -> projected.get(0)).isInstanceOf(ArrayIndexOutOfBoundsException.class); + } + + @TestTemplate + public void testBasicProjection() throws Exception { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + Record record = GenericRecord.create(writeSchema); + record.setField("id", 34L); + record.setField("data", "test"); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + Record projected = writeAndRead("basic_projection_id", writeSchema, idOnly, record); + assertThat(projected.getField("data")).as("Should not project data").isNull(); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + + Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); + + projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + + int cmp = + Comparators.charSequences().compare("test", (CharSequence) projected.getField("data")); + assertThat(cmp).as("Should contain the correct data value").isEqualTo(0); + } + + @TestTemplate + public void testRename() throws Exception { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + Record record = GenericRecord.create(writeSchema); + record.setField("id", 34L); + record.setField("data", "test"); + + Schema readSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "renamed", Types.StringType.get())); + + Record projected = writeAndRead("project_and_rename", writeSchema, readSchema, record); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + + int cmp = + Comparators.charSequences().compare("test", (CharSequence) projected.getField("renamed")); + assertThat(cmp).as("Should contain the correct data/renamed value").isEqualTo(0); + } + + @TestTemplate + public void testNestedStructProjection() throws Exception { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 3, + "location", + Types.StructType.of( + Types.NestedField.required(1, "lat", Types.FloatType.get()), + Types.NestedField.required(2, "long", Types.FloatType.get())))); + + Record record = GenericRecord.create(writeSchema); + record.setField("id", 34L); + Record location = GenericRecord.create(writeSchema.findType("location").asStructType()); + location.setField("lat", 52.995143f); + location.setField("long", -1.539054f); + record.setField("location", location); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + Record projected = writeAndRead("id_only", writeSchema, idOnly, record); + Record projectedLocation = (Record) projected.getField("location"); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + assertThat(projectedLocation).as("Should not project location").isNull(); + + Schema latOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); + + projected = writeAndRead("latitude_only", writeSchema, latOnly, record); + projectedLocation = (Record) projected.getField("location"); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("location")).as("Should project location").isNotNull(); + assertThat(projectedLocation.getField("long")).as("Should not project longitude").isNull(); + assertThat((float) projectedLocation.getField("lat")) + .as("Should project latitude") + .isCloseTo(52.995143f, within(0.000001f)); + + Schema longOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); + + projected = writeAndRead("longitude_only", writeSchema, longOnly, record); + projectedLocation = (Record) projected.getField("location"); + + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("location")).as("Should project location").isNotNull(); + assertThat(projectedLocation.getField("lat")).as("Should not project latitude").isNull(); + assertThat((float) projectedLocation.getField("long")) + .as("Should project longitude") + .isCloseTo(-1.539054f, within(0.000001f)); + + Schema locationOnly = writeSchema.select("location"); + projected = writeAndRead("location_only", writeSchema, locationOnly, record); + projectedLocation = (Record) projected.getField("location"); + + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("location")).as("Should project location").isNotNull(); + assertThat((float) projectedLocation.getField("lat")) + .as("Should project latitude") + .isCloseTo(52.995143f, within(0.000001f)); + assertThat((float) projectedLocation.getField("long")) + .as("Should project longitude") + .isCloseTo(-1.539054f, within(0.000001f)); + } + + @TestTemplate + public void testMapProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 5, + "properties", + Types.MapType.ofOptional(6, 7, Types.StringType.get(), Types.StringType.get()))); + + Map properties = ImmutableMap.of("a", "A", "b", "B"); + + Record record = GenericRecord.create(writeSchema); + record.setField("id", 34L); + record.setField("properties", properties); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + Record projected = writeAndRead("id_only", writeSchema, idOnly, record); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + assertThat(projected.getField("properties")).as("Should not project properties map").isNull(); + + Schema keyOnly = writeSchema.select("properties.key"); + projected = writeAndRead("key_only", writeSchema, keyOnly, record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(toStringMap((Map) projected.getField("properties"))) + .as("Should project entire map") + .isEqualTo(properties); + + Schema valueOnly = writeSchema.select("properties.value"); + projected = writeAndRead("value_only", writeSchema, valueOnly, record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(toStringMap((Map) projected.getField("properties"))) + .as("Should project entire map") + .isEqualTo(properties); + + Schema mapOnly = writeSchema.select("properties"); + projected = writeAndRead("map_only", writeSchema, mapOnly, record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(toStringMap((Map) projected.getField("properties"))) + .as("Should project entire map") + .isEqualTo(properties); + } + + private Map toStringMap(Map map) { + Map stringMap = Maps.newHashMap(); + for (Map.Entry entry : map.entrySet()) { + if (entry.getValue() instanceof CharSequence) { + stringMap.put(entry.getKey().toString(), entry.getValue().toString()); + } else { + stringMap.put(entry.getKey().toString(), entry.getValue()); + } + } + return stringMap; + } + + @TestTemplate + public void testMapOfStructsProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 5, + "locations", + Types.MapType.ofOptional( + 6, + 7, + Types.StringType.get(), + Types.StructType.of( + Types.NestedField.required(1, "lat", Types.FloatType.get()), + Types.NestedField.required(2, "long", Types.FloatType.get()))))); + + Record record = GenericRecord.create(writeSchema); + record.setField("id", 34L); + Record l1 = GenericRecord.create(writeSchema.findType("locations.value").asStructType()); + l1.setField("lat", 53.992811f); + l1.setField("long", -1.542616f); + Record l2 = GenericRecord.create(l1.struct()); + l2.setField("lat", 52.995143f); + l2.setField("long", -1.539054f); + record.setField("locations", ImmutableMap.of("L1", l1, "L2", l2)); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + Record projected = writeAndRead("id_only", writeSchema, idOnly, record); + assertThat(34L) + .as("Should contain the correct id value") + .isEqualTo((long) projected.getField("id")); + assertThat(projected.getField("locations")).as("Should not project locations map").isNull(); + + projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(toStringMap((Map) projected.getField("locations"))) + .as("Should project locations map") + .isEqualTo(record.getField("locations")); + + projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + + Map locations = toStringMap((Map) projected.getField("locations")); + assertThat(locations).isNotNull().containsKeys("L1", "L2"); + + Record projectedL1 = (Record) locations.get("L1"); + assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat((float) projectedL1.getField("lat")) + .as("L1 should contain lat") + .isCloseTo(53.992811f, within(0.000001f)); + assertThat(projectedL1.getField("long")).as("L1 should not contain long").isNull(); + + Record projectedL2 = (Record) locations.get("L2"); + assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat((float) projectedL2.getField("lat")) + .as("L2 should contain lat") + .isCloseTo(52.995143f, within(0.000001f)); + assertThat(projectedL2.getField("long")).as("L2 should not contain long").isNull(); + + projected = + writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + + locations = toStringMap((Map) projected.getField("locations")); + assertThat(locations).isNotNull().containsKeys("L1", "L2"); + + projectedL1 = (Record) locations.get("L1"); + assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat(projectedL1.getField("lat")).as("L1 should not contain lat").isNull(); + assertThat((float) projectedL1.getField("long")) + .as("L1 should contain long") + .isCloseTo(-1.542616f, within(0.000001f)); + + projectedL2 = (Record) locations.get("L2"); + assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat(projectedL2.getField("lat")).as("L2 should not contain lat").isNull(); + assertThat((float) projectedL2.getField("long")) + .as("L2 should contain long") + .isCloseTo(-1.539054f, within(0.000001f)); + + Schema latitiudeRenamed = + new Schema( + Types.NestedField.optional( + 5, + "locations", + Types.MapType.ofOptional( + 6, + 7, + Types.StringType.get(), + Types.StructType.of( + Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); + + projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + locations = toStringMap((Map) projected.getField("locations")); + assertThat(locations).isNotNull().containsKeys("L1", "L2"); + + projectedL1 = (Record) locations.get("L1"); + assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat((float) projectedL1.getField("latitude")) + .as("L1 should contain latitude") + .isCloseTo(53.992811f, within(0.000001f)); + assertThat(projectedL1.getField("lat")).as("L1 should not contain lat").isNull(); + assertThat(projectedL1.getField("long")).as("L1 should not contain long").isNull(); + + projectedL2 = (Record) locations.get("L2"); + assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat((float) projectedL2.getField("latitude")) + .as("L2 should contain latitude") + .isCloseTo(52.995143f, within(0.000001f)); + assertThat(projectedL2.getField("lat")).as("L2 should not contain lat").isNull(); + assertThat(projectedL2.getField("long")).as("L2 should not contain long").isNull(); + } + + @TestTemplate + public void testListProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 10, "values", Types.ListType.ofOptional(11, Types.LongType.get()))); + + List values = ImmutableList.of(56L, 57L, 58L); + + Record record = GenericRecord.create(writeSchema); + record.setField("id", 34L); + record.setField("values", values); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + Record projected = writeAndRead("id_only", writeSchema, idOnly, record); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + assertThat(projected.getField("values")).as("Should not project values list").isNull(); + + Schema elementOnly = writeSchema.select("values.element"); + projected = writeAndRead("element_only", writeSchema, elementOnly, record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("values")).as("Should project entire list").isEqualTo(values); + + Schema listOnly = writeSchema.select("values"); + projected = writeAndRead("list_only", writeSchema, listOnly, record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("values")).as("Should project entire list").isEqualTo(values); + } + + @TestTemplate + @SuppressWarnings("unchecked") + public void testListOfStructsProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 22, + "points", + Types.ListType.ofOptional( + 21, + Types.StructType.of( + Types.NestedField.required(19, "x", Types.IntegerType.get()), + Types.NestedField.optional(18, "y", Types.IntegerType.get()))))); + + Record record = GenericRecord.create(writeSchema); + record.setField("id", 34L); + Record p1 = GenericRecord.create(writeSchema.findType("points.element").asStructType()); + p1.setField("x", 1); + p1.setField("y", 2); + Record p2 = GenericRecord.create(p1.struct()); + p2.setField("x", 3); + p2.setField("y", null); + record.setField("points", ImmutableList.of(p1, p2)); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + Record projected = writeAndRead("id_only", writeSchema, idOnly, record); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + assertThat(projected.getField("points")).as("Should not project points list").isNull(); + + projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("points")) + .as("Should project points list") + .isEqualTo(record.getField("points")); + + projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("points")).as("Should project points list").isNotNull(); + + List points = (List) projected.getField("points"); + assertThat(points).as("Should read 2 points").hasSize(2); + assertThat(points) + .element(0) + .satisfies( + projectedP1 -> { + assertThat((int) projectedP1.getField("x")).isEqualTo(1); + assertThat(projectedP1.getField("y")).isNull(); + }); + assertThat(points) + .element(1) + .satisfies( + projectedP2 -> { + assertThat((int) projectedP2.getField("x")).isEqualTo(3); + assertThat(projectedP2.getField("y")).isNull(); + }); + + projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("points")).as("Should project points list").isNotNull(); + + points = (List) projected.getField("points"); + assertThat(points).as("Should read 2 points").hasSize(2); + assertThat(points) + .element(0) + .satisfies( + projectedP1 -> { + assertThat(projectedP1.getField("x")).isNull(); + assertThat((int) projectedP1.getField("y")).isEqualTo(2); + }); + assertThat(points) + .element(1) + .satisfies( + projectedP2 -> { + assertThat(projectedP2.getField("x")).isNull(); + assertThat(projectedP2.getField("y")).isNull(); + }); + + Schema yRenamed = + new Schema( + Types.NestedField.optional( + 22, + "points", + Types.ListType.ofOptional( + 21, + Types.StructType.of( + Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); + + projected = writeAndRead("y_renamed", writeSchema, yRenamed, record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("points")).as("Should project points list").isNotNull(); + + points = (List) projected.getField("points"); + assertThat(points).as("Should read 2 points").hasSize(2); + assertThat(points) + .element(0) + .satisfies( + projectedP1 -> { + assertThat(projectedP1.getField("x")).isNull(); + assertThat(projectedP1.getField("y")).isNull(); + assertThat((int) projectedP1.getField("z")).isEqualTo(2); + }); + assertThat(points) + .element(1) + .satisfies( + projectedP2 -> { + assertThat(projectedP2.getField("x")).isNull(); + assertThat(projectedP2.getField("y")).isNull(); + assertThat(projectedP2.getField("z")).isNull(); + }); + + Schema zAdded = + new Schema( + Types.NestedField.optional( + 22, + "points", + Types.ListType.ofOptional( + 21, + Types.StructType.of( + Types.NestedField.required(19, "x", Types.IntegerType.get()), + Types.NestedField.optional(18, "y", Types.IntegerType.get()), + Types.NestedField.optional(20, "z", Types.IntegerType.get()))))); + + projected = writeAndRead("z_added", writeSchema, zAdded, record); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("points")).as("Should project points list").isNotNull(); + + points = (List) projected.getField("points"); + assertThat(points).as("Should read 2 points").hasSize(2); + assertThat(points) + .element(0) + .satisfies( + projectedP1 -> { + assertThat((int) projectedP1.getField("x")).isEqualTo(1); + assertThat((int) projectedP1.getField("y")).isEqualTo(2); + assertThat(projectedP1.getField("z")).isNull(); + }); + assertThat(points) + .element(1) + .satisfies( + projectedP2 -> { + assertThat((int) projectedP2.getField("x")).isEqualTo(3); + assertThat(projectedP2.getField("y")).isNull(); + assertThat(projectedP2.getField("z")).isNull(); + }); + } + + private static org.apache.avro.Schema fromOption(org.apache.avro.Schema schema) { + Preconditions.checkArgument( + schema.getType() == UNION, "Expected union schema but was passed: %s", schema); + Preconditions.checkArgument( + schema.getTypes().size() == 2, "Expected optional schema, but was passed: %s", schema); + if (schema.getTypes().get(0).getType() == org.apache.avro.Schema.Type.NULL) { + return schema.getTypes().get(1); + } else { + return schema.getTypes().get(0); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java new file mode 100644 index 000000000000..5dbfc7fa6c0f --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java @@ -0,0 +1,310 @@ +/* + * 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.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRequiredDistributionAndOrdering extends CatalogTestBase { + + @AfterEach + public void dropTestTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testDefaultLocalSort() throws NoSuchTableException { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " + + "USING iceberg " + + "PARTITIONED BY (c3)", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(4, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(5, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(6, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(7, "BBBBBBBBBB", "A")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); + + // should insert a local sort by partition columns by default + inputDF.writeTo(tableName).append(); + + assertEquals( + "Row count must match", + ImmutableList.of(row(7L)), + sql("SELECT count(*) FROM %s", tableName)); + } + + @TestTemplate + public void testPartitionColumnsArePrependedForRangeDistribution() throws NoSuchTableException { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " + + "USING iceberg " + + "PARTITIONED BY (c3)", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(4, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(5, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(6, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(7, "BBBBBBBBBB", "A")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); + + Table table = validationCatalog.loadTable(tableIdent); + + // should automatically prepend partition columns to the ordering + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE) + .commit(); + table.replaceSortOrder().asc("c1").asc("c2").commit(); + inputDF.writeTo(tableName).append(); + + assertEquals( + "Row count must match", + ImmutableList.of(row(7L)), + sql("SELECT count(*) FROM %s", tableName)); + } + + @TestTemplate + public void testSortOrderIncludesPartitionColumns() throws NoSuchTableException { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " + + "USING iceberg " + + "PARTITIONED BY (c3)", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(4, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(5, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(6, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(7, "BBBBBBBBBB", "A")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); + + Table table = validationCatalog.loadTable(tableIdent); + + // should succeed with a correct sort order + table.replaceSortOrder().asc("c3").asc("c1").asc("c2").commit(); + inputDF.writeTo(tableName).append(); + + assertEquals( + "Row count must match", + ImmutableList.of(row(7L)), + sql("SELECT count(*) FROM %s", tableName)); + } + + @TestTemplate + public void testDisabledDistributionAndOrdering() { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " + + "USING iceberg " + + "PARTITIONED BY (c3)", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(4, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(5, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(6, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(7, "BBBBBBBBBB", "A")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); + + // should fail if ordering is disabled + assertThatThrownBy( + () -> + inputDF + .writeTo(tableName) + .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") + .option(SparkWriteOptions.FANOUT_ENABLED, "false") + .append()) + .cause() + .isInstanceOf(IllegalStateException.class) + .hasMessageStartingWith( + "Incoming records violate the writer assumption that records are clustered by spec " + + "and by partition within each spec. Either cluster the incoming records or switch to fanout writers."); + } + + @TestTemplate + public void testHashDistribution() throws NoSuchTableException { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " + + "USING iceberg " + + "PARTITIONED BY (c3)", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(4, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(5, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(6, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(7, "BBBBBBBBBB", "A")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); + + Table table = validationCatalog.loadTable(tableIdent); + + // should automatically prepend partition columns to the local ordering after hash distribution + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, TableProperties.WRITE_DISTRIBUTION_MODE_HASH) + .commit(); + table.replaceSortOrder().asc("c1").asc("c2").commit(); + inputDF.writeTo(tableName).append(); + + assertEquals( + "Row count must match", + ImmutableList.of(row(7L)), + sql("SELECT count(*) FROM %s", tableName)); + } + + @TestTemplate + public void testSortBucketTransformsWithoutExtensions() throws NoSuchTableException { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING, c3 STRING) " + + "USING iceberg " + + "PARTITIONED BY (bucket(2, c1))", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBB", "B"), + new ThreeColumnRecord(3, "BBBB", "B"), + new ThreeColumnRecord(4, "BBBB", "B")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); + + inputDF.writeTo(tableName).append(); + + List expected = + ImmutableList.of( + row(1, null, "A"), row(2, "BBBB", "B"), row(3, "BBBB", "B"), row(4, "BBBB", "B")); + + assertEquals("Rows must match", expected, sql("SELECT * FROM %s ORDER BY c1", tableName)); + } + + @TestTemplate + public void testRangeDistributionWithQuotedColumnsNames() throws NoSuchTableException { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING, `c.3` STRING) " + + "USING iceberg " + + "PARTITIONED BY (`c.3`)", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(4, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(5, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(6, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(7, "BBBBBBBBBB", "A")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = + ds.selectExpr("c1", "c2", "c3 as `c.3`").coalesce(1).sortWithinPartitions("c1"); + + Table table = validationCatalog.loadTable(tableIdent); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE) + .commit(); + table.replaceSortOrder().asc("c1").asc("c2").commit(); + inputDF.writeTo(tableName).append(); + + assertEquals( + "Row count must match", + ImmutableList.of(row(7L)), + sql("SELECT count(*) FROM %s", tableName)); + } + + @TestTemplate + public void testHashDistributionWithQuotedColumnsNames() throws NoSuchTableException { + sql( + "CREATE TABLE %s (c1 INT, c2 STRING, `c``3` STRING) " + + "USING iceberg " + + "PARTITIONED BY (`c``3`)", + tableName); + + List data = + ImmutableList.of( + new ThreeColumnRecord(1, null, "A"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(3, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(4, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(5, "BBBBBBBBBB", "A"), + new ThreeColumnRecord(6, "BBBBBBBBBB", "B"), + new ThreeColumnRecord(7, "BBBBBBBBBB", "A")); + Dataset ds = spark.createDataFrame(data, ThreeColumnRecord.class); + Dataset inputDF = + ds.selectExpr("c1", "c2", "c3 as `c``3`").coalesce(1).sortWithinPartitions("c1"); + + Table table = validationCatalog.loadTable(tableIdent); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, TableProperties.WRITE_DISTRIBUTION_MODE_HASH) + .commit(); + table.replaceSortOrder().asc("c1").asc("c2").commit(); + inputDF.writeTo(tableName).append(); + + assertEquals( + "Row count must match", + ImmutableList.of(row(7L)), + sql("SELECT count(*) FROM %s", tableName)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java new file mode 100644 index 000000000000..e7346e270f38 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestRuntimeFiltering.java @@ -0,0 +1,510 @@ +/* + * 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.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Set; +import org.apache.commons.lang3.StringUtils; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRuntimeFiltering extends TestBaseWithCatalog { + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, planningMode = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + LOCAL + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + DISTRIBUTED + } + }; + } + + @Parameter(index = 3) + private PlanningMode planningMode; + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS dim"); + } + + @TestTemplate + public void testIdentityPartitionedTable() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (date)", + tableName); + configurePlanningMode(planningMode); + + Dataset df = + spark + .range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset dimDF = + spark.range(1, 10).withColumn("date", expr("DATE '1970-01-02'")).select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = + String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.date = d.date AND d.id = 1 ORDER BY id", + tableName); + + assertQueryContainsRuntimeFilter(query); + + deleteNotMatchingFiles(Expressions.equal("date", 1), 3); + + assertEquals( + "Should have expected rows", + sql("SELECT * FROM %s WHERE date = DATE '1970-01-02' ORDER BY id", tableName), + sql(query)); + } + + @TestTemplate + public void testBucketedTable() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (bucket(8, id))", + tableName); + configurePlanningMode(planningMode); + + Dataset df = + spark + .range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset dimDF = + spark.range(1, 2).withColumn("date", expr("DATE '1970-01-02'")).select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = + String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.id = d.id AND d.date = DATE '1970-01-02' ORDER BY date", + tableName); + + assertQueryContainsRuntimeFilter(query); + + deleteNotMatchingFiles(Expressions.equal("id", 1), 7); + + assertEquals( + "Should have expected rows", + sql("SELECT * FROM %s WHERE id = 1 ORDER BY date", tableName), + sql(query)); + } + + @TestTemplate + public void testRenamedSourceColumnTable() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (bucket(8, id))", + tableName); + configurePlanningMode(planningMode); + + Dataset df = + spark + .range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset dimDF = + spark.range(1, 2).withColumn("date", expr("DATE '1970-01-02'")).select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + sql("ALTER TABLE %s RENAME COLUMN id TO row_id", tableName); + + String query = + String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.row_id = d.id AND d.date = DATE '1970-01-02' ORDER BY date", + tableName); + + assertQueryContainsRuntimeFilter(query); + + deleteNotMatchingFiles(Expressions.equal("row_id", 1), 7); + + assertEquals( + "Should have expected rows", + sql("SELECT * FROM %s WHERE row_id = 1 ORDER BY date", tableName), + sql(query)); + } + + @TestTemplate + public void testMultipleRuntimeFilters() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (data, bucket(8, id))", + tableName); + configurePlanningMode(planningMode); + + Dataset df = + spark + .range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE, data STRING) USING parquet"); + Dataset dimDF = + spark + .range(1, 2) + .withColumn("date", expr("DATE '1970-01-02'")) + .withColumn("data", expr("'1970-01-02'")) + .select("id", "date", "data"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = + String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.id = d.id AND f.data = d.data AND d.date = DATE '1970-01-02'", + tableName); + + assertQueryContainsRuntimeFilters(query, 2, "Query should have 2 runtime filters"); + + deleteNotMatchingFiles(Expressions.equal("id", 1), 31); + + assertEquals( + "Should have expected rows", + sql("SELECT * FROM %s WHERE id = 1 AND data = '1970-01-02'", tableName), + sql(query)); + } + + @TestTemplate + public void testCaseSensitivityOfRuntimeFilters() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (data, bucket(8, id))", + tableName); + configurePlanningMode(planningMode); + + Dataset df = + spark + .range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE, data STRING) USING parquet"); + Dataset dimDF = + spark + .range(1, 2) + .withColumn("date", expr("DATE '1970-01-02'")) + .withColumn("data", expr("'1970-01-02'")) + .select("id", "date", "data"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String caseInsensitiveQuery = + String.format( + "select f.* from %s F join dim d ON f.Id = d.iD and f.DaTa = d.dAtA and d.dAtE = date '1970-01-02'", + tableName); + + assertQueryContainsRuntimeFilters( + caseInsensitiveQuery, 2, "Query should have 2 runtime filters"); + + deleteNotMatchingFiles(Expressions.equal("id", 1), 31); + + assertEquals( + "Should have expected rows", + sql("SELECT * FROM %s WHERE id = 1 AND data = '1970-01-02'", tableName), + sql(caseInsensitiveQuery)); + } + + @TestTemplate + public void testBucketedTableWithMultipleSpecs() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) USING iceberg", + tableName); + configurePlanningMode(planningMode); + + Dataset df1 = + spark + .range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 2 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df1.coalesce(1).writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + table.updateSpec().addField(Expressions.bucket("id", 8)).commit(); + + sql("REFRESH TABLE %s", tableName); + + Dataset df2 = + spark + .range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df2.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset dimDF = + spark.range(1, 2).withColumn("date", expr("DATE '1970-01-02'")).select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = + String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.id = d.id AND d.date = DATE '1970-01-02' ORDER BY date", + tableName); + + assertQueryContainsRuntimeFilter(query); + + deleteNotMatchingFiles(Expressions.equal("id", 1), 7); + + assertEquals( + "Should have expected rows", + sql("SELECT * FROM %s WHERE id = 1 ORDER BY date", tableName), + sql(query)); + } + + @TestTemplate + public void testSourceColumnWithDots() throws NoSuchTableException { + sql( + "CREATE TABLE %s (`i.d` BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (bucket(8, `i.d`))", + tableName); + configurePlanningMode(planningMode); + + Dataset df = + spark + .range(1, 100) + .withColumnRenamed("id", "i.d") + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(`i.d` % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("`i.d`", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("SELECT * FROM %s WHERE `i.d` = 1", tableName); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset dimDF = + spark.range(1, 2).withColumn("date", expr("DATE '1970-01-02'")).select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = + String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.`i.d` = d.id AND d.date = DATE '1970-01-02' ORDER BY date", + tableName); + + assertQueryContainsRuntimeFilter(query); + + deleteNotMatchingFiles(Expressions.equal("i.d", 1), 7); + + sql(query); + + assertEquals( + "Should have expected rows", + sql("SELECT * FROM %s WHERE `i.d` = 1 ORDER BY date", tableName), + sql(query)); + } + + @TestTemplate + public void testSourceColumnWithBackticks() throws NoSuchTableException { + sql( + "CREATE TABLE %s (`i``d` BIGINT, data STRING, date DATE, ts TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (bucket(8, `i``d`))", + tableName); + configurePlanningMode(planningMode); + + Dataset df = + spark + .range(1, 100) + .withColumnRenamed("id", "i`d") + .withColumn( + "date", date_add(expr("DATE '1970-01-01'"), expr("CAST(`i``d` % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("`i``d`", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset dimDF = + spark.range(1, 2).withColumn("date", expr("DATE '1970-01-02'")).select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = + String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.`i``d` = d.id AND d.date = DATE '1970-01-02' ORDER BY date", + tableName); + + assertQueryContainsRuntimeFilter(query); + + deleteNotMatchingFiles(Expressions.equal("i`d", 1), 7); + + assertEquals( + "Should have expected rows", + sql("SELECT * FROM %s WHERE `i``d` = 1 ORDER BY date", tableName), + sql(query)); + } + + @TestTemplate + public void testUnpartitionedTable() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT, data STRING, date DATE, ts TIMESTAMP) USING iceberg", + tableName); + configurePlanningMode(planningMode); + + Dataset df = + spark + .range(1, 100) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id % 4 AS INT)"))) + .withColumn("ts", expr("TO_TIMESTAMP(date)")) + .withColumn("data", expr("CAST(date AS STRING)")) + .select("id", "data", "date", "ts"); + + df.coalesce(1).writeTo(tableName).append(); + + sql("CREATE TABLE dim (id BIGINT, date DATE) USING parquet"); + Dataset dimDF = + spark.range(1, 2).withColumn("date", expr("DATE '1970-01-02'")).select("id", "date"); + dimDF.coalesce(1).write().mode("append").insertInto("dim"); + + String query = + String.format( + "SELECT f.* FROM %s f JOIN dim d ON f.id = d.id AND d.date = DATE '1970-01-02' ORDER BY date", + tableName); + + assertQueryContainsNoRuntimeFilter(query); + + assertEquals( + "Should have expected rows", + sql("SELECT * FROM %s WHERE id = 1 ORDER BY date", tableName), + sql(query)); + } + + private void assertQueryContainsRuntimeFilter(String query) { + assertQueryContainsRuntimeFilters(query, 1, "Query should have 1 runtime filter"); + } + + private void assertQueryContainsNoRuntimeFilter(String query) { + assertQueryContainsRuntimeFilters(query, 0, "Query should have no runtime filters"); + } + + private void assertQueryContainsRuntimeFilters( + String query, int expectedFilterCount, String errorMessage) { + List output = spark.sql("EXPLAIN EXTENDED " + query).collectAsList(); + String plan = output.get(0).getString(0); + int actualFilterCount = StringUtils.countMatches(plan, "dynamicpruningexpression"); + assertThat(actualFilterCount).as(errorMessage).isEqualTo(expectedFilterCount); + } + + // delete files that don't match the filter to ensure dynamic filtering works and only required + // files are read + private void deleteNotMatchingFiles(Expression filter, int expectedDeletedFileCount) { + Table table = validationCatalog.loadTable(tableIdent); + FileIO io = table.io(); + + Set matchingFileLocations = Sets.newHashSet(); + try (CloseableIterable files = table.newScan().filter(filter).planFiles()) { + for (FileScanTask file : files) { + String path = file.file().location(); + matchingFileLocations.add(path); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + Set deletedFileLocations = Sets.newHashSet(); + try (CloseableIterable files = table.newScan().planFiles()) { + for (FileScanTask file : files) { + String path = file.file().location(); + if (!matchingFileLocations.contains(path)) { + io.deleteFile(path); + deletedFileLocations.add(path); + } + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + assertThat(deletedFileLocations) + .as("Deleted unexpected number of files") + .hasSize(expectedDeletedFileCount); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java new file mode 100644 index 000000000000..11865db7fce5 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java @@ -0,0 +1,590 @@ +/* + * 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.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.net.InetAddress; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSnapshotSelection { + + @Parameters(name = "properties = {0}") + public static Object[] parameters() { + return new Object[][] { + { + ImmutableMap.of( + TableProperties.DATA_PLANNING_MODE, LOCAL.modeName(), + TableProperties.DELETE_PLANNING_MODE, LOCAL.modeName()) + }, + { + ImmutableMap.of( + TableProperties.DATA_PLANNING_MODE, DISTRIBUTED.modeName(), + TableProperties.DELETE_PLANNING_MODE, DISTRIBUTED.modeName()) + } + }; + } + + private static final Configuration CONF = new Configuration(); + private static final Schema SCHEMA = + new Schema( + optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); + + @TempDir private Path temp; + + private static SparkSession spark = null; + + @Parameter(index = 0) + private Map properties; + + @BeforeAll + public static void startSpark() { + TestSnapshotSelection.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestSnapshotSelection.spark; + TestSnapshotSelection.spark = null; + currentSpark.stop(); + } + + @TestTemplate + public void testSnapshotSelectionById() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, properties, tableLocation); + + // produce the first snapshot + List firstBatchRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset firstDf = spark.createDataFrame(firstBatchRecords, SimpleRecord.class); + firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + // produce the second snapshot + List secondBatchRecords = + Lists.newArrayList( + new SimpleRecord(4, "d"), new SimpleRecord(5, "e"), new SimpleRecord(6, "f")); + Dataset secondDf = spark.createDataFrame(secondBatchRecords, SimpleRecord.class); + secondDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + assertThat(table.snapshots()).as("Expected 2 snapshots").hasSize(2); + + // verify records in the current snapshot + Dataset currentSnapshotResult = spark.read().format("iceberg").load(tableLocation); + List currentSnapshotRecords = + currentSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(firstBatchRecords); + expectedRecords.addAll(secondBatchRecords); + assertThat(currentSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); + + // verify records in the previous snapshot + Snapshot currentSnapshot = table.currentSnapshot(); + Long parentSnapshotId = currentSnapshot.parentId(); + Dataset previousSnapshotResult = + spark.read().format("iceberg").option("snapshot-id", parentSnapshotId).load(tableLocation); + List previousSnapshotRecords = + previousSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(previousSnapshotRecords) + .as("Previous snapshot rows should match") + .isEqualTo(firstBatchRecords); + } + + @TestTemplate + public void testSnapshotSelectionByTimestamp() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, properties, tableLocation); + + // produce the first snapshot + List firstBatchRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset firstDf = spark.createDataFrame(firstBatchRecords, SimpleRecord.class); + firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + // remember the time when the first snapshot was valid + long firstSnapshotTimestamp = System.currentTimeMillis(); + + // produce the second snapshot + List secondBatchRecords = + Lists.newArrayList( + new SimpleRecord(4, "d"), new SimpleRecord(5, "e"), new SimpleRecord(6, "f")); + Dataset secondDf = spark.createDataFrame(secondBatchRecords, SimpleRecord.class); + secondDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + assertThat(table.snapshots()).as("Expected 2 snapshots").hasSize(2); + + // verify records in the current snapshot + Dataset currentSnapshotResult = spark.read().format("iceberg").load(tableLocation); + List currentSnapshotRecords = + currentSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(firstBatchRecords); + expectedRecords.addAll(secondBatchRecords); + assertThat(currentSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); + + // verify records in the previous snapshot + Dataset previousSnapshotResult = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.AS_OF_TIMESTAMP, firstSnapshotTimestamp) + .load(tableLocation); + List previousSnapshotRecords = + previousSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(previousSnapshotRecords) + .as("Previous snapshot rows should match") + .isEqualTo(firstBatchRecords); + } + + @TestTemplate + public void testSnapshotSelectionByInvalidSnapshotId() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + 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) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find snapshot with ID -10"); + } + + @TestTemplate + public void testSnapshotSelectionByInvalidTimestamp() { + long timestamp = System.currentTimeMillis(); + + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + tables.create(SCHEMA, spec, properties, tableLocation); + + assertThatThrownBy( + () -> + spark + .read() + .format("iceberg") + .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) + .load(tableLocation)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot find a snapshot older than"); + } + + @TestTemplate + public void testSnapshotSelectionBySnapshotIdAndTimestamp() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, properties, tableLocation); + + List firstBatchRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset firstDf = spark.createDataFrame(firstBatchRecords, SimpleRecord.class); + firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + long timestamp = System.currentTimeMillis(); + long snapshotId = table.currentSnapshot().snapshotId(); + + assertThatThrownBy( + () -> + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SNAPSHOT_ID, snapshotId) + .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) + .load(tableLocation)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Can specify only one of snapshot-id") + .hasMessageContaining("as-of-timestamp") + .hasMessageContaining("branch") + .hasMessageContaining("tag"); + } + + @TestTemplate + public void testSnapshotSelectionByTag() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, properties, tableLocation); + + // produce the first snapshot + List firstBatchRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset firstDf = spark.createDataFrame(firstBatchRecords, SimpleRecord.class); + firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + table.manageSnapshots().createTag("tag", table.currentSnapshot().snapshotId()).commit(); + + // produce the second snapshot + List secondBatchRecords = + Lists.newArrayList( + new SimpleRecord(4, "d"), new SimpleRecord(5, "e"), new SimpleRecord(6, "f")); + Dataset secondDf = spark.createDataFrame(secondBatchRecords, SimpleRecord.class); + secondDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + // verify records in the current snapshot by tag + Dataset currentSnapshotResult = + spark.read().format("iceberg").option("tag", "tag").load(tableLocation); + List currentSnapshotRecords = + currentSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(firstBatchRecords); + assertThat(currentSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); + } + + @TestTemplate + public void testSnapshotSelectionByBranch() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, properties, tableLocation); + + // produce the first snapshot + List firstBatchRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset firstDf = spark.createDataFrame(firstBatchRecords, SimpleRecord.class); + firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); + + // produce the second snapshot + List secondBatchRecords = + Lists.newArrayList( + new SimpleRecord(4, "d"), new SimpleRecord(5, "e"), new SimpleRecord(6, "f")); + Dataset secondDf = spark.createDataFrame(secondBatchRecords, SimpleRecord.class); + secondDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + // verify records in the current snapshot by branch + Dataset currentSnapshotResult = + spark.read().format("iceberg").option("branch", "branch").load(tableLocation); + List currentSnapshotRecords = + currentSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(firstBatchRecords); + assertThat(currentSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); + } + + @TestTemplate + public void testSnapshotSelectionByBranchAndTagFails() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, properties, tableLocation); + + // produce the first snapshot + List firstBatchRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset firstDf = spark.createDataFrame(firstBatchRecords, SimpleRecord.class); + firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); + table.manageSnapshots().createTag("tag", table.currentSnapshot().snapshotId()).commit(); + + assertThatThrownBy( + () -> + spark + .read() + .format("iceberg") + .option(SparkReadOptions.TAG, "tag") + .option(SparkReadOptions.BRANCH, "branch") + .load(tableLocation) + .show()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Can specify only one of snapshot-id"); + } + + @TestTemplate + public void testSnapshotSelectionByTimestampAndBranchOrTagFails() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, properties, tableLocation); + + List firstBatchRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset firstDf = spark.createDataFrame(firstBatchRecords, SimpleRecord.class); + firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + long timestamp = System.currentTimeMillis(); + table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); + table.manageSnapshots().createTag("tag", table.currentSnapshot().snapshotId()).commit(); + + assertThatThrownBy( + () -> + spark + .read() + .format("iceberg") + .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) + .option(SparkReadOptions.BRANCH, "branch") + .load(tableLocation) + .show()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Can specify only one of snapshot-id"); + + assertThatThrownBy( + () -> + spark + .read() + .format("iceberg") + .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) + .option(SparkReadOptions.TAG, "tag") + .load(tableLocation) + .show()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Can specify only one of snapshot-id"); + } + + @TestTemplate + public void testSnapshotSelectionByBranchWithSchemaChange() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, properties, tableLocation); + + // produce the first snapshot + List firstBatchRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset firstDf = spark.createDataFrame(firstBatchRecords, SimpleRecord.class); + firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); + + Dataset branchSnapshotResult = + spark.read().format("iceberg").option("branch", "branch").load(tableLocation); + List branchSnapshotRecords = + branchSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(firstBatchRecords); + assertThat(branchSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); + + // Deleting a column to indicate schema change + table.updateSchema().deleteColumn("data").commit(); + + // The data should not have the deleted column + assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .orderBy("id") + .collectAsList()) + .containsExactly(RowFactory.create(1), RowFactory.create(2), RowFactory.create(3)); + + // re-introducing the column should not let the data re-appear + table.updateSchema().addColumn("data", Types.StringType.get()).commit(); + + assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .orderBy("id") + .as(Encoders.bean(SimpleRecord.class)) + .collectAsList()) + .containsExactly( + new SimpleRecord(1, null), new SimpleRecord(2, null), new SimpleRecord(3, null)); + } + + @TestTemplate + public void testWritingToBranchAfterSchemaChange() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, properties, tableLocation); + + // produce the first snapshot + List firstBatchRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset firstDf = spark.createDataFrame(firstBatchRecords, SimpleRecord.class); + firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); + + Dataset branchSnapshotResult = + spark.read().format("iceberg").option("branch", "branch").load(tableLocation); + List branchSnapshotRecords = + branchSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(firstBatchRecords); + assertThat(branchSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); + + // Deleting and add a new column of the same type to indicate schema change + table.updateSchema().deleteColumn("data").addColumn("zip", Types.IntegerType.get()).commit(); + + assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .orderBy("id") + .collectAsList()) + .containsExactly( + RowFactory.create(1, null), RowFactory.create(2, null), RowFactory.create(3, null)); + + // writing new records into the branch should work with the new column + List records = + Lists.newArrayList( + RowFactory.create(4, 12345), RowFactory.create(5, 54321), RowFactory.create(6, 67890)); + + Dataset dataFrame = + spark.createDataFrame( + records, + SparkSchemaUtil.convert( + new Schema( + optional(1, "id", Types.IntegerType.get()), + optional(2, "zip", Types.IntegerType.get())))); + dataFrame + .select("id", "zip") + .write() + .format("iceberg") + .option("branch", "branch") + .mode("append") + .save(tableLocation); + + assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .collectAsList()) + .hasSize(6) + .contains( + RowFactory.create(1, null), RowFactory.create(2, null), RowFactory.create(3, null)) + .containsAll(records); + } + + @TestTemplate + public void testSnapshotSelectionByTagWithSchemaChange() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, properties, tableLocation); + + // produce the first snapshot + List firstBatchRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset firstDf = spark.createDataFrame(firstBatchRecords, SimpleRecord.class); + firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + table.manageSnapshots().createTag("tag", table.currentSnapshot().snapshotId()).commit(); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(firstBatchRecords); + + Dataset tagSnapshotResult = + spark.read().format("iceberg").option("tag", "tag").load(tableLocation); + List tagSnapshotRecords = + tagSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(tagSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); + + // Deleting a column to indicate schema change + table.updateSchema().deleteColumn("data").commit(); + + // 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); + List deletedColumnTagSnapshotRecords = + deletedColumnTagSnapshotResult + .orderBy("id") + .as(Encoders.bean(SimpleRecord.class)) + .collectAsList(); + assertThat(deletedColumnTagSnapshotRecords) + .as("Current snapshot rows should match") + .isEqualTo(expectedRecords); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java new file mode 100644 index 000000000000..6cbe3914dcf8 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkAggregates.java @@ -0,0 +1,76 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkAggregates; +import org.apache.spark.sql.connector.expressions.FieldReference; +import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.expressions.aggregate.Count; +import org.apache.spark.sql.connector.expressions.aggregate.CountStar; +import org.apache.spark.sql.connector.expressions.aggregate.Max; +import org.apache.spark.sql.connector.expressions.aggregate.Min; +import org.junit.jupiter.api.Test; + +public class TestSparkAggregates { + + @Test + public void testAggregates() { + Map attrMap = Maps.newHashMap(); + attrMap.put("id", "id"); + attrMap.put("`i.d`", "i.d"); + attrMap.put("`i``d`", "i`d"); + attrMap.put("`d`.b.`dd```", "d.b.dd`"); + attrMap.put("a.`aa```.c", "a.aa`.c"); + + attrMap.forEach( + (quoted, unquoted) -> { + NamedReference namedReference = FieldReference.apply(quoted); + + Max max = new Max(namedReference); + Expression expectedMax = Expressions.max(unquoted); + Expression actualMax = SparkAggregates.convert(max); + assertThat(actualMax).asString().isEqualTo(expectedMax.toString()); + + Min min = new Min(namedReference); + Expression expectedMin = Expressions.min(unquoted); + Expression actualMin = SparkAggregates.convert(min); + assertThat(actualMin).asString().isEqualTo(expectedMin.toString()); + + Count count = new Count(namedReference, false); + Expression expectedCount = Expressions.count(unquoted); + Expression actualCount = SparkAggregates.convert(count); + assertThat(actualCount).asString().isEqualTo(expectedCount.toString()); + + Count countDistinct = new Count(namedReference, true); + Expression convertedCountDistinct = SparkAggregates.convert(countDistinct); + assertThat(convertedCountDistinct).as("Count Distinct is converted to null").isNull(); + + CountStar countStar = new CountStar(); + Expression expectedCountStar = Expressions.countStar(); + Expression actualCountStar = SparkAggregates.convert(countStar); + assertThat(actualCountStar).asString().isEqualTo(expectedCountStar.toString()); + }); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java new file mode 100644 index 000000000000..1f266380cdc1 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java @@ -0,0 +1,68 @@ +/* + * 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.Map; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.connector.catalog.FunctionCatalog; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.SupportsNamespaces; +import org.apache.spark.sql.connector.catalog.Table; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.apache.spark.sql.connector.catalog.ViewCatalog; + +public class TestSparkCatalog< + T extends TableCatalog & FunctionCatalog & SupportsNamespaces & ViewCatalog> + extends SparkSessionCatalog { + + private static final Map TABLE_MAP = Maps.newHashMap(); + + public static void setTable(Identifier ident, Table table) { + Preconditions.checkArgument( + !TABLE_MAP.containsKey(ident), "Cannot set " + ident + ". It is already set"); + TABLE_MAP.put(ident, table); + } + + @Override + public Table loadTable(Identifier ident) throws NoSuchTableException { + if (TABLE_MAP.containsKey(ident)) { + return TABLE_MAP.get(ident); + } + + TableIdentifier tableIdentifier = Spark3Util.identifierToTableIdentifier(ident); + Namespace namespace = tableIdentifier.namespace(); + + TestTables.TestTable table = TestTables.load(tableIdentifier.toString()); + if (table == null && namespace.equals(Namespace.of("default"))) { + table = TestTables.load(tableIdentifier.name()); + } + + return new SparkTable(table, false); + } + + public static void clearTables() { + TABLE_MAP.clear(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java new file mode 100644 index 000000000000..f16db1972e7b --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java @@ -0,0 +1,155 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.apache.iceberg.CachingCatalog; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkCatalogCacheExpiration extends TestBaseWithCatalog { + + private static final Map SESSION_CATALOG_CONFIG = + ImmutableMap.of( + "type", + "hadoop", + "default-namespace", + "default", + CatalogProperties.CACHE_ENABLED, + "true", + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS, + "3000"); + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + {"spark_catalog", SparkSessionCatalog.class.getName(), SESSION_CATALOG_CONFIG}, + }; + } + + private static String asSqlConfCatalogKeyFor(String catalog, String configKey) { + // configKey is empty when the catalog's class is being defined + if (configKey.isEmpty()) { + return String.format("spark.sql.catalog.%s", catalog); + } else { + return String.format("spark.sql.catalog.%s.%s", catalog, configKey); + } + } + + // Add more catalogs to the spark session, so we only need to start spark one time for multiple + // different catalog configuration tests. + @BeforeAll + public static void beforeClass() { + // Catalog - expiration_disabled: Catalog with caching on and expiration disabled. + ImmutableMap.of( + "", + "org.apache.iceberg.spark.SparkCatalog", + "type", + "hive", + CatalogProperties.CACHE_ENABLED, + "true", + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS, + "-1") + .forEach((k, v) -> spark.conf().set(asSqlConfCatalogKeyFor("expiration_disabled", k), v)); + + // Catalog - cache_disabled_implicitly: Catalog that does not cache, as the cache expiration + // interval is 0. + ImmutableMap.of( + "", + "org.apache.iceberg.spark.SparkCatalog", + "type", + "hive", + CatalogProperties.CACHE_ENABLED, + "true", + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS, + "0") + .forEach( + (k, v) -> spark.conf().set(asSqlConfCatalogKeyFor("cache_disabled_implicitly", k), v)); + } + + @TestTemplate + public void testSparkSessionCatalogWithExpirationEnabled() { + SparkSessionCatalog sparkCatalog = sparkSessionCatalog(); + assertThat(sparkCatalog) + .extracting("icebergCatalog") + .extracting("cacheEnabled") + .isEqualTo(true); + + assertThat(sparkCatalog) + .extracting("icebergCatalog") + .extracting("icebergCatalog") + .isInstanceOfSatisfying( + Catalog.class, + icebergCatalog -> { + assertThat(icebergCatalog) + .isExactlyInstanceOf(CachingCatalog.class) + .extracting("expirationIntervalMillis") + .isEqualTo(3000L); + }); + } + + @TestTemplate + public void testCacheEnabledAndExpirationDisabled() { + SparkCatalog sparkCatalog = getSparkCatalog("expiration_disabled"); + assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(true); + + assertThat(sparkCatalog) + .extracting("icebergCatalog") + .isInstanceOfSatisfying( + CachingCatalog.class, + icebergCatalog -> { + assertThat(icebergCatalog).extracting("expirationIntervalMillis").isEqualTo(-1L); + }); + } + + @TestTemplate + public void testCacheDisabledImplicitly() { + SparkCatalog sparkCatalog = getSparkCatalog("cache_disabled_implicitly"); + assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(false); + + assertThat(sparkCatalog) + .extracting("icebergCatalog") + .isInstanceOfSatisfying( + Catalog.class, + icebergCatalog -> assertThat(icebergCatalog).isNotInstanceOf(CachingCatalog.class)); + } + + private SparkSessionCatalog sparkSessionCatalog() { + TableCatalog catalog = + (TableCatalog) spark.sessionState().catalogManager().catalog("spark_catalog"); + return (SparkSessionCatalog) catalog; + } + + private SparkCatalog getSparkCatalog(String catalog) { + return (SparkCatalog) spark.sessionState().catalogManager().catalog(catalog); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java new file mode 100644 index 000000000000..fd155a6bcaf3 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java @@ -0,0 +1,143 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.KryoHelpers; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkCatalogHadoopOverrides extends CatalogTestBase { + + private static final String CONFIG_TO_OVERRIDE = "fs.s3a.buffer.dir"; + // prepend "hadoop." so that the test base formats SQLConf correctly + // as `spark.sql.catalogs..hadoop. + private static final String HADOOP_PREFIXED_CONFIG_TO_OVERRIDE = "hadoop." + CONFIG_TO_OVERRIDE; + private static final String CONFIG_OVERRIDE_VALUE = "/tmp-overridden"; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + "testhive", + SparkCatalog.class.getName(), + ImmutableMap.of( + "type", + "hive", + "default-namespace", + "default", + HADOOP_PREFIXED_CONFIG_TO_OVERRIDE, + CONFIG_OVERRIDE_VALUE) + }, + { + "testhadoop", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "hadoop", HADOOP_PREFIXED_CONFIG_TO_OVERRIDE, CONFIG_OVERRIDE_VALUE) + }, + { + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", + "hive", + "default-namespace", + "default", + HADOOP_PREFIXED_CONFIG_TO_OVERRIDE, + CONFIG_OVERRIDE_VALUE) + } + }; + } + + @BeforeEach + public void createTable() { + sql("CREATE TABLE IF NOT EXISTS %s (id bigint) USING iceberg", tableName(tableIdent.name())); + } + + @AfterEach + public void dropTable() { + sql("DROP TABLE IF EXISTS %s", tableName(tableIdent.name())); + } + + @TestTemplate + public void testTableFromCatalogHasOverrides() throws Exception { + Table table = getIcebergTableFromSparkCatalog(); + Configuration conf = ((Configurable) table.io()).getConf(); + String actualCatalogOverride = conf.get(CONFIG_TO_OVERRIDE, "/whammies"); + assertThat(actualCatalogOverride) + .as( + "Iceberg tables from spark should have the overridden hadoop configurations from the spark config") + .isEqualTo(CONFIG_OVERRIDE_VALUE); + } + + @TestTemplate + public void ensureRoundTripSerializedTableRetainsHadoopConfig() throws Exception { + Table table = getIcebergTableFromSparkCatalog(); + Configuration originalConf = ((Configurable) table.io()).getConf(); + String actualCatalogOverride = originalConf.get(CONFIG_TO_OVERRIDE, "/whammies"); + assertThat(actualCatalogOverride) + .as( + "Iceberg tables from spark should have the overridden hadoop configurations from the spark config") + .isEqualTo(CONFIG_OVERRIDE_VALUE); + + // Now convert to SerializableTable and ensure overridden property is still present. + Table serializableTable = SerializableTableWithSize.copyOf(table); + Table kryoSerializedTable = + KryoHelpers.roundTripSerialize(SerializableTableWithSize.copyOf(table)); + Configuration configFromKryoSerde = ((Configurable) kryoSerializedTable.io()).getConf(); + String kryoSerializedCatalogOverride = configFromKryoSerde.get(CONFIG_TO_OVERRIDE, "/whammies"); + assertThat(kryoSerializedCatalogOverride) + .as( + "Tables serialized with Kryo serialization should retain overridden hadoop configuration properties") + .isEqualTo(CONFIG_OVERRIDE_VALUE); + + // Do the same for Java based serde + Table javaSerializedTable = TestHelpers.roundTripSerialize(serializableTable); + Configuration configFromJavaSerde = ((Configurable) javaSerializedTable.io()).getConf(); + String javaSerializedCatalogOverride = configFromJavaSerde.get(CONFIG_TO_OVERRIDE, "/whammies"); + assertThat(javaSerializedCatalogOverride) + .as( + "Tables serialized with Java serialization should retain overridden hadoop configuration properties") + .isEqualTo(CONFIG_OVERRIDE_VALUE); + } + + @SuppressWarnings("ThrowSpecificity") + private Table getIcebergTableFromSparkCatalog() throws Exception { + Identifier identifier = Identifier.of(tableIdent.namespace().levels(), tableIdent.name()); + TableCatalog catalog = + (TableCatalog) spark.sessionState().catalogManager().catalog(catalogName); + SparkTable sparkTable = (SparkTable) catalog.loadTable(identifier); + return sparkTable.table(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDVWriters.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDVWriters.java new file mode 100644 index 000000000000..dfc693d3094d --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDVWriters.java @@ -0,0 +1,69 @@ +/* + * 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 org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestDVWriters; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +public class TestSparkDVWriters extends TestDVWriters { + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return SparkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(dataFormat()) + .deleteFileFormat(dataFormat()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected InternalRow toRow(Integer id, String data) { + InternalRow row = new GenericInternalRow(2); + row.update(0, id); + row.update(1, UTF8String.fromString(data)); + return row; + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + StructType sparkType = SparkSchemaUtil.convert(table.schema()); + for (InternalRow row : rows) { + InternalRowWrapper wrapper = new InternalRowWrapper(sparkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java new file mode 100644 index 000000000000..35a837b6b88b --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java @@ -0,0 +1,350 @@ +/* + * 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.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestReader; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.UpdatePartitionSpec; +import org.apache.iceberg.hadoop.HadoopTables; +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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.spark.SparkDataFile; +import org.apache.iceberg.spark.SparkDeleteFile; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.ColumnName; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestSparkDataFile { + + private static final HadoopTables TABLES = new HadoopTables(new Configuration()); + private static final Schema SCHEMA = + new Schema( + required(100, "id", Types.LongType.get()), + optional(101, "data", Types.StringType.get()), + required(102, "b", Types.BooleanType.get()), + optional(103, "i", Types.IntegerType.get()), + required(104, "l", Types.LongType.get()), + optional(105, "f", Types.FloatType.get()), + required(106, "d", Types.DoubleType.get()), + optional(107, "date", Types.DateType.get()), + required(108, "ts", Types.TimestampType.withZone()), + required(109, "tsntz", Types.TimestampType.withoutZone()), + required(110, "s", Types.StringType.get()), + optional(113, "bytes", Types.BinaryType.get()), + required(114, "dec_9_0", Types.DecimalType.of(9, 0)), + required(115, "dec_11_2", Types.DecimalType.of(11, 2)), + required(116, "dec_38_10", Types.DecimalType.of(38, 10)) // maximum precision + ); + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA) + .identity("b") + .bucket("i", 2) + .identity("l") + .identity("f") + .identity("d") + .identity("date") + .hour("ts") + .identity("ts") + .identity("tsntz") + .truncate("s", 2) + .identity("bytes") + .bucket("dec_9_0", 2) + .bucket("dec_11_2", 2) + .bucket("dec_38_10", 2) + .build(); + + private static SparkSession spark; + private static JavaSparkContext sparkContext = null; + + @BeforeAll + public static void startSpark() { + TestSparkDataFile.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .getOrCreate(); + TestSparkDataFile.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestSparkDataFile.spark; + TestSparkDataFile.spark = null; + TestSparkDataFile.sparkContext = null; + currentSpark.stop(); + } + + @TempDir private File tableDir; + private String tableLocation = null; + + @BeforeEach + public void setupTableLocation() { + this.tableLocation = tableDir.toURI().toString(); + } + + @Test + public void testValueConversion() throws IOException { + Table table = + TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap(), tableLocation); + checkSparkContentFiles(table); + } + + @Test + public void testValueConversionPartitionedTable() throws IOException { + Table table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); + checkSparkContentFiles(table); + } + + @Test + public void testValueConversionWithEmptyStats() throws IOException { + Map props = Maps.newHashMap(); + props.put(TableProperties.DEFAULT_WRITE_METRICS_MODE, "none"); + Table table = TABLES.create(SCHEMA, SPEC, props, tableLocation); + checkSparkContentFiles(table); + } + + private void checkSparkContentFiles(Table table) throws IOException { + Iterable rows = RandomData.generateSpark(table.schema(), 200, 0); + JavaRDD rdd = sparkContext.parallelize(Lists.newArrayList(rows)); + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + + Dataset df = + ((org.apache.spark.sql.classic.SparkSession) spark) + .internalCreateDataFrame( + JavaRDD.toRDD(rdd), SparkSchemaUtil.convert(table.schema()), false); + + df.write().format("iceberg").mode("append").save(tableLocation); + + table.refresh(); + + PartitionSpec dataFilesSpec = table.spec(); + + List manifests = table.currentSnapshot().allManifests(table.io()); + assertThat(manifests).hasSize(1); + + List dataFiles = Lists.newArrayList(); + try (ManifestReader reader = ManifestFiles.read(manifests.get(0), table.io())) { + for (DataFile dataFile : reader) { + checkDataFile(dataFile.copy(), DataFiles.builder(dataFilesSpec).copy(dataFile).build()); + dataFiles.add(dataFile.copy()); + } + } + + UpdatePartitionSpec updateSpec = table.updateSpec(); + for (PartitionField field : dataFilesSpec.fields()) { + updateSpec.removeField(field.name()); + } + updateSpec.commit(); + + List positionDeleteFiles = Lists.newArrayList(); + List equalityDeleteFiles = Lists.newArrayList(); + + RowDelta rowDelta = table.newRowDelta(); + + for (DataFile dataFile : dataFiles) { + DeleteFile positionDeleteFile = createPositionDeleteFile(table, dataFile); + positionDeleteFiles.add(positionDeleteFile); + rowDelta.addDeletes(positionDeleteFile); + } + + DeleteFile equalityDeleteFile1 = createEqualityDeleteFile(table); + equalityDeleteFiles.add(equalityDeleteFile1); + rowDelta.addDeletes(equalityDeleteFile1); + + DeleteFile equalityDeleteFile2 = createEqualityDeleteFile(table); + equalityDeleteFiles.add(equalityDeleteFile2); + rowDelta.addDeletes(equalityDeleteFile2); + + rowDelta.commit(); + + Dataset dataFileDF = spark.read().format("iceberg").load(tableLocation + "#data_files"); + List sparkDataFiles = shuffleColumns(dataFileDF).collectAsList(); + assertThat(sparkDataFiles).hasSameSizeAs(dataFiles); + + Types.StructType dataFileType = DataFile.getType(dataFilesSpec.partitionType()); + StructType sparkDataFileType = sparkDataFiles.get(0).schema(); + SparkDataFile dataFileWrapper = new SparkDataFile(dataFileType, sparkDataFileType); + + for (int i = 0; i < dataFiles.size(); i++) { + checkDataFile(dataFiles.get(i), dataFileWrapper.wrap(sparkDataFiles.get(i))); + } + + Dataset positionDeleteFileDF = + spark.read().format("iceberg").load(tableLocation + "#delete_files").where("content = 1"); + List sparkPositionDeleteFiles = shuffleColumns(positionDeleteFileDF).collectAsList(); + assertThat(sparkPositionDeleteFiles).hasSameSizeAs(positionDeleteFiles); + + Types.StructType positionDeleteFileType = DataFile.getType(dataFilesSpec.partitionType()); + StructType sparkPositionDeleteFileType = sparkPositionDeleteFiles.get(0).schema(); + SparkDeleteFile positionDeleteFileWrapper = + new SparkDeleteFile(positionDeleteFileType, sparkPositionDeleteFileType); + + for (int i = 0; i < positionDeleteFiles.size(); i++) { + checkDeleteFile( + positionDeleteFiles.get(i), + positionDeleteFileWrapper.wrap(sparkPositionDeleteFiles.get(i))); + } + + Dataset equalityDeleteFileDF = + spark.read().format("iceberg").load(tableLocation + "#delete_files").where("content = 2"); + List sparkEqualityDeleteFiles = shuffleColumns(equalityDeleteFileDF).collectAsList(); + assertThat(sparkEqualityDeleteFiles).hasSameSizeAs(equalityDeleteFiles); + + Types.StructType equalityDeleteFileType = DataFile.getType(table.spec().partitionType()); + StructType sparkEqualityDeleteFileType = sparkEqualityDeleteFiles.get(0).schema(); + SparkDeleteFile equalityDeleteFileWrapper = + new SparkDeleteFile(equalityDeleteFileType, sparkEqualityDeleteFileType); + + for (int i = 0; i < equalityDeleteFiles.size(); i++) { + checkDeleteFile( + equalityDeleteFiles.get(i), + equalityDeleteFileWrapper.wrap(sparkEqualityDeleteFiles.get(i))); + } + } + + private Dataset shuffleColumns(Dataset df) { + List columns = + Arrays.stream(df.columns()).map(ColumnName::new).collect(Collectors.toList()); + Collections.shuffle(columns); + return df.select(columns.toArray(new Column[0])); + } + + private void checkDataFile(DataFile expected, DataFile actual) { + assertThat(expected.equalityFieldIds()).isNull(); + assertThat(actual.equalityFieldIds()).isNull(); + checkContentFile(expected, actual); + checkStructLike(expected.partition(), actual.partition()); + } + + private void checkDeleteFile(DeleteFile expected, DeleteFile actual) { + assertThat(expected.equalityFieldIds()).isEqualTo(actual.equalityFieldIds()); + checkContentFile(expected, actual); + checkStructLike(expected.partition(), actual.partition()); + } + + private void checkContentFile(ContentFile expected, ContentFile actual) { + assertThat(actual.content()).isEqualTo(expected.content()); + assertThat(actual.location()).isEqualTo(expected.location()); + assertThat(actual.format()).isEqualTo(expected.format()); + assertThat(actual.recordCount()).isEqualTo(expected.recordCount()); + assertThat(actual.fileSizeInBytes()).isEqualTo(expected.fileSizeInBytes()); + assertThat(actual.valueCounts()).isEqualTo(expected.valueCounts()); + assertThat(actual.nullValueCounts()).isEqualTo(expected.nullValueCounts()); + assertThat(actual.nanValueCounts()).isEqualTo(expected.nanValueCounts()); + assertThat(actual.lowerBounds()).isEqualTo(expected.lowerBounds()); + assertThat(actual.upperBounds()).isEqualTo(expected.upperBounds()); + assertThat(actual.keyMetadata()).isEqualTo(expected.keyMetadata()); + assertThat(actual.splitOffsets()).isEqualTo(expected.splitOffsets()); + assertThat(actual.sortOrderId()).isEqualTo(expected.sortOrderId()); + } + + private void checkStructLike(StructLike expected, StructLike actual) { + assertThat(actual.size()).isEqualTo(expected.size()); + for (int i = 0; i < expected.size(); i++) { + assertThat(actual.get(i, Object.class)).isEqualTo(expected.get(i, Object.class)); + } + } + + private DeleteFile createPositionDeleteFile(Table table, DataFile dataFile) { + PartitionSpec spec = table.specs().get(dataFile.specId()); + return FileMetadata.deleteFileBuilder(spec) + .ofPositionDeletes() + .withPath("/path/to/pos-deletes-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(dataFile.fileSizeInBytes() / 4) + .withPartition(dataFile.partition()) + .withRecordCount(2) + .withMetrics( + new Metrics( + 2L, + null, // no column sizes + null, // no value counts + null, // no null counts + null, // no NaN counts + ImmutableMap.of( + MetadataColumns.DELETE_FILE_PATH.fieldId(), + Conversions.toByteBuffer(Types.StringType.get(), dataFile.location())), + ImmutableMap.of( + MetadataColumns.DELETE_FILE_PATH.fieldId(), + Conversions.toByteBuffer(Types.StringType.get(), dataFile.location())))) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) + .build(); + } + + private DeleteFile createEqualityDeleteFile(Table table) { + return FileMetadata.deleteFileBuilder(table.spec()) + .ofEqualityDeletes(3, 4) + .withPath("/path/to/eq-deletes-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(250) + .withRecordCount(1) + .withSortOrder(SortOrder.unsorted()) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) + .build(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java new file mode 100644 index 000000000000..4ccbf86f1257 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -0,0 +1,739 @@ +/* + * 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.SPARK_WRITE_PARTITIONED_FANOUT_ENABLED; +import static org.apache.iceberg.types.Types.NestedField.optional; +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 static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.io.File; +import java.net.InetAddress; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.hadoop.HadoopTables; +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.Lists; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkDataWrite { + private static final Configuration CONF = new Configuration(); + + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private String branch; + + private static SparkSession spark = null; + private static final Schema SCHEMA = + new Schema( + optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); + + @TempDir private Path temp; + + @Parameters(name = "format = {0}, branch = {1}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {FileFormat.PARQUET, null}, + new Object[] {FileFormat.PARQUET, "main"}, + new Object[] {FileFormat.PARQUET, "testBranch"}, + new Object[] {FileFormat.AVRO, null}, + new Object[] {FileFormat.ORC, "testBranch"} + }; + } + + @BeforeAll + public static void startSpark() { + TestSparkDataWrite.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .getOrCreate(); + } + + @AfterEach + public void clearSourceCache() { + ManualSource.clearTables(); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestSparkDataWrite.spark; + TestSparkDataWrite.spark = null; + currentSpark.stop(); + } + + @TestTemplate + public void testBasicWrite() { + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + List expected = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + // TODO: incoming columns must be ordered according to the table's schema + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + createBranch(table); + table.refresh(); + + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + for (ManifestFile manifest : + SnapshotUtil.latestSnapshot(table, branch).allManifests(table.io())) { + for (DataFile file : ManifestFiles.read(manifest, table.io())) { + // TODO: avro not support split + if (!format.equals(FileFormat.AVRO)) { + assertThat(file.splitOffsets()).as("Split offsets not present").isNotNull(); + } + assertThat(file.recordCount()).as("Should have reported record count as 1").isEqualTo(1); + // TODO: append more metric info + if (format.equals(FileFormat.PARQUET)) { + assertThat(file.columnSizes()).as("Column sizes metric not present").isNotNull(); + assertThat(file.valueCounts()).as("Counts metric not present").isNotNull(); + assertThat(file.nullValueCounts()).as("Null value counts metric not present").isNotNull(); + assertThat(file.lowerBounds()).as("Lower bounds metric not present").isNotNull(); + assertThat(file.upperBounds()).as("Upper bounds metric not present").isNotNull(); + } + } + } + } + + @TestTemplate + public void testAppend() { + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + List expected = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "a"), + new SimpleRecord(5, "b"), + new SimpleRecord(6, "c")); + + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + createBranch(table); + + df.withColumn("id", df.col("id").plus(3)) + .select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(targetLocation); + + table.refresh(); + + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + + @TestTemplate + public void testEmptyOverwrite() { + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("id").build(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + List expected = records; + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + createBranch(table); + + Dataset empty = spark.createDataFrame(ImmutableList.of(), SimpleRecord.class); + empty + .select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Overwrite) + .option("overwrite-mode", "dynamic") + .save(targetLocation); + + table.refresh(); + + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + + @TestTemplate + public void testOverwrite() { + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("id").build(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + List expected = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "a"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "b"), + new SimpleRecord(6, "c")); + + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + createBranch(table); + + // overwrite with 2*id to replace record 2, append 4 and 6 + df.withColumn("id", df.col("id").multiply(2)) + .select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Overwrite) + .option("overwrite-mode", "dynamic") + .save(targetLocation); + + table.refresh(); + + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + + @TestTemplate + public void testUnpartitionedOverwrite() { + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + List expected = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + createBranch(table); + + // overwrite with the same data; should not produce two copies + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Overwrite) + .save(targetLocation); + + table.refresh(); + + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + + @TestTemplate + public void testUnpartitionedCreateWithTargetFileSizeViaTableProperties() { + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + table + .updateProperties() + .set(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, "4") // ~4 bytes; low enough to trigger + .commit(); + + List expected = Lists.newArrayListWithCapacity(4000); + for (int i = 0; i < 4000; i++) { + expected.add(new SimpleRecord(i, "a")); + } + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + createBranch(table); + table.refresh(); + + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + + List files = Lists.newArrayList(); + for (ManifestFile manifest : + SnapshotUtil.latestSnapshot(table, branch).allManifests(table.io())) { + for (DataFile file : ManifestFiles.read(manifest, table.io())) { + files.add(file); + } + } + + assertThat(files) + .hasSize(4) + .allSatisfy( + dataFile -> + assertThat(dataFile.recordCount()) + .as("All DataFiles contain 1000 rows") + .isEqualTo(1000)); + } + + @TestTemplate + public void testPartitionedCreateWithTargetFileSizeViaOption() { + partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType.NONE); + } + + @TestTemplate + public void testPartitionedFanoutCreateWithTargetFileSizeViaOption() { + partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType.TABLE); + } + + @TestTemplate + public void testPartitionedFanoutCreateWithTargetFileSizeViaOption2() { + partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType.JOB); + } + + @TestTemplate + public void testWriteProjection() { + assumeThat(spark.version()) + .as("Not supported in Spark 3; analysis requires all columns are present") + .startsWith("2"); + + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + List expected = + Lists.newArrayList( + new SimpleRecord(1, null), new SimpleRecord(2, null), new SimpleRecord(3, null)); + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + df.select("id") + .write() // select only id column + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + createBranch(table); + table.refresh(); + + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + + @TestTemplate + public void testWriteProjectionWithMiddle() { + assumeThat(spark.version()) + .as("Not supported in Spark 3; analysis requires all columns are present") + .startsWith("2"); + + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Schema schema = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + Table table = tables.create(schema, spec, location.toString()); + + List expected = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "hello"), + new ThreeColumnRecord(2, null, "world"), + new ThreeColumnRecord(3, null, null)); + + Dataset df = spark.createDataFrame(expected, ThreeColumnRecord.class); + + df.select("c1", "c3") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + createBranch(table); + table.refresh(); + + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = + result.orderBy("c1").as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + } + + @TestTemplate + public void testViewsReturnRecentResults() { + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + tables.create(SCHEMA, spec, location.toString()); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + Table table = tables.load(location.toString()); + createBranch(table); + + Dataset query = spark.read().format("iceberg").load(targetLocation).where("id = 1"); + query.createOrReplaceTempView("tmp"); + + List actual1 = + spark.table("tmp").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + List expected1 = Lists.newArrayList(new SimpleRecord(1, "a")); + assertThat(actual1).hasSameSizeAs(expected1).isEqualTo(expected1); + + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(targetLocation); + + List actual2 = + spark.table("tmp").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + List expected2 = + Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(1, "a")); + assertThat(actual2).hasSameSizeAs(expected2).isEqualTo(expected2); + } + + public void partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType option) { + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Map properties = + ImmutableMap.of( + TableProperties.WRITE_DISTRIBUTION_MODE, TableProperties.WRITE_DISTRIBUTION_MODE_NONE); + Table table = tables.create(SCHEMA, spec, properties, location.toString()); + + List expected = Lists.newArrayListWithCapacity(8000); + for (int i = 0; i < 2000; i++) { + expected.add(new SimpleRecord(i, "a")); + expected.add(new SimpleRecord(i, "b")); + expected.add(new SimpleRecord(i, "c")); + expected.add(new SimpleRecord(i, "d")); + } + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + switch (option) { + case NONE: + df.select("id", "data") + .sort("data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, 4) // ~4 bytes; low enough to trigger + .save(location.toString()); + break; + case TABLE: + table.updateProperties().set(SPARK_WRITE_PARTITIONED_FANOUT_ENABLED, "true").commit(); + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, 4) // ~4 bytes; low enough to trigger + .save(location.toString()); + break; + case JOB: + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, 4) // ~4 bytes; low enough to trigger + .option(SparkWriteOptions.FANOUT_ENABLED, true) + .save(location.toString()); + break; + default: + break; + } + + createBranch(table); + table.refresh(); + + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = + result.orderBy("id", "data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + + List files = Lists.newArrayList(); + for (ManifestFile manifest : + SnapshotUtil.latestSnapshot(table, branch).allManifests(table.io())) { + for (DataFile file : ManifestFiles.read(manifest, table.io())) { + files.add(file); + } + } + assertThat(files) + .hasSize(8) + .allSatisfy( + dataFile -> + assertThat(dataFile.recordCount()) + .as("All DataFiles contain 1000 rows") + .isEqualTo(1000)); + } + + @TestTemplate + public void testCommitUnknownException() { + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "commitunknown"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + createBranch(table); + table.refresh(); + + List records2 = + Lists.newArrayList( + new SimpleRecord(4, "d"), new SimpleRecord(5, "e"), new SimpleRecord(6, "f")); + + Dataset df2 = spark.createDataFrame(records2, SimpleRecord.class); + + AppendFiles append = table.newFastAppend(); + if (branch != null) { + append.toBranch(branch); + } + + AppendFiles spyAppend = spy(append); + doAnswer( + invocation -> { + append.commit(); + throw new CommitStateUnknownException(new RuntimeException("Datacenter on Fire")); + }) + .when(spyAppend) + .commit(); + + Table spyTable = spy(table); + when(spyTable.newAppend()).thenReturn(spyAppend); + SparkTable sparkTable = new SparkTable(spyTable, false); + + String manualTableName = "unknown_exception"; + ManualSource.setTable(manualTableName, sparkTable); + + // Although an exception is thrown here, write and commit have succeeded + assertThatThrownBy( + () -> + df2.select("id", "data") + .sort("data") + .write() + .format("org.apache.iceberg.spark.source.ManualSource") + .option(ManualSource.TABLE_NAME, manualTableName) + .mode(SaveMode.Append) + .save(targetLocation)) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("Datacenter on Fire"); + + // Since write and commit succeeded, the rows should be readable + Dataset result = spark.read().format("iceberg").load(targetLocation); + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual) + .hasSize(records.size() + records2.size()) + .containsExactlyInAnyOrder( + ImmutableList.builder() + .addAll(records) + .addAll(records2) + .build() + .toArray(new SimpleRecord[0])); + } + + public enum IcebergOptionsType { + NONE, + TABLE, + JOB + } + + private String locationWithBranch(File location) { + if (branch == null) { + return location.toString(); + } + + return location + "#branch_" + branch; + } + + private void createBranch(Table table) { + if (branch != null && !branch.equals(SnapshotRef.MAIN_BRANCH)) { + table.manageSnapshots().createBranch(branch, table.currentSnapshot().snapshotId()).commit(); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java new file mode 100644 index 000000000000..575e6658db22 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java @@ -0,0 +1,69 @@ +/* + * 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 org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestFileWriterFactory; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +public class TestSparkFileWriterFactory extends TestFileWriterFactory { + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return SparkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected InternalRow toRow(Integer id, String data) { + InternalRow row = new GenericInternalRow(2); + row.update(0, id); + row.update(1, UTF8String.fromString(data)); + return row; + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + StructType sparkType = SparkSchemaUtil.convert(table.schema()); + for (InternalRow row : rows) { + InternalRowWrapper wrapper = new InternalRowWrapper(sparkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java new file mode 100644 index 000000000000..68f6f17aaed5 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -0,0 +1,371 @@ +/* + * 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.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.FORMAT_VERSION; +import static org.apache.iceberg.TableProperties.ORC_VECTORIZATION_ENABLED; +import static org.apache.iceberg.TableProperties.PARQUET_BATCH_SIZE; +import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED; +import static org.apache.spark.sql.functions.expr; +import static org.apache.spark.sql.functions.lit; +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.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +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.TestBase; +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; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.types.StructType; +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; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkMetadataColumns extends TestBase { + + private static final String TABLE_NAME = "test_table"; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "category", Types.StringType.get()), + Types.NestedField.optional(3, "data", Types.StringType.get())); + private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); + private static final PartitionSpec UNKNOWN_SPEC = + TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(1) + .addField("zero", 1, "id_zero") + .build(); + + @Parameters(name = "fileFormat = {0}, vectorized = {1}, formatVersion = {2}") + public static Object[][] parameters() { + List parameters = Lists.newArrayList(); + for (int version : TestHelpers.ALL_VERSIONS) { + parameters.add(new Object[] {FileFormat.PARQUET, false, version}); + parameters.add(new Object[] {FileFormat.PARQUET, true, version}); + parameters.add(new Object[] {FileFormat.AVRO, false, version}); + parameters.add(new Object[] {FileFormat.ORC, false, version}); + parameters.add(new Object[] {FileFormat.ORC, true, version}); + } + return parameters.toArray(new Object[0][]); + } + + @TempDir private Path temp; + + @Parameter(index = 0) + private FileFormat fileFormat; + + @Parameter(index = 1) + private boolean vectorized; + + @Parameter(index = 2) + private int formatVersion; + + private Table table = null; + + @BeforeAll + public static void setupSpark() { + ImmutableMap config = + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "cache-enabled", "true"); + spark + .conf() + .set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.source.TestSparkCatalog"); + config.forEach( + (key, value) -> spark.conf().set("spark.sql.catalog.spark_catalog." + key, value)); + } + + @BeforeEach + public void setupTable() throws IOException { + createAndInitTable(); + } + + @AfterEach + public void dropTable() { + TestTables.clearTables(); + } + + @TestTemplate + public void testSpecAndPartitionMetadataColumns() { + // TODO: support metadata structs in vectorized ORC reads + assumeThat(fileFormat).isNotEqualTo(FileFormat.ORC); + assumeThat(vectorized).isFalse(); + + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", TABLE_NAME); + + table.refresh(); + table.updateSpec().addField("data").commit(); + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", TABLE_NAME); + + table.refresh(); + table.updateSpec().addField(Expressions.bucket("category", 8)).commit(); + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", TABLE_NAME); + + table.refresh(); + table.updateSpec().removeField("data").commit(); + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", TABLE_NAME); + + table.refresh(); + table.updateSpec().renameField("category_bucket_8", "category_bucket_8_another_name").commit(); + + List expected = + ImmutableList.of( + row(0, row(null, null)), + row(1, row("b1", null)), + row(2, row("b1", 2)), + row(3, row(null, 2))); + assertEquals( + "Rows must match", + expected, + sql("SELECT _spec_id, _partition FROM %s ORDER BY _spec_id", TABLE_NAME)); + } + + @TestTemplate + public void testPartitionMetadataColumnWithManyColumns() { + List fields = + Lists.newArrayList(Types.NestedField.required(0, "id", Types.LongType.get())); + List additionalCols = + IntStream.range(1, 1010) + .mapToObj(i -> Types.NestedField.optional(i, "c" + i, Types.StringType.get())) + .collect(Collectors.toList()); + fields.addAll(additionalCols); + Schema manyColumnsSchema = new Schema(fields); + PartitionSpec spec = PartitionSpec.builderFor(manyColumnsSchema).identity("id").build(); + + TableOperations ops = ((HasTableOperations) table).operations(); + TableMetadata base = ops.current(); + ops.commit(base, base.updateSchema(manyColumnsSchema).updatePartitionSpec(spec)); + + Dataset df = + spark + .range(2) + .withColumns( + IntStream.range(1, 1010) + .boxed() + .collect(Collectors.toMap(i -> "c" + i, i -> expr("CAST(id as STRING)")))); + StructType sparkSchema = spark.table(TABLE_NAME).schema(); + spark + .createDataFrame(df.rdd(), sparkSchema) + .coalesce(1) + .write() + .format("iceberg") + .mode("append") + .save(TABLE_NAME); + + assertThat(spark.table(TABLE_NAME).select("*", "_partition").count()).isEqualTo(2); + List expected = + ImmutableList.of(row(row(0L), 0L, "0", "0", "0"), row(row(1L), 1L, "1", "1", "1")); + assertEquals( + "Rows must match", + expected, + sql("SELECT _partition, id, c999, c1000, c1001 FROM %s ORDER BY id", TABLE_NAME)); + } + + @TestTemplate + public void testPositionMetadataColumnWithMultipleRowGroups() throws NoSuchTableException { + assumeThat(fileFormat).isEqualTo(FileFormat.PARQUET); + + table.updateProperties().set(PARQUET_ROW_GROUP_SIZE_BYTES, "100").commit(); + + List ids = Lists.newArrayList(); + for (long id = 0L; id < 200L; id++) { + ids.add(id); + } + Dataset df = + spark + .createDataset(ids, Encoders.LONG()) + .withColumnRenamed("value", "id") + .withColumn("category", lit("hr")) + .withColumn("data", lit("ABCDEF")); + df.coalesce(1).writeTo(TABLE_NAME).append(); + + assertThat(spark.table(TABLE_NAME).count()).isEqualTo(200); + + List expectedRows = ids.stream().map(this::row).collect(Collectors.toList()); + assertEquals("Rows must match", expectedRows, sql("SELECT _pos FROM %s", TABLE_NAME)); + } + + @TestTemplate + public void testPositionMetadataColumnWithMultipleBatches() throws NoSuchTableException { + assumeThat(fileFormat).isEqualTo(FileFormat.PARQUET); + + table.updateProperties().set(PARQUET_BATCH_SIZE, "1000").commit(); + + List ids = Lists.newArrayList(); + for (long id = 0L; id < 7500L; id++) { + ids.add(id); + } + Dataset df = + spark + .createDataset(ids, Encoders.LONG()) + .withColumnRenamed("value", "id") + .withColumn("category", lit("hr")) + .withColumn("data", lit("ABCDEF")); + df.coalesce(1).writeTo(TABLE_NAME).append(); + + assertThat(spark.table(TABLE_NAME).count()).isEqualTo(7500); + + List expectedRows = ids.stream().map(this::row).collect(Collectors.toList()); + assertEquals("Rows must match", expectedRows, sql("SELECT _pos FROM %s", TABLE_NAME)); + } + + @TestTemplate + public void testPartitionMetadataColumnWithUnknownTransforms() { + // replace the table spec to include an unknown transform + TableOperations ops = ((HasTableOperations) table).operations(); + TableMetadata base = ops.current(); + ops.commit(base, base.updatePartitionSpec(UNKNOWN_SPEC)); + + assertThatThrownBy(() -> sql("SELECT _partition FROM %s", TABLE_NAME)) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot build table partition type, unknown transforms: [zero]"); + } + + @TestTemplate + public void testConflictingColumns() { + table + .updateSchema() + .addColumn(MetadataColumns.SPEC_ID.name(), Types.IntegerType.get()) + .addColumn(MetadataColumns.FILE_PATH.name(), Types.StringType.get()) + .commit(); + + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1', -1, 'path/to/file')", TABLE_NAME); + + assertEquals( + "Rows must match", + ImmutableList.of(row(1L, "a1")), + sql("SELECT id, category FROM %s", TABLE_NAME)); + + assertThatThrownBy(() -> sql("SELECT * FROM %s", TABLE_NAME)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Table column names conflict with names reserved for Iceberg metadata columns: [_spec_id, _file]."); + + table.refresh(); + + table + .updateSchema() + .renameColumn(MetadataColumns.SPEC_ID.name(), "_renamed" + MetadataColumns.SPEC_ID.name()) + .renameColumn( + MetadataColumns.FILE_PATH.name(), "_renamed" + MetadataColumns.FILE_PATH.name()) + .commit(); + + assertEquals( + "Rows must match", + ImmutableList.of(row(0, null, -1)), + sql("SELECT _spec_id, _partition, _renamed_spec_id FROM %s", TABLE_NAME)); + } + + @TestTemplate + public void testIdentifierFields() { + table.updateSchema().setIdentifierFields("id").commit(); + + sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", TABLE_NAME); + + assertEquals( + "Rows must match", + ImmutableList.of(row(1L, 0, null)), + sql("SELECT id, _spec_id, _partition FROM %s", TABLE_NAME)); + } + + @TestTemplate + public void testRowLineageColumnsResolvedInV3OrHigher() { + if (formatVersion >= 3) { + // Test against an empty table to ensure column resolution in formats supporting row lineage + // and so that the test doesn't have to change with inheritance + assertEquals( + "Rows must match", + ImmutableList.of(), + sql("SELECT _row_id, _last_updated_sequence_number, id FROM %s", TABLE_NAME)); + } else { + // Should fail to resolve row lineage metadata columns in V1/V2 tables + assertThatThrownBy(() -> sql("SELECT _row_id FROM %s", TABLE_NAME)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "A column, variable, or function parameter with name `_row_id` cannot be resolved"); + assertThatThrownBy(() -> sql("SELECT _last_updated_sequence_number FROM %s", TABLE_NAME)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "A column, variable, or function parameter with name `_last_updated_sequence_number` cannot be resolved"); + } + } + + private void createAndInitTable() throws IOException { + Map properties = Maps.newHashMap(); + properties.put(FORMAT_VERSION, String.valueOf(formatVersion)); + properties.put(DEFAULT_FILE_FORMAT, fileFormat.name()); + + switch (fileFormat) { + case PARQUET: + properties.put(PARQUET_VECTORIZATION_ENABLED, String.valueOf(vectorized)); + break; + case ORC: + properties.put(ORC_VECTORIZATION_ENABLED, String.valueOf(vectorized)); + break; + default: + Preconditions.checkState( + !vectorized, "File format %s does not support vectorized reads", fileFormat); + } + + this.table = + TestTables.create( + Files.createTempDirectory(temp, "junit").toFile(), + TABLE_NAME, + SCHEMA, + SPEC, + properties); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java new file mode 100644 index 000000000000..979abd21e7f7 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java @@ -0,0 +1,69 @@ +/* + * 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 org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestPartitioningWriters; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +public class TestSparkPartitioningWriters extends TestPartitioningWriters { + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return SparkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected InternalRow toRow(Integer id, String data) { + InternalRow row = new GenericInternalRow(2); + row.update(0, id); + row.update(1, UTF8String.fromString(data)); + return row; + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + StructType sparkType = SparkSchemaUtil.convert(table.schema()); + for (InternalRow row : rows) { + InternalRowWrapper wrapper = new InternalRowWrapper(sparkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPlanningUtil.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPlanningUtil.java new file mode 100644 index 000000000000..e4602532f029 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPlanningUtil.java @@ -0,0 +1,211 @@ +/* + * 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.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.when; + +import java.util.List; +import org.apache.iceberg.BaseScanTaskGroup; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataTask; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.MockFileScanTask; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.TestHelpers.Row; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mockito; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkPlanningUtil extends TestBaseWithCatalog { + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "category", Types.StringType.get())); + private static final PartitionSpec SPEC_1 = + PartitionSpec.builderFor(SCHEMA).withSpecId(1).bucket("id", 16).identity("data").build(); + private static final PartitionSpec SPEC_2 = + PartitionSpec.builderFor(SCHEMA).withSpecId(2).identity("data").build(); + private static final List EXECUTOR_LOCATIONS = + ImmutableList.of("host1_exec1", "host1_exec2", "host1_exec3", "host2_exec1", "host2_exec2"); + + @TestTemplate + public void testFileScanTaskWithoutDeletes() { + List tasks = + ImmutableList.of( + new MockFileScanTask(mockDataFile(Row.of(1, "a")), SCHEMA, SPEC_1), + new MockFileScanTask(mockDataFile(Row.of(2, "b")), SCHEMA, SPEC_1), + new MockFileScanTask(mockDataFile(Row.of(3, "c")), SCHEMA, SPEC_1)); + ScanTaskGroup taskGroup = new BaseScanTaskGroup<>(tasks); + List> taskGroups = ImmutableList.of(taskGroup); + + String[][] locations = SparkPlanningUtil.assignExecutors(taskGroups, EXECUTOR_LOCATIONS); + + // should not assign executors if there are no deletes + assertThat(locations).hasDimensions(1, 0); + } + + @TestTemplate + public void testFileScanTaskWithDeletes() { + StructLike partition1 = Row.of("k2", null); + StructLike partition2 = Row.of("k1"); + List tasks = + ImmutableList.of( + new MockFileScanTask( + mockDataFile(partition1), mockDeleteFiles(1, partition1), SCHEMA, SPEC_1), + new MockFileScanTask( + mockDataFile(partition2), mockDeleteFiles(3, partition2), SCHEMA, SPEC_2), + new MockFileScanTask( + mockDataFile(partition1), mockDeleteFiles(2, partition1), SCHEMA, SPEC_1)); + ScanTaskGroup taskGroup = new BaseScanTaskGroup<>(tasks); + List> taskGroups = ImmutableList.of(taskGroup); + + String[][] locations = SparkPlanningUtil.assignExecutors(taskGroups, EXECUTOR_LOCATIONS); + + // should assign executors and handle different size of partitions + assertThat(locations.length).isEqualTo(1); + assertThat(locations[0].length).isGreaterThanOrEqualTo(1); + } + + @TestTemplate + public void testFileScanTaskWithUnpartitionedDeletes() { + List tasks1 = + ImmutableList.of( + new MockFileScanTask( + mockDataFile(Row.of()), + mockDeleteFiles(2, Row.of()), + SCHEMA, + PartitionSpec.unpartitioned()), + new MockFileScanTask( + mockDataFile(Row.of()), + mockDeleteFiles(2, Row.of()), + SCHEMA, + PartitionSpec.unpartitioned()), + new MockFileScanTask( + mockDataFile(Row.of()), + mockDeleteFiles(2, Row.of()), + SCHEMA, + PartitionSpec.unpartitioned())); + ScanTaskGroup taskGroup1 = new BaseScanTaskGroup<>(tasks1); + List tasks2 = + ImmutableList.of( + new MockFileScanTask( + mockDataFile(null), + mockDeleteFiles(2, null), + SCHEMA, + PartitionSpec.unpartitioned()), + new MockFileScanTask( + mockDataFile(null), + mockDeleteFiles(2, null), + SCHEMA, + PartitionSpec.unpartitioned()), + new MockFileScanTask( + mockDataFile(null), + mockDeleteFiles(2, null), + SCHEMA, + PartitionSpec.unpartitioned())); + ScanTaskGroup taskGroup2 = new BaseScanTaskGroup<>(tasks2); + List> taskGroups = ImmutableList.of(taskGroup1, taskGroup2); + + String[][] locations = SparkPlanningUtil.assignExecutors(taskGroups, EXECUTOR_LOCATIONS); + + // should not assign executors if the table is unpartitioned + assertThat(locations).hasDimensions(2, 0); + } + + @TestTemplate + public void testDataTasks() { + List tasks = + ImmutableList.of( + new MockDataTask(mockDataFile(Row.of(1, "a"))), + new MockDataTask(mockDataFile(Row.of(2, "b"))), + new MockDataTask(mockDataFile(Row.of(3, "c")))); + ScanTaskGroup taskGroup = new BaseScanTaskGroup<>(tasks); + List> taskGroups = ImmutableList.of(taskGroup); + + String[][] locations = SparkPlanningUtil.assignExecutors(taskGroups, EXECUTOR_LOCATIONS); + + // should not assign executors for data tasks + assertThat(locations).hasDimensions(1, 0); + } + + @TestTemplate + public void testUnknownTasks() { + List tasks = ImmutableList.of(new UnknownScanTask(), new UnknownScanTask()); + ScanTaskGroup taskGroup = new BaseScanTaskGroup<>(tasks); + List> taskGroups = ImmutableList.of(taskGroup); + + String[][] locations = SparkPlanningUtil.assignExecutors(taskGroups, EXECUTOR_LOCATIONS); + + // should not assign executors for unknown tasks + assertThat(locations).hasDimensions(1, 0); + } + + private static DataFile mockDataFile(StructLike partition) { + DataFile file = Mockito.mock(DataFile.class); + when(file.partition()).thenReturn(partition); + return file; + } + + private static DeleteFile[] mockDeleteFiles(int count, StructLike partition) { + DeleteFile[] files = new DeleteFile[count]; + for (int index = 0; index < count; index++) { + files[index] = mockDeleteFile(partition); + } + return files; + } + + private static DeleteFile mockDeleteFile(StructLike partition) { + DeleteFile file = Mockito.mock(DeleteFile.class); + when(file.partition()).thenReturn(partition); + return file; + } + + private static class MockDataTask extends MockFileScanTask implements DataTask { + + MockDataTask(DataFile file) { + super(file); + } + + @Override + public PartitionSpec spec() { + return PartitionSpec.unpartitioned(); + } + + @Override + public CloseableIterable rows() { + throw new UnsupportedOperationException(); + } + } + + private static class UnknownScanTask implements ScanTask {} +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java new file mode 100644 index 000000000000..9dc56abf9fb6 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java @@ -0,0 +1,69 @@ +/* + * 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 org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestPositionDeltaWriters; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.unsafe.types.UTF8String; + +public class TestSparkPositionDeltaWriters extends TestPositionDeltaWriters { + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return SparkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected InternalRow toRow(Integer id, String data) { + InternalRow row = new GenericInternalRow(2); + row.update(0, id); + row.update(1, UTF8String.fromString(data)); + return row; + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + StructType sparkType = SparkSchemaUtil.convert(table.schema()); + for (InternalRow row : rows) { + InternalRowWrapper wrapper = new InternalRowWrapper(sparkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java new file mode 100644 index 000000000000..57b2c7146b79 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java @@ -0,0 +1,279 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; +import static scala.collection.JavaConverters.seqAsJavaListConverter; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.execution.SparkPlan; +import org.apache.spark.sql.execution.metric.SQLMetric; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import scala.collection.JavaConverters; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkReadMetrics extends TestBaseWithCatalog { + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testReadMetricsForV1Table() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT) USING iceberg TBLPROPERTIES ('format-version'='1')", + tableName); + + spark.range(10000).coalesce(1).writeTo(tableName).append(); + spark.range(10001, 20000).coalesce(1).writeTo(tableName).append(); + + Dataset df = spark.sql(String.format("select * from %s where id < 10000", tableName)); + df.collect(); + + List sparkPlans = + seqAsJavaListConverter(df.queryExecution().executedPlan().collectLeaves()).asJava(); + Map metricsMap = + JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); + // Common + assertThat(metricsMap) + .hasEntrySatisfying( + "totalPlanningDuration", sqlMetric -> assertThat(sqlMetric.value()).isNotEqualTo(0)); + + // data manifests + assertThat(metricsMap) + .hasEntrySatisfying( + "totalDataManifest", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(2)); + assertThat(metricsMap) + .hasEntrySatisfying( + "scannedDataManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(2)); + assertThat(metricsMap) + .hasEntrySatisfying( + "skippedDataManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + + // data files + assertThat(metricsMap) + .hasEntrySatisfying( + "resultDataFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(1)); + assertThat(metricsMap) + .hasEntrySatisfying( + "skippedDataFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(1)); + assertThat(metricsMap) + .hasEntrySatisfying( + "totalDataFileSize", sqlMetric -> assertThat(sqlMetric.value()).isNotEqualTo(0)); + + // delete manifests + assertThat(metricsMap) + .hasEntrySatisfying( + "totalDeleteManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "scannedDeleteManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "skippedDeleteManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + + // delete files + assertThat(metricsMap) + .hasEntrySatisfying( + "totalDeleteFileSize", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "resultDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "equalityDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "indexedDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "positionalDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "skippedDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + } + + @TestTemplate + public void testReadMetricsForV2Table() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT) USING iceberg TBLPROPERTIES ('format-version'='2')", + tableName); + + spark.range(10000).coalesce(1).writeTo(tableName).append(); + spark.range(10001, 20000).coalesce(1).writeTo(tableName).append(); + + Dataset df = spark.sql(String.format("select * from %s where id < 10000", tableName)); + df.collect(); + + List sparkPlans = + seqAsJavaListConverter(df.queryExecution().executedPlan().collectLeaves()).asJava(); + Map metricsMap = + JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); + + // Common + assertThat(metricsMap) + .hasEntrySatisfying( + "totalPlanningDuration", sqlMetric -> assertThat(sqlMetric.value()).isNotEqualTo(0)); + + // data manifests + assertThat(metricsMap) + .hasEntrySatisfying( + "totalDataManifest", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(2)); + assertThat(metricsMap) + .hasEntrySatisfying( + "scannedDataManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(2)); + assertThat(metricsMap) + .hasEntrySatisfying( + "skippedDataManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + + // data files + assertThat(metricsMap) + .hasEntrySatisfying( + "resultDataFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(1)); + assertThat(metricsMap) + .hasEntrySatisfying( + "skippedDataFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(1)); + assertThat(metricsMap) + .hasEntrySatisfying( + "totalDataFileSize", sqlMetric -> assertThat(sqlMetric.value()).isNotEqualTo(0)); + + // delete manifests + assertThat(metricsMap) + .hasEntrySatisfying( + "totalDeleteManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "scannedDeleteManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "skippedDeleteManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + + // delete files + assertThat(metricsMap) + .hasEntrySatisfying( + "totalDeleteFileSize", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "resultDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "equalityDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "indexedDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "positionalDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "skippedDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + } + + @TestTemplate + public void testDeleteMetrics() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT)" + + " USING iceberg" + + " TBLPROPERTIES (\n" + + " 'write.delete.mode'='merge-on-read',\n" + + " 'write.update.mode'='merge-on-read',\n" + + " 'write.merge.mode'='merge-on-read',\n" + + " 'format-version'='2'\n" + + " )", + tableName); + + spark.range(10000).coalesce(1).writeTo(tableName).append(); + + spark.sql(String.format("DELETE FROM %s WHERE id = 1", tableName)).collect(); + Dataset df = spark.sql(String.format("SELECT * FROM %s", tableName)); + df.collect(); + + List sparkPlans = + seqAsJavaListConverter(df.queryExecution().executedPlan().collectLeaves()).asJava(); + Map metricsMap = + JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); + + // Common + assertThat(metricsMap) + .hasEntrySatisfying( + "totalPlanningDuration", sqlMetric -> assertThat(sqlMetric.value()).isNotEqualTo(0)); + + // data manifests + assertThat(metricsMap) + .hasEntrySatisfying( + "totalDataManifest", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(1)); + assertThat(metricsMap) + .hasEntrySatisfying( + "scannedDataManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(1)); + assertThat(metricsMap) + .hasEntrySatisfying( + "skippedDataManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + + // data files + assertThat(metricsMap) + .hasEntrySatisfying( + "resultDataFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(1)); + assertThat(metricsMap) + .hasEntrySatisfying( + "skippedDataFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "totalDataFileSize", sqlMetric -> assertThat(sqlMetric.value()).isNotEqualTo(0)); + + // delete manifests + assertThat(metricsMap) + .hasEntrySatisfying( + "totalDeleteManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(1)); + assertThat(metricsMap) + .hasEntrySatisfying( + "scannedDeleteManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(1)); + assertThat(metricsMap) + .hasEntrySatisfying( + "skippedDeleteManifests", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + + // delete files + assertThat(metricsMap) + .hasEntrySatisfying( + "totalDeleteFileSize", sqlMetric -> assertThat(sqlMetric.value()).isNotEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "resultDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(1)); + assertThat(metricsMap) + .hasEntrySatisfying( + "equalityDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + assertThat(metricsMap) + .hasEntrySatisfying( + "indexedDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(1)); + assertThat(metricsMap) + .hasEntrySatisfying( + "positionalDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(1)); + assertThat(metricsMap) + .hasEntrySatisfying( + "skippedDeleteFiles", sqlMetric -> assertThat(sqlMetric.value()).isEqualTo(0)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java new file mode 100644 index 000000000000..8ccea303d0c1 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadProjection.java @@ -0,0 +1,273 @@ +/* + * 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.Files.localOutput; +import static org.apache.iceberg.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.apache.iceberg.data.FileHelpers.encrypt; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.net.InetAddress; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericFileWriterFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.DataWriter; +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.SparkValueConverter; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkReadProjection extends TestReadProjection { + + private static SparkSession spark = null; + + @Parameters(name = "format = {0}, vectorized = {1}, planningMode = {2}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.PARQUET, false, LOCAL}, + {FileFormat.PARQUET, true, DISTRIBUTED}, + {FileFormat.AVRO, false, LOCAL}, + {FileFormat.ORC, false, DISTRIBUTED}, + {FileFormat.ORC, true, LOCAL} + }; + } + + @Parameter(index = 1) + private boolean vectorized; + + @Parameter(index = 2) + private PlanningMode planningMode; + + @BeforeAll + public static void startSpark() { + TestSparkReadProjection.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .getOrCreate(); + ImmutableMap config = + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "parquet-enabled", "true", + "cache-enabled", "false"); + spark + .conf() + .set("spark.sql.catalog.spark_catalog", "org.apache.iceberg.spark.source.TestSparkCatalog"); + config.forEach( + (key, value) -> spark.conf().set("spark.sql.catalog.spark_catalog." + key, value)); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestSparkReadProjection.spark; + TestSparkReadProjection.spark = null; + currentSpark.stop(); + } + + @Override + protected Record writeAndRead(String desc, Schema writeSchema, Schema readSchema, Record record) + throws IOException { + File parent = new File(temp.toFile(), desc); + File location = new File(parent, "test"); + File dataFolder = new File(location, "data"); + assertThat(dataFolder.mkdirs()).as("mkdirs should succeed").isTrue(); + + File testFile = new File(dataFolder, format.addExtension(UUID.randomUUID().toString())); + + Table table = + TestTables.create( + location, + desc, + writeSchema, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DATA_PLANNING_MODE, planningMode.modeName(), + TableProperties.DELETE_PLANNING_MODE, planningMode.modeName())); + try { + // Important: use the table's schema for the rest of the test + // When tables are created, the column ids are reassigned. + Schema tableSchema = table.schema(); + + DataWriter writer = + new GenericFileWriterFactory.Builder() + .dataFileFormat(format) + .dataSchema(tableSchema) + .build() + .newDataWriter(encrypt(localOutput(testFile)), PartitionSpec.unpartitioned(), null); + try (writer) { + writer.write(record); + } + + DataFile file = writer.toDataFile(); + + table.newAppend().appendFile(file).commit(); + + // rewrite the read schema for the table's reassigned ids + Map idMapping = Maps.newHashMap(); + for (int id : allIds(writeSchema)) { + // translate each id to the original schema's column name, then to the new schema's id + String originalName = writeSchema.findColumnName(id); + idMapping.put(id, tableSchema.findField(originalName).fieldId()); + } + Schema expectedSchema = reassignIds(readSchema, idMapping); + + // Set the schema to the expected schema directly to simulate the table schema evolving + TestTables.replaceMetadata(desc, TestTables.readMetadata(desc).updateSchema(expectedSchema)); + + Dataset df = + spark + .read() + .format("org.apache.iceberg.spark.source.TestIcebergSource") + .option("iceberg.table.name", desc) + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(); + + return SparkValueConverter.convert(readSchema, df.collectAsList().get(0)); + + } finally { + TestTables.clearTables(); + } + } + + private List allIds(Schema schema) { + List ids = Lists.newArrayList(); + TypeUtil.visit( + schema, + new TypeUtil.SchemaVisitor() { + @Override + public Void field(Types.NestedField field, Void fieldResult) { + ids.add(field.fieldId()); + return null; + } + + @Override + public Void list(Types.ListType list, Void elementResult) { + ids.add(list.elementId()); + return null; + } + + @Override + public Void map(Types.MapType map, Void keyResult, Void valueResult) { + ids.add(map.keyId()); + ids.add(map.valueId()); + return null; + } + }); + return ids; + } + + private Schema reassignIds(Schema schema, Map idMapping) { + return new Schema( + TypeUtil.visit( + schema, + new TypeUtil.SchemaVisitor() { + private int mapId(int id) { + if (idMapping.containsKey(id)) { + return idMapping.get(id); + } + return 1000 + id; // make sure the new IDs don't conflict with reassignment + } + + @Override + public Type schema(Schema schema, Type structResult) { + return structResult; + } + + @Override + public Type struct(Types.StructType struct, List fieldResults) { + List newFields = + Lists.newArrayListWithExpectedSize(fieldResults.size()); + List fields = struct.fields(); + for (int i = 0; i < fields.size(); i += 1) { + Types.NestedField field = fields.get(i); + if (field.isOptional()) { + newFields.add( + optional(mapId(field.fieldId()), field.name(), fieldResults.get(i))); + } else { + newFields.add( + required(mapId(field.fieldId()), field.name(), fieldResults.get(i))); + } + } + return Types.StructType.of(newFields); + } + + @Override + public Type field(Types.NestedField field, Type fieldResult) { + return fieldResult; + } + + @Override + public Type list(Types.ListType list, Type elementResult) { + if (list.isElementOptional()) { + return Types.ListType.ofOptional(mapId(list.elementId()), elementResult); + } else { + return Types.ListType.ofRequired(mapId(list.elementId()), elementResult); + } + } + + @Override + public Type map(Types.MapType map, Type keyResult, Type valueResult) { + if (map.isValueOptional()) { + return Types.MapType.ofOptional( + mapId(map.keyId()), mapId(map.valueId()), keyResult, valueResult); + } else { + return Types.MapType.ofRequired( + mapId(map.keyId()), mapId(map.valueId()), keyResult, valueResult); + } + } + + @Override + public Type primitive(Type.PrimitiveType primitive) { + return primitive; + } + }) + .asNestedType() + .asStructType() + .fields()); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java new file mode 100644 index 000000000000..c7716d343024 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -0,0 +1,755 @@ +/* + * 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.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; +import static org.apache.iceberg.spark.source.SparkSQLExecutionHelper.lastExecutedMetricValue; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.apache.spark.sql.types.DataTypes.IntegerType; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.File; +import java.io.IOException; +import java.time.LocalDate; +import java.util.List; +import java.util.Set; +import javax.annotation.Nonnull; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.DeleteReadTests; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetSchemaUtil; +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.Sets; +import org.apache.iceberg.spark.ImmutableParquetBatchReadConf; +import org.apache.iceberg.spark.ParquetBatchReadConf; +import org.apache.iceberg.spark.ParquetReaderType; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkStructLike; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.data.SparkParquetWriters; +import org.apache.iceberg.spark.source.metrics.NumDeletes; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.iceberg.util.TableScanUtil; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkReaderDeletes extends DeleteReadTests { + private static TestHiveMetastore metastore = null; + protected static SparkSession spark = null; + protected static HiveCatalog catalog = null; + + @Parameter(index = 2) + private boolean vectorized; + + @Parameter(index = 3) + private PlanningMode planningMode; + + @Parameters(name = "fileFormat = {0}, formatVersion = {1}, vectorized = {2}, planningMode = {3}") + public static Object[][] parameters() { + List parameters = Lists.newArrayList(); + for (int version : TestHelpers.V2_AND_ABOVE) { + parameters.add(new Object[] {FileFormat.PARQUET, version, false, PlanningMode.DISTRIBUTED}); + parameters.add(new Object[] {FileFormat.PARQUET, version, true, PlanningMode.LOCAL}); + if (version == 2) { + parameters.add(new Object[] {FileFormat.ORC, version, false, PlanningMode.DISTRIBUTED}); + parameters.add(new Object[] {FileFormat.AVRO, version, false, PlanningMode.LOCAL}); + } + } + return parameters.toArray(new Object[0][]); + } + + @BeforeAll + public static void startMetastoreAndSpark() { + metastore = new TestHiveMetastore(); + metastore.start(); + HiveConf hiveConf = metastore.hiveConf(); + + spark = + SparkSession.builder() + .master("local[2]") + .config("spark.appStateStore.asyncTracking.enable", false) + .config("spark.ui.liveUpdate.period", 0) + .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) + .enableHiveSupport() + .getOrCreate(); + + catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + + try { + catalog.createNamespace(Namespace.of("default")); + } catch (AlreadyExistsException ignored) { + // the default namespace already exists. ignore the create error + } + } + + @AfterAll + public static void stopMetastoreAndSpark() throws Exception { + catalog = null; + metastore.stop(); + metastore = null; + spark.stop(); + spark = null; + } + + @AfterEach + @Override + public void cleanup() throws IOException { + super.cleanup(); + dropTable("test3"); + } + + @Override + protected Table createTable(String name, Schema schema, PartitionSpec spec) { + Table table = catalog.createTable(TableIdentifier.of("default", name), schema); + TableOperations ops = ((BaseTable) table).operations(); + TableMetadata meta = ops.current(); + ops.commit(meta, meta.upgradeToFormatVersion(formatVersion)); + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) + .set(TableProperties.DATA_PLANNING_MODE, planningMode.modeName()) + .set(TableProperties.DELETE_PLANNING_MODE, planningMode.modeName()) + .set(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)) + .commit(); + if (format.equals(FileFormat.PARQUET) || format.equals(FileFormat.ORC)) { + String vectorizationEnabled = + format.equals(FileFormat.PARQUET) + ? TableProperties.PARQUET_VECTORIZATION_ENABLED + : TableProperties.ORC_VECTORIZATION_ENABLED; + String batchSize = + format.equals(FileFormat.PARQUET) + ? TableProperties.PARQUET_BATCH_SIZE + : TableProperties.ORC_BATCH_SIZE; + table.updateProperties().set(vectorizationEnabled, String.valueOf(vectorized)).commit(); + if (vectorized) { + // split 7 records to two batches to cover more code paths + table.updateProperties().set(batchSize, "4").commit(); + } + } + return table; + } + + @Override + protected void dropTable(String name) { + catalog.dropTable(TableIdentifier.of("default", name)); + } + + protected boolean countDeletes() { + return true; + } + + @Override + protected long deleteCount() { + return Long.parseLong(lastExecutedMetricValue(spark, NumDeletes.DISPLAY_STRING)); + } + + @Override + public StructLikeSet rowSet(String name, Table table, String... columns) { + return rowSet(name, table.schema().select(columns).asStruct(), columns); + } + + public StructLikeSet rowSet(String name, Types.StructType projection, String... columns) { + Dataset df = + spark + .read() + .format("iceberg") + .load(TableIdentifier.of("default", name).toString()) + .selectExpr(columns); + + StructLikeSet set = StructLikeSet.create(projection); + df.collectAsList() + .forEach( + row -> { + SparkStructLike rowWrapper = new SparkStructLike(projection); + set.add(rowWrapper.wrap(row)); + }); + + return set; + } + + @TestTemplate + public void testEqualityDeleteWithFilter() throws IOException { + String tableName = table.name().substring(table.name().lastIndexOf(".") + 1); + Schema deleteRowSchema = table.schema().select("data"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + List dataDeletes = + Lists.newArrayList( + dataDelete.copy("data", "a"), // id = 29 + dataDelete.copy("data", "d"), // id = 89 + dataDelete.copy("data", "g") // id = 122 + ); + + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + dataDeletes, + deleteRowSchema); + + table.newRowDelta().addDeletes(eqDeletes).commit(); + + Types.StructType projection = table.schema().select("*").asStruct(); + Dataset df = + spark + .read() + .format("iceberg") + .load(TableIdentifier.of("default", tableName).toString()) + .filter("data = 'a'") // select a deleted row + .selectExpr("*"); + + StructLikeSet actual = StructLikeSet.create(projection); + df.collectAsList() + .forEach( + row -> { + SparkStructLike rowWrapper = new SparkStructLike(projection); + actual.add(rowWrapper.wrap(row)); + }); + + assertThat(actual).as("Table should contain no rows").hasSize(0); + } + + @TestTemplate + public void testReadEqualityDeleteRows() throws IOException { + Schema deleteSchema1 = table.schema().select("data"); + Record dataDelete = GenericRecord.create(deleteSchema1); + List dataDeletes = + Lists.newArrayList( + dataDelete.copy("data", "a"), // id = 29 + dataDelete.copy("data", "d") // id = 89 + ); + + Schema deleteSchema2 = table.schema().select("id"); + Record idDelete = GenericRecord.create(deleteSchema2); + List idDeletes = + Lists.newArrayList( + idDelete.copy("id", 121), // id = 121 + idDelete.copy("id", 122) // id = 122 + ); + + DeleteFile eqDelete1 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + dataDeletes, + deleteSchema1); + + DeleteFile eqDelete2 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + idDeletes, + deleteSchema2); + + table.newRowDelta().addDeletes(eqDelete1).addDeletes(eqDelete2).commit(); + + StructLikeSet expectedRowSet = rowSetWithIds(29, 89, 121, 122); + + Types.StructType type = table.schema().asStruct(); + StructLikeSet actualRowSet = StructLikeSet.create(type); + + CloseableIterable tasks = + TableScanUtil.planTasks( + table.newScan().planFiles(), + TableProperties.METADATA_SPLIT_SIZE_DEFAULT, + TableProperties.SPLIT_LOOKBACK_DEFAULT, + TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT); + + for (CombinedScanTask task : tasks) { + try (EqualityDeleteRowReader reader = + new EqualityDeleteRowReader(task, table, null, table.schema(), false, true)) { + while (reader.next()) { + actualRowSet.add( + new InternalRowWrapper( + SparkSchemaUtil.convert(table.schema()), table.schema().asStruct()) + .wrap(reader.get().copy())); + } + } + } + + assertThat(actualRowSet).as("should include 4 deleted row").hasSize(4); + assertThat(actualRowSet).as("deleted row should be matched").isEqualTo(expectedRowSet); + } + + @TestTemplate + public void testPosDeletesAllRowsInBatch() throws IOException { + // read.parquet.vectorization.batch-size is set to 4, so the 4 rows in the first batch are all + // deleted. + List> deletes = + Lists.newArrayList( + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 + ); + + Pair posDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + deletes, + formatVersion); + + table + .newRowDelta() + .addDeletes(posDeletes.first()) + .validateDataFilesExist(posDeletes.second()) + .commit(); + + StructLikeSet expected = rowSetWithoutIds(table, records, 29, 43, 61, 89); + StructLikeSet actual = rowSet(tableName, table, "*"); + + assertThat(actual).as("Table should contain expected rows").isEqualTo(expected); + checkDeleteCount(4L); + } + + @TestTemplate + public void testPosDeletesWithDeletedColumn() throws IOException { + // read.parquet.vectorization.batch-size is set to 4, so the 4 rows in the first batch are all + // deleted. + List> deletes = + Lists.newArrayList( + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 + ); + + Pair posDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + deletes, + formatVersion); + + table + .newRowDelta() + .addDeletes(posDeletes.first()) + .validateDataFilesExist(posDeletes.second()) + .commit(); + + StructLikeSet expected = expectedRowSet(29, 43, 61, 89); + StructLikeSet actual = + rowSet(tableName, PROJECTION_SCHEMA.asStruct(), "id", "data", "_deleted"); + + assertThat(actual).as("Table should contain expected row").isEqualTo(expected); + checkDeleteCount(4L); + } + + @TestTemplate + public void testEqualityDeleteWithDeletedColumn() throws IOException { + String tableName = table.name().substring(table.name().lastIndexOf(".") + 1); + Schema deleteRowSchema = table.schema().select("data"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + List dataDeletes = + Lists.newArrayList( + dataDelete.copy("data", "a"), // id = 29 + dataDelete.copy("data", "d"), // id = 89 + dataDelete.copy("data", "g") // id = 122 + ); + + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + dataDeletes, + deleteRowSchema); + + table.newRowDelta().addDeletes(eqDeletes).commit(); + + StructLikeSet expected = expectedRowSet(29, 89, 122); + StructLikeSet actual = + rowSet(tableName, PROJECTION_SCHEMA.asStruct(), "id", "data", "_deleted"); + + assertThat(actual).as("Table should contain expected row").isEqualTo(expected); + checkDeleteCount(3L); + } + + @TestTemplate + public void testMixedPosAndEqDeletesWithDeletedColumn() throws IOException { + Schema dataSchema = table.schema().select("data"); + Record dataDelete = GenericRecord.create(dataSchema); + List dataDeletes = + Lists.newArrayList( + dataDelete.copy("data", "a"), // id = 29 + dataDelete.copy("data", "d"), // id = 89 + dataDelete.copy("data", "g") // id = 122 + ); + + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + dataDeletes, + dataSchema); + + List> deletes = + Lists.newArrayList( + Pair.of(dataFile.location(), 3L), // id = 89 + Pair.of(dataFile.location(), 5L) // id = 121 + ); + + Pair posDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + deletes, + formatVersion); + + table + .newRowDelta() + .addDeletes(eqDeletes) + .addDeletes(posDeletes.first()) + .validateDataFilesExist(posDeletes.second()) + .commit(); + + StructLikeSet expected = expectedRowSet(29, 89, 121, 122); + StructLikeSet actual = + rowSet(tableName, PROJECTION_SCHEMA.asStruct(), "id", "data", "_deleted"); + + assertThat(actual).as("Table should contain expected row").isEqualTo(expected); + checkDeleteCount(4L); + } + + @TestTemplate + public void testFilterOnDeletedMetadataColumn() throws IOException { + List> deletes = + Lists.newArrayList( + Pair.of(dataFile.location(), 0L), // id = 29 + Pair.of(dataFile.location(), 1L), // id = 43 + Pair.of(dataFile.location(), 2L), // id = 61 + Pair.of(dataFile.location(), 3L) // id = 89 + ); + + Pair posDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + deletes, + formatVersion); + + table + .newRowDelta() + .addDeletes(posDeletes.first()) + .validateDataFilesExist(posDeletes.second()) + .commit(); + + StructLikeSet expected = expectedRowSetWithNonDeletesOnly(29, 43, 61, 89); + + // get non-deleted rows + Dataset df = + spark + .read() + .format("iceberg") + .load(TableIdentifier.of("default", tableName).toString()) + .select("id", "data", "_deleted") + .filter("_deleted = false"); + + Types.StructType projection = PROJECTION_SCHEMA.asStruct(); + StructLikeSet actual = StructLikeSet.create(projection); + df.collectAsList() + .forEach( + row -> { + SparkStructLike rowWrapper = new SparkStructLike(projection); + actual.add(rowWrapper.wrap(row)); + }); + + assertThat(actual).as("Table should contain expected row").isEqualTo(expected); + + StructLikeSet expectedDeleted = expectedRowSetWithDeletesOnly(29, 43, 61, 89); + + // get deleted rows + df = + spark + .read() + .format("iceberg") + .load(TableIdentifier.of("default", tableName).toString()) + .select("id", "data", "_deleted") + .filter("_deleted = true"); + + StructLikeSet actualDeleted = StructLikeSet.create(projection); + df.collectAsList() + .forEach( + row -> { + SparkStructLike rowWrapper = new SparkStructLike(projection); + actualDeleted.add(rowWrapper.wrap(row)); + }); + + assertThat(actualDeleted).as("Table should contain expected row").isEqualTo(expectedDeleted); + } + + @TestTemplate + public void testIsDeletedColumnWithoutDeleteFile() { + StructLikeSet expected = expectedRowSet(); + StructLikeSet actual = + rowSet(tableName, PROJECTION_SCHEMA.asStruct(), "id", "data", "_deleted"); + assertThat(actual).as("Table should contain expected row").isEqualTo(expected); + checkDeleteCount(0L); + } + + @TestTemplate + public void testPosDeletesOnParquetFileWithMultipleRowGroups() throws IOException { + assumeThat(format).isEqualTo(FileFormat.PARQUET); + + String tblName = "test3"; + Table tbl = createTable(tblName, SCHEMA, PartitionSpec.unpartitioned()); + + List fileSplits = Lists.newArrayList(); + StructType sparkSchema = SparkSchemaUtil.convert(SCHEMA); + Configuration conf = new Configuration(); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + Path testFilePath = new Path(testFile.getAbsolutePath()); + + // Write a Parquet file with more than one row group + ParquetFileWriter parquetFileWriter = + new ParquetFileWriter(conf, ParquetSchemaUtil.convert(SCHEMA, "test3Schema"), testFilePath); + parquetFileWriter.start(); + for (int i = 0; i < 2; i += 1) { + File split = File.createTempFile("junit", null, temp.toFile()); + assertThat(split.delete()).as("Delete should succeed").isTrue(); + Path splitPath = new Path(split.getAbsolutePath()); + fileSplits.add(splitPath); + try (FileAppender writer = + Parquet.write(Files.localOutput(split)) + .createWriterFunc(msgType -> SparkParquetWriters.buildWriter(sparkSchema, msgType)) + .schema(SCHEMA) + .overwrite() + .build()) { + Iterable records = RandomData.generateSpark(SCHEMA, 100, 34 * i + 37); + writer.addAll(records); + } + parquetFileWriter.appendFile( + org.apache.parquet.hadoop.util.HadoopInputFile.fromPath(splitPath, conf)); + } + parquetFileWriter.end( + ParquetFileWriter.mergeMetadataFiles(fileSplits, conf) + .getFileMetaData() + .getKeyValueMetaData()); + + // Add the file to the table + DataFile dataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withInputFile(org.apache.iceberg.hadoop.HadoopInputFile.fromPath(testFilePath, conf)) + .withFormat("parquet") + .withRecordCount(200) + .build(); + tbl.newAppend().appendFile(dataFile).commit(); + + // Add positional deletes to the table + List> deletes = + Lists.newArrayList( + Pair.of(dataFile.location(), 97L), + Pair.of(dataFile.location(), 98L), + Pair.of(dataFile.location(), 99L), + Pair.of(dataFile.location(), 101L), + Pair.of(dataFile.location(), 103L), + Pair.of(dataFile.location(), 107L), + Pair.of(dataFile.location(), 109L)); + Pair posDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + deletes, + formatVersion); + tbl.newRowDelta() + .addDeletes(posDeletes.first()) + .validateDataFilesExist(posDeletes.second()) + .commit(); + + assertThat(rowSet(tblName, tbl, "*")).hasSize(193); + } + + @TestTemplate + public void testEqualityDeleteWithDifferentScanAndDeleteColumns() throws IOException { + assumeThat(format).isEqualTo(FileFormat.PARQUET); + initDateTable(); + + Schema deleteRowSchema = dateTable.schema().select("dt"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + List dataDeletes = + Lists.newArrayList( + dataDelete.copy("dt", LocalDate.parse("2021-09-01")), + dataDelete.copy("dt", LocalDate.parse("2021-09-02")), + dataDelete.copy("dt", LocalDate.parse("2021-09-03"))); + + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + dateTable, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + dataDeletes.subList(0, 3), + deleteRowSchema); + + dateTable.newRowDelta().addDeletes(eqDeletes).commit(); + + CloseableIterable tasks = + TableScanUtil.planTasks( + dateTable.newScan().planFiles(), + TableProperties.METADATA_SPLIT_SIZE_DEFAULT, + TableProperties.SPLIT_LOOKBACK_DEFAULT, + TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT); + + ParquetBatchReadConf conf = + ImmutableParquetBatchReadConf.builder() + .batchSize(7) + .readerType(ParquetReaderType.ICEBERG) + .build(); + + for (CombinedScanTask task : tasks) { + try (BatchDataReader reader = + new BatchDataReader( + // expected column is id, while the equality filter column is dt + dateTable, + task, + dateTable.schema(), + dateTable.schema().select("id"), + false, + conf, + null, + true)) { + while (reader.next()) { + ColumnarBatch columnarBatch = reader.get(); + int numOfCols = columnarBatch.numCols(); + assertThat(numOfCols).as("Number of columns").isEqualTo(1); + assertThat(columnarBatch.column(0).dataType()).as("Column type").isEqualTo(IntegerType); + } + } + } + } + + private static final Schema PROJECTION_SCHEMA = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + MetadataColumns.IS_DELETED); + + private static StructLikeSet expectedRowSet(int... idsToRemove) { + return expectedRowSet(false, false, idsToRemove); + } + + private static StructLikeSet expectedRowSetWithDeletesOnly(int... idsToRemove) { + return expectedRowSet(false, true, idsToRemove); + } + + private static StructLikeSet expectedRowSetWithNonDeletesOnly(int... idsToRemove) { + return expectedRowSet(true, false, idsToRemove); + } + + private static StructLikeSet expectedRowSet( + boolean removeDeleted, boolean removeNonDeleted, int... idsToRemove) { + Set deletedIds = Sets.newHashSet(ArrayUtil.toIntList(idsToRemove)); + List records = recordsWithDeletedColumn(); + // mark rows deleted + records.forEach( + record -> { + if (deletedIds.contains(record.getField("id"))) { + record.setField(MetadataColumns.IS_DELETED.name(), true); + } + }); + + records.removeIf(record -> deletedIds.contains(record.getField("id")) && removeDeleted); + records.removeIf(record -> !deletedIds.contains(record.getField("id")) && removeNonDeleted); + + StructLikeSet set = StructLikeSet.create(PROJECTION_SCHEMA.asStruct()); + records.forEach( + record -> set.add(new InternalRecordWrapper(PROJECTION_SCHEMA.asStruct()).wrap(record))); + + return set; + } + + @Nonnull + private static List recordsWithDeletedColumn() { + List records = Lists.newArrayList(); + + // records all use IDs that are in bucket id_bucket=0 + GenericRecord record = GenericRecord.create(PROJECTION_SCHEMA); + records.add(record.copy("id", 29, "data", "a", "_deleted", false)); + records.add(record.copy("id", 43, "data", "b", "_deleted", false)); + records.add(record.copy("id", 61, "data", "c", "_deleted", false)); + records.add(record.copy("id", 89, "data", "d", "_deleted", false)); + records.add(record.copy("id", 100, "data", "e", "_deleted", false)); + records.add(record.copy("id", 121, "data", "f", "_deleted", false)); + records.add(record.copy("id", 122, "data", "g", "_deleted", false)); + return records; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java new file mode 100644 index 000000000000..d22ecb02d483 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java @@ -0,0 +1,376 @@ +/* + * 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.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; +import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX; +import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; +import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT; +import static org.apache.iceberg.data.FileHelpers.encrypt; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.file.Path; +import java.time.LocalDate; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers.Row; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericFileWriterFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.io.OutputFile; +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.SparkValueConverter; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +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; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkReaderWithBloomFilter { + + protected String tableName = null; + protected Table table = null; + protected List records = null; + protected DataFile dataFile = null; + + private static TestHiveMetastore metastore = null; + protected static SparkSession spark = null; + protected static HiveCatalog catalog = null; + + @Parameter(index = 0) + protected boolean vectorized; + + @Parameter(index = 1) + protected boolean useBloomFilter; + + // Schema passed to create tables + public static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "id_long", Types.LongType.get()), + Types.NestedField.required(3, "id_double", Types.DoubleType.get()), + Types.NestedField.required(4, "id_float", Types.FloatType.get()), + Types.NestedField.required(5, "id_string", Types.StringType.get()), + Types.NestedField.optional(6, "id_boolean", Types.BooleanType.get()), + Types.NestedField.optional(7, "id_date", Types.DateType.get()), + Types.NestedField.optional(8, "id_int_decimal", Types.DecimalType.of(8, 2)), + Types.NestedField.optional(9, "id_long_decimal", Types.DecimalType.of(14, 2)), + Types.NestedField.optional(10, "id_fixed_decimal", Types.DecimalType.of(31, 2))); + + private static final int INT_MIN_VALUE = 30; + private static final int INT_MAX_VALUE = 329; + private static final int INT_VALUE_COUNT = INT_MAX_VALUE - INT_MIN_VALUE + 1; + private static final long LONG_BASE = 1000L; + private static final double DOUBLE_BASE = 10000D; + private static final float FLOAT_BASE = 100000F; + private static final String BINARY_PREFIX = "BINARY测试_"; + + @TempDir private Path temp; + + @BeforeEach + public void writeTestDataFile() throws IOException { + this.tableName = "test"; + createTable(tableName, SCHEMA); + this.records = Lists.newArrayList(); + + // records all use IDs that are in bucket id_bucket=0 + GenericRecord record = GenericRecord.create(table.schema()); + + for (int i = 0; i < INT_VALUE_COUNT; i += 1) { + records.add( + record.copy( + ImmutableMap.of( + "id", + INT_MIN_VALUE + i, + "id_long", + LONG_BASE + INT_MIN_VALUE + i, + "id_double", + DOUBLE_BASE + INT_MIN_VALUE + i, + "id_float", + FLOAT_BASE + INT_MIN_VALUE + i, + "id_string", + BINARY_PREFIX + (INT_MIN_VALUE + i), + "id_boolean", + i % 2 == 0, + "id_date", + LocalDate.parse("2021-09-05"), + "id_int_decimal", + new BigDecimal(String.valueOf(77.77)), + "id_long_decimal", + new BigDecimal(String.valueOf(88.88)), + "id_fixed_decimal", + new BigDecimal(String.valueOf(99.99))))); + } + + this.dataFile = + writeDataFile( + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + Row.of(0), + records); + + table.newAppend().appendFile(dataFile).commit(); + } + + @AfterEach + public void cleanup() throws IOException { + dropTable("test"); + } + + @Parameters(name = "vectorized = {0}, useBloomFilter = {1}") + public static Object[][] parameters() { + return new Object[][] {{false, false}, {true, false}, {false, true}, {true, true}}; + } + + @BeforeAll + public static void startMetastoreAndSpark() { + metastore = new TestHiveMetastore(); + metastore.start(); + HiveConf hiveConf = metastore.hiveConf(); + + spark = + SparkSession.builder() + .master("local[2]") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) + .enableHiveSupport() + .getOrCreate(); + + catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + + try { + catalog.createNamespace(Namespace.of("default")); + } catch (AlreadyExistsException ignored) { + // the default namespace already exists. ignore the create error + } + } + + @AfterAll + public static void stopMetastoreAndSpark() throws Exception { + catalog = null; + metastore.stop(); + metastore = null; + spark.stop(); + spark = null; + } + + protected void createTable(String name, Schema schema) { + table = catalog.createTable(TableIdentifier.of("default", name), schema); + TableOperations ops = ((BaseTable) table).operations(); + TableMetadata meta = ops.current(); + ops.commit(meta, meta.upgradeToFormatVersion(2)); + + if (useBloomFilter) { + table + .updateProperties() + .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id", "true") + .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_long", "true") + .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_double", "true") + .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_float", "true") + .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_string", "true") + .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_boolean", "true") + .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_date", "true") + .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_int_decimal", "true") + .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_long_decimal", "true") + .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_fixed_decimal", "true") + .commit(); + } + + table + .updateProperties() + .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, "100") // to have multiple row groups + .commit(); + if (vectorized) { + table + .updateProperties() + .set(TableProperties.PARQUET_VECTORIZATION_ENABLED, "true") + .set(TableProperties.PARQUET_BATCH_SIZE, "4") + .commit(); + } + } + + protected void dropTable(String name) { + catalog.dropTable(TableIdentifier.of("default", name)); + } + + private DataFile writeDataFile(OutputFile out, StructLike partition, List rows) + throws IOException { + FileFormat format = defaultFormat(table.properties()); + Map writeProperties = Maps.newHashMap(); + + boolean useBloomFilterCol1 = + PropertyUtil.propertyAsBoolean( + table.properties(), PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id", false); + writeProperties.put( + PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id", Boolean.toString(useBloomFilterCol1)); + boolean useBloomFilterCol2 = + PropertyUtil.propertyAsBoolean( + table.properties(), PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_long", false); + writeProperties.put( + PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_long", + Boolean.toString(useBloomFilterCol2)); + boolean useBloomFilterCol3 = + PropertyUtil.propertyAsBoolean( + table.properties(), PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_double", false); + writeProperties.put( + PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_double", + Boolean.toString(useBloomFilterCol3)); + boolean useBloomFilterCol4 = + PropertyUtil.propertyAsBoolean( + table.properties(), PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_float", false); + writeProperties.put( + PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_float", + Boolean.toString(useBloomFilterCol4)); + boolean useBloomFilterCol5 = + PropertyUtil.propertyAsBoolean( + table.properties(), PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_string", false); + writeProperties.put( + PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_string", + Boolean.toString(useBloomFilterCol5)); + boolean useBloomFilterCol6 = + PropertyUtil.propertyAsBoolean( + table.properties(), PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_boolean", false); + writeProperties.put( + PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_boolean", + Boolean.toString(useBloomFilterCol6)); + boolean useBloomFilterCol7 = + PropertyUtil.propertyAsBoolean( + table.properties(), PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_date", false); + writeProperties.put( + PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_date", + Boolean.toString(useBloomFilterCol7)); + boolean useBloomFilterCol8 = + PropertyUtil.propertyAsBoolean( + table.properties(), + PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_int_decimal", + false); + writeProperties.put( + PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_int_decimal", + Boolean.toString(useBloomFilterCol8)); + boolean useBloomFilterCol9 = + PropertyUtil.propertyAsBoolean( + table.properties(), + PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_long_decimal", + false); + writeProperties.put( + PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_long_decimal", + Boolean.toString(useBloomFilterCol9)); + boolean useBloomFilterCol10 = + PropertyUtil.propertyAsBoolean( + table.properties(), + PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_fixed_decimal", + false); + writeProperties.put( + PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id_fixed_decimal", + Boolean.toString(useBloomFilterCol10)); + int blockSize = + PropertyUtil.propertyAsInt( + table.properties(), PARQUET_ROW_GROUP_SIZE_BYTES, PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT); + writeProperties.put(PARQUET_ROW_GROUP_SIZE_BYTES, Integer.toString(blockSize)); + + DataWriter writer = + new GenericFileWriterFactory.Builder() + .dataSchema(table.schema()) + .dataFileFormat(format) + .writerProperties(writeProperties) + .build() + .newDataWriter(encrypt(out), table.spec(), partition); + try (writer) { + writer.write(rows); + } + + return writer.toDataFile(); + } + + private FileFormat defaultFormat(Map properties) { + String formatString = properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); + return FileFormat.fromString(formatString); + } + + @TestTemplate + public void testReadWithFilter() { + Dataset df = + spark + .read() + .format("iceberg") + .load(TableIdentifier.of("default", tableName).toString()) + // this is from the first row group + .filter( + "id = 30 AND id_long = 1030 AND id_double = 10030.0 AND id_float = 100030.0" + + " AND id_string = 'BINARY测试_30' AND id_boolean = true AND id_date = '2021-09-05'" + + " AND id_int_decimal = 77.77 AND id_long_decimal = 88.88 AND id_fixed_decimal = 99.99"); + + Record record = SparkValueConverter.convert(table.schema(), df.collectAsList().get(0)); + + assertThat(df.collectAsList()).as("Table should contain 1 row").hasSize(1); + assertThat(record.get(0)).as("Table should contain expected rows").isEqualTo(30); + + df = + spark + .read() + .format("iceberg") + .load(TableIdentifier.of("default", tableName).toString()) + // this is from the third row group + .filter( + "id = 250 AND id_long = 1250 AND id_double = 10250.0 AND id_float = 100250.0" + + " AND id_string = 'BINARY测试_250' AND id_boolean = true AND id_date = '2021-09-05'" + + " AND id_int_decimal = 77.77 AND id_long_decimal = 88.88 AND id_fixed_decimal = 99.99"); + + record = SparkValueConverter.convert(table.schema(), df.collectAsList().get(0)); + + assertThat(df.collectAsList()).as("Table should contain 1 row").hasSize(1); + assertThat(record.get(0)).as("Table should contain expected rows").isEqualTo(250); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java new file mode 100644 index 000000000000..5ebeafcb8cef --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java @@ -0,0 +1,55 @@ +/* + * 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 org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestRollingFileWriters; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.unsafe.types.UTF8String; + +public class TestSparkRollingFileWriters extends TestRollingFileWriters { + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return SparkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected InternalRow toRow(Integer id, String data) { + InternalRow row = new GenericInternalRow(2); + row.update(0, id); + row.update(1, UTF8String.fromString(data)); + return row; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java new file mode 100644 index 000000000000..1ddf9318f608 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java @@ -0,0 +1,1089 @@ +/* + * 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.puffin.StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1; +import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.createPartitionedTable; +import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.createUnpartitionedTable; +import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToDayOrdinal; +import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToHourOrdinal; +import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToMonthOrdinal; +import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToYearOrdinal; +import static org.apache.spark.sql.functions.date_add; +import static org.apache.spark.sql.functions.expr; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +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.Lists; +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.SparkSQLProperties; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.iceberg.spark.functions.BucketFunction; +import org.apache.iceberg.spark.functions.DaysFunction; +import org.apache.iceberg.spark.functions.HoursFunction; +import org.apache.iceberg.spark.functions.MonthsFunction; +import org.apache.iceberg.spark.functions.TruncateFunction; +import org.apache.iceberg.spark.functions.YearsFunction; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.connector.catalog.functions.BoundFunction; +import org.apache.spark.sql.connector.expressions.Expression; +import org.apache.spark.sql.connector.expressions.FieldReference; +import org.apache.spark.sql.connector.expressions.LiteralValue; +import org.apache.spark.sql.connector.expressions.NamedReference; +import org.apache.spark.sql.connector.expressions.UserDefinedScalarFunc; +import org.apache.spark.sql.connector.expressions.filter.And; +import org.apache.spark.sql.connector.expressions.filter.Not; +import org.apache.spark.sql.connector.expressions.filter.Or; +import org.apache.spark.sql.connector.expressions.filter.Predicate; +import org.apache.spark.sql.connector.read.Batch; +import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.connector.read.Statistics; +import org.apache.spark.sql.connector.read.SupportsPushDownV2Filters; +import org.apache.spark.sql.connector.read.colstats.ColumnStatistics; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkScan extends TestBaseWithCatalog { + + private static final String DUMMY_BLOB_TYPE = "sum-data-size-bytes-v1"; + + @Parameter(index = 3) + private String format; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, format = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + "parquet" + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + "avro" + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + "orc" + } + }; + } + + @BeforeEach + public void useCatalog() { + sql("USE %s", catalogName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testEstimatedRowCount() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT, date DATE) USING iceberg TBLPROPERTIES('%s' = '%s')", + tableName, TableProperties.DEFAULT_FILE_FORMAT, format); + + Dataset df = + spark + .range(10000) + .withColumn("date", date_add(expr("DATE '1970-01-01'"), expr("CAST(id AS INT)"))) + .select("id", "date"); + + df.coalesce(1).writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + Statistics stats = scan.estimateStatistics(); + + assertThat(stats.numRows().getAsLong()).isEqualTo(10000L); + } + + @TestTemplate + public void testTableWithoutColStats() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + // The expected col NDVs are nulls + withSQLConf( + reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, Maps.newHashMap())); + } + + @TestTemplate + public void testTableWithoutApacheDatasketchColStat() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + DUMMY_BLOB_TYPE, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("data_size", "4")))); + + table.updateStatistics().setStatistics(statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + // The expected col NDVs are nulls + withSQLConf( + reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, Maps.newHashMap())); + } + + @TestTemplate + public void testTableWithOneColStats() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("ndv", "4")))); + + table.updateStatistics().setStatistics(statisticsFile).commit(); + + List newRecords = + Lists.newArrayList(new SimpleRecord(5, "a"), new SimpleRecord(6, "b")); + spark + .createDataset(newRecords, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + table.refresh(); + long snapshotId2 = table.currentSnapshot().snapshotId(); + + GenericStatisticsFile statisticsFile2 = + new GenericStatisticsFile( + snapshotId2, + "/test/statistics/file2.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId2, + 2, + ImmutableList.of(1), + ImmutableMap.of("ndv", "6")))); + + table.updateStatistics().setStatistics(statisticsFile2).commit(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + checkColStatisticsNotReported(scan, 6L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 6L)); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + Map expectedNDV = Maps.newHashMap(); + expectedNDV.put("id", 6L); + withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 6L, expectedNDV)); + } + + @TestTemplate + public void testTableWithOneApacheDatasketchColStatAndOneDifferentColStat() + throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("ndv", "4")), + new GenericBlobMetadata( + DUMMY_BLOB_TYPE, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("data_size", "2")))); + + table.updateStatistics().setStatistics(statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + + Map expectedOneNDV = Maps.newHashMap(); + expectedOneNDV.put("id", 4L); + withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, expectedOneNDV)); + } + + @TestTemplate + public void testTableWithTwoColStats() throws NoSuchTableException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "a"), + new SimpleRecord(4, "b")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + + SparkScanBuilder scanBuilder = + new SparkScanBuilder(spark, table, CaseInsensitiveStringMap.empty()); + SparkScan scan = (SparkScan) scanBuilder.build(); + + Map reportColStatsDisabled = + ImmutableMap.of( + SQLConf.CBO_ENABLED().key(), "true", SparkSQLProperties.REPORT_COLUMN_STATS, "false"); + + Map reportColStatsEnabled = + ImmutableMap.of(SQLConf.CBO_ENABLED().key(), "true"); + + GenericStatisticsFile statisticsFile = + new GenericStatisticsFile( + snapshotId, + "/test/statistics/file.puffin", + 100, + 42, + ImmutableList.of( + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(1), + ImmutableMap.of("ndv", "4")), + new GenericBlobMetadata( + APACHE_DATASKETCHES_THETA_V1, + snapshotId, + 1, + ImmutableList.of(2), + ImmutableMap.of("ndv", "2")))); + + table.updateStatistics().setStatistics(statisticsFile).commit(); + + checkColStatisticsNotReported(scan, 4L); + withSQLConf(reportColStatsDisabled, () -> checkColStatisticsNotReported(scan, 4L)); + + Map expectedTwoNDVs = Maps.newHashMap(); + expectedTwoNDVs.put("id", 4L); + expectedTwoNDVs.put("data", 2L); + withSQLConf(reportColStatsEnabled, () -> checkColStatisticsReported(scan, 4L, expectedTwoNDVs)); + } + + @TestTemplate + public void testUnpartitionedYears() throws Exception { + createUnpartitionedTable(spark, tableName); + + SparkScanBuilder builder = scanBuilder(); + + YearsFunction.TimestampToYearsFunction function = new YearsFunction.TimestampToYearsFunction(); + UserDefinedScalarFunc udf = toUDF(function, expressions(fieldRef("ts"))); + Predicate predicate = + new Predicate( + "=", + expressions( + udf, intLit(timestampStrToYearOrdinal("2017-11-22T00:00:00.000000+00:00")))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + + // NOT Equal + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + // notEq can't be answered using column bounds because they are not exact + assertThat(scan.planInputPartitions()).hasSize(10); + } + + @TestTemplate + public void testPartitionedYears() throws Exception { + createPartitionedTable(spark, tableName, "years(ts)"); + + SparkScanBuilder builder = scanBuilder(); + + YearsFunction.TimestampToYearsFunction function = new YearsFunction.TimestampToYearsFunction(); + UserDefinedScalarFunc udf = toUDF(function, expressions(fieldRef("ts"))); + Predicate predicate = + new Predicate( + "=", + expressions( + udf, intLit(timestampStrToYearOrdinal("2017-11-22T00:00:00.000000+00:00")))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + + // NOT Equal + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + } + + @TestTemplate + public void testUnpartitionedMonths() throws Exception { + createUnpartitionedTable(spark, tableName); + + SparkScanBuilder builder = scanBuilder(); + + MonthsFunction.TimestampToMonthsFunction function = + new MonthsFunction.TimestampToMonthsFunction(); + UserDefinedScalarFunc udf = toUDF(function, expressions(fieldRef("ts"))); + Predicate predicate = + new Predicate( + ">", + expressions( + udf, intLit(timestampStrToMonthOrdinal("2017-11-22T00:00:00.000000+00:00")))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + + // NOT GT + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + } + + @TestTemplate + public void testPartitionedMonths() throws Exception { + createPartitionedTable(spark, tableName, "months(ts)"); + + SparkScanBuilder builder = scanBuilder(); + + MonthsFunction.TimestampToMonthsFunction function = + new MonthsFunction.TimestampToMonthsFunction(); + UserDefinedScalarFunc udf = toUDF(function, expressions(fieldRef("ts"))); + Predicate predicate = + new Predicate( + ">", + expressions( + udf, intLit(timestampStrToMonthOrdinal("2017-11-22T00:00:00.000000+00:00")))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + + // NOT GT + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + } + + @TestTemplate + public void testUnpartitionedDays() throws Exception { + createUnpartitionedTable(spark, tableName); + + SparkScanBuilder builder = scanBuilder(); + + DaysFunction.TimestampToDaysFunction function = new DaysFunction.TimestampToDaysFunction(); + UserDefinedScalarFunc udf = toUDF(function, expressions(fieldRef("ts"))); + Predicate predicate = + new Predicate( + "<", + expressions( + udf, dateLit(timestampStrToDayOrdinal("2018-11-20T00:00:00.000000+00:00")))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + + // NOT LT + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + } + + @TestTemplate + public void testPartitionedDays() throws Exception { + createPartitionedTable(spark, tableName, "days(ts)"); + + SparkScanBuilder builder = scanBuilder(); + + DaysFunction.TimestampToDaysFunction function = new DaysFunction.TimestampToDaysFunction(); + UserDefinedScalarFunc udf = toUDF(function, expressions(fieldRef("ts"))); + Predicate predicate = + new Predicate( + "<", + expressions( + udf, dateLit(timestampStrToDayOrdinal("2018-11-20T00:00:00.000000+00:00")))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + + // NOT LT + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + } + + @TestTemplate + public void testUnpartitionedHours() throws Exception { + createUnpartitionedTable(spark, tableName); + + SparkScanBuilder builder = scanBuilder(); + + HoursFunction.TimestampToHoursFunction function = new HoursFunction.TimestampToHoursFunction(); + UserDefinedScalarFunc udf = toUDF(function, expressions(fieldRef("ts"))); + Predicate predicate = + new Predicate( + ">=", + expressions( + udf, intLit(timestampStrToHourOrdinal("2017-11-22T06:02:09.243857+00:00")))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(8); + + // NOT GTEQ + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(2); + } + + @TestTemplate + public void testPartitionedHours() throws Exception { + createPartitionedTable(spark, tableName, "hours(ts)"); + + SparkScanBuilder builder = scanBuilder(); + + HoursFunction.TimestampToHoursFunction function = new HoursFunction.TimestampToHoursFunction(); + UserDefinedScalarFunc udf = toUDF(function, expressions(fieldRef("ts"))); + Predicate predicate = + new Predicate( + ">=", + expressions( + udf, intLit(timestampStrToHourOrdinal("2017-11-22T06:02:09.243857+00:00")))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(8); + + // NOT GTEQ + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(2); + } + + @TestTemplate + public void testUnpartitionedBucketLong() throws Exception { + createUnpartitionedTable(spark, tableName); + + SparkScanBuilder builder = scanBuilder(); + + BucketFunction.BucketLong function = new BucketFunction.BucketLong(DataTypes.LongType); + UserDefinedScalarFunc udf = toUDF(function, expressions(intLit(5), fieldRef("id"))); + Predicate predicate = new Predicate(">=", expressions(udf, intLit(2))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(10); + + // NOT GTEQ + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(10); + } + + @TestTemplate + public void testPartitionedBucketLong() throws Exception { + createPartitionedTable(spark, tableName, "bucket(5, id)"); + + SparkScanBuilder builder = scanBuilder(); + + BucketFunction.BucketLong function = new BucketFunction.BucketLong(DataTypes.LongType); + UserDefinedScalarFunc udf = toUDF(function, expressions(intLit(5), fieldRef("id"))); + Predicate predicate = new Predicate(">=", expressions(udf, intLit(2))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(6); + + // NOT GTEQ + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(4); + } + + @TestTemplate + public void testUnpartitionedBucketString() throws Exception { + createUnpartitionedTable(spark, tableName); + + SparkScanBuilder builder = scanBuilder(); + + BucketFunction.BucketString function = new BucketFunction.BucketString(); + UserDefinedScalarFunc udf = toUDF(function, expressions(intLit(5), fieldRef("data"))); + Predicate predicate = new Predicate("<=", expressions(udf, intLit(2))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(10); + + // NOT LTEQ + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(10); + } + + @TestTemplate + public void testPartitionedBucketString() throws Exception { + createPartitionedTable(spark, tableName, "bucket(5, data)"); + + SparkScanBuilder builder = scanBuilder(); + + BucketFunction.BucketString function = new BucketFunction.BucketString(); + UserDefinedScalarFunc udf = toUDF(function, expressions(intLit(5), fieldRef("data"))); + Predicate predicate = new Predicate("<=", expressions(udf, intLit(2))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(6); + + // NOT LTEQ + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(4); + } + + @TestTemplate + public void testUnpartitionedTruncateString() throws Exception { + createUnpartitionedTable(spark, tableName); + + SparkScanBuilder builder = scanBuilder(); + + TruncateFunction.TruncateString function = new TruncateFunction.TruncateString(); + UserDefinedScalarFunc udf = toUDF(function, expressions(intLit(4), fieldRef("data"))); + Predicate predicate = new Predicate("<>", expressions(udf, stringLit("data"))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(10); + + // NOT NotEqual + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + } + + @TestTemplate + public void testPartitionedTruncateString() throws Exception { + createPartitionedTable(spark, tableName, "truncate(4, data)"); + + SparkScanBuilder builder = scanBuilder(); + + TruncateFunction.TruncateString function = new TruncateFunction.TruncateString(); + UserDefinedScalarFunc udf = toUDF(function, expressions(intLit(4), fieldRef("data"))); + Predicate predicate = new Predicate("<>", expressions(udf, stringLit("data"))); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + + // NOT NotEqual + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + } + + @TestTemplate + public void testUnpartitionedIsNull() throws Exception { + createUnpartitionedTable(spark, tableName); + + SparkScanBuilder builder = scanBuilder(); + + TruncateFunction.TruncateString function = new TruncateFunction.TruncateString(); + UserDefinedScalarFunc udf = toUDF(function, expressions(intLit(4), fieldRef("data"))); + Predicate predicate = new Predicate("IS_NULL", expressions(udf)); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).isEmpty(); + + // NOT IsNull + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(10); + } + + @TestTemplate + public void testPartitionedIsNull() throws Exception { + createPartitionedTable(spark, tableName, "truncate(4, data)"); + + SparkScanBuilder builder = scanBuilder(); + + TruncateFunction.TruncateString function = new TruncateFunction.TruncateString(); + UserDefinedScalarFunc udf = toUDF(function, expressions(intLit(4), fieldRef("data"))); + Predicate predicate = new Predicate("IS_NULL", expressions(udf)); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).isEmpty(); + + // NOT IsNULL + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(10); + } + + @TestTemplate + public void testUnpartitionedIsNotNull() throws Exception { + createUnpartitionedTable(spark, tableName); + + SparkScanBuilder builder = scanBuilder(); + + TruncateFunction.TruncateString function = new TruncateFunction.TruncateString(); + UserDefinedScalarFunc udf = toUDF(function, expressions(intLit(4), fieldRef("data"))); + Predicate predicate = new Predicate("IS_NOT_NULL", expressions(udf)); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(10); + + // NOT IsNotNull + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).isEmpty(); + } + + @TestTemplate + public void testPartitionedIsNotNull() throws Exception { + createPartitionedTable(spark, tableName, "truncate(4, data)"); + + SparkScanBuilder builder = scanBuilder(); + + TruncateFunction.TruncateString function = new TruncateFunction.TruncateString(); + UserDefinedScalarFunc udf = toUDF(function, expressions(intLit(4), fieldRef("data"))); + Predicate predicate = new Predicate("IS_NOT_NULL", expressions(udf)); + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(10); + + // NOT IsNotNULL + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).isEmpty(); + } + + @TestTemplate + public void testUnpartitionedAnd() throws Exception { + createUnpartitionedTable(spark, tableName); + + SparkScanBuilder builder = scanBuilder(); + + YearsFunction.TimestampToYearsFunction tsToYears = new YearsFunction.TimestampToYearsFunction(); + UserDefinedScalarFunc udf1 = toUDF(tsToYears, expressions(fieldRef("ts"))); + Predicate predicate1 = new Predicate("=", expressions(udf1, intLit(2017 - 1970))); + + BucketFunction.BucketLong bucketLong = new BucketFunction.BucketLong(DataTypes.LongType); + UserDefinedScalarFunc udf = toUDF(bucketLong, expressions(intLit(5), fieldRef("id"))); + Predicate predicate2 = new Predicate(">=", expressions(udf, intLit(2))); + Predicate predicate = new And(predicate1, predicate2); + + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(5); + + // NOT (years(ts) = 47 AND bucket(id, 5) >= 2) + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(10); + } + + @TestTemplate + public void testPartitionedAnd() throws Exception { + createPartitionedTable(spark, tableName, "years(ts), bucket(5, id)"); + + SparkScanBuilder builder = scanBuilder(); + + YearsFunction.TimestampToYearsFunction tsToYears = new YearsFunction.TimestampToYearsFunction(); + UserDefinedScalarFunc udf1 = toUDF(tsToYears, expressions(fieldRef("ts"))); + Predicate predicate1 = new Predicate("=", expressions(udf1, intLit(2017 - 1970))); + + BucketFunction.BucketLong bucketLong = new BucketFunction.BucketLong(DataTypes.LongType); + UserDefinedScalarFunc udf = toUDF(bucketLong, expressions(intLit(5), fieldRef("id"))); + Predicate predicate2 = new Predicate(">=", expressions(udf, intLit(2))); + Predicate predicate = new And(predicate1, predicate2); + + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(1); + + // NOT (years(ts) = 47 AND bucket(id, 5) >= 2) + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(9); + } + + @TestTemplate + public void testUnpartitionedOr() throws Exception { + createUnpartitionedTable(spark, tableName); + + SparkScanBuilder builder = scanBuilder(); + + YearsFunction.TimestampToYearsFunction tsToYears = new YearsFunction.TimestampToYearsFunction(); + UserDefinedScalarFunc udf1 = toUDF(tsToYears, expressions(fieldRef("ts"))); + Predicate predicate1 = new Predicate("=", expressions(udf1, intLit(2017 - 1970))); + + BucketFunction.BucketLong bucketLong = new BucketFunction.BucketLong(DataTypes.LongType); + UserDefinedScalarFunc udf = toUDF(bucketLong, expressions(intLit(5), fieldRef("id"))); + Predicate predicate2 = new Predicate(">=", expressions(udf, intLit(2))); + Predicate predicate = new Or(predicate1, predicate2); + + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(10); + + // NOT (years(ts) = 47 OR bucket(id, 5) >= 2) + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(10); + } + + @TestTemplate + public void testPartitionedOr() throws Exception { + createPartitionedTable(spark, tableName, "years(ts), bucket(5, id)"); + + SparkScanBuilder builder = scanBuilder(); + + YearsFunction.TimestampToYearsFunction tsToYears = new YearsFunction.TimestampToYearsFunction(); + UserDefinedScalarFunc udf1 = toUDF(tsToYears, expressions(fieldRef("ts"))); + Predicate predicate1 = new Predicate("=", expressions(udf1, intLit(2018 - 1970))); + + BucketFunction.BucketLong bucketLong = new BucketFunction.BucketLong(DataTypes.LongType); + UserDefinedScalarFunc udf = toUDF(bucketLong, expressions(intLit(5), fieldRef("id"))); + Predicate predicate2 = new Predicate(">=", expressions(udf, intLit(2))); + Predicate predicate = new Or(predicate1, predicate2); + + pushFilters(builder, predicate); + Batch scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(6); + + // NOT (years(ts) = 48 OR bucket(id, 5) >= 2) + builder = scanBuilder(); + + predicate = new Not(predicate); + pushFilters(builder, predicate); + scan = builder.build().toBatch(); + + assertThat(scan.planInputPartitions()).hasSize(4); + } + + private SparkScanBuilder scanBuilder() throws Exception { + Table table = Spark3Util.loadIcebergTable(spark, tableName); + CaseInsensitiveStringMap options = + new CaseInsensitiveStringMap(ImmutableMap.of("path", tableName)); + + return new SparkScanBuilder(spark, table, options); + } + + private void pushFilters(ScanBuilder scan, Predicate... predicates) { + assertThat(scan).isInstanceOf(SupportsPushDownV2Filters.class); + SupportsPushDownV2Filters filterable = (SupportsPushDownV2Filters) scan; + filterable.pushPredicates(predicates); + } + + private Expression[] expressions(Expression... expressions) { + return expressions; + } + + private void checkColStatisticsNotReported(SparkScan scan, long expectedRowCount) { + Statistics stats = scan.estimateStatistics(); + assertThat(stats.numRows().getAsLong()).isEqualTo(expectedRowCount); + + Map columnStats = stats.columnStats(); + assertThat(columnStats).isEmpty(); + } + + private void checkColStatisticsReported( + SparkScan scan, long expectedRowCount, Map expectedNDVs) { + Statistics stats = scan.estimateStatistics(); + assertThat(stats.numRows().getAsLong()).isEqualTo(expectedRowCount); + + Map columnStats = stats.columnStats(); + if (expectedNDVs.isEmpty()) { + assertThat(columnStats.values().stream().allMatch(value -> value.distinctCount().isEmpty())) + .isTrue(); + } else { + for (Map.Entry entry : expectedNDVs.entrySet()) { + assertThat( + columnStats.get(FieldReference.column(entry.getKey())).distinctCount().getAsLong()) + .isEqualTo(entry.getValue()); + } + } + } + + private static LiteralValue intLit(int value) { + return LiteralValue.apply(value, DataTypes.IntegerType); + } + + private static LiteralValue dateLit(int value) { + return LiteralValue.apply(value, DataTypes.DateType); + } + + private static LiteralValue stringLit(String value) { + return LiteralValue.apply(value, DataTypes.StringType); + } + + private static NamedReference fieldRef(String col) { + return FieldReference.apply(col); + } + + private static UserDefinedScalarFunc toUDF(BoundFunction function, Expression[] expressions) { + return new UserDefinedScalarFunc(function.name(), function.canonicalName(), expressions); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java new file mode 100644 index 000000000000..e444b7cb1f7c --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkStagedScan.java @@ -0,0 +1,128 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.util.List; +import java.util.UUID; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.ScanTaskSetManager; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkStagedScan extends CatalogTestBase { + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testTaskSetLoading() throws NoSuchTableException, IOException { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + + List records = + ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should produce 1 snapshot").hasSize(1); + + try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { + ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + String setID = UUID.randomUUID().toString(); + taskSetManager.stageTasks(table, setID, ImmutableList.copyOf(fileScanTasks)); + + // load the staged file set + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, setID) + .load(tableName); + + // write the records back essentially duplicating data + scanDF.writeTo(tableName).append(); + } + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "a"), row(1, "a"), row(2, "b"), row(2, "b")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testTaskSetPlanning() throws NoSuchTableException, IOException { + sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName); + + List records = + ImmutableList.of(new SimpleRecord(1, "a"), new SimpleRecord(2, "b")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.coalesce(1).writeTo(tableName).append(); + df.coalesce(1).writeTo(tableName).append(); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should produce 2 snapshot").hasSize(2); + + try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { + ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + String setID = UUID.randomUUID().toString(); + List tasks = ImmutableList.copyOf(fileScanTasks); + taskSetManager.stageTasks(table, setID, tasks); + + // load the staged file set and make sure each file is in a separate split + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, setID) + .option(SparkReadOptions.SPLIT_SIZE, tasks.get(0).file().fileSizeInBytes()) + .load(tableName); + assertThat(scanDF.javaRDD().getNumPartitions()) + .as("Num partitions should match") + .isEqualTo(2); + + // load the staged file set and make sure we combine both files into a single split + scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SCAN_TASK_SET_ID, setID) + .option(SparkReadOptions.SPLIT_SIZE, Long.MAX_VALUE) + .load(tableName); + assertThat(scanDF.javaRDD().getNumPartitions()) + .as("Num partitions should match") + .isEqualTo(1); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java new file mode 100644 index 000000000000..d14b1a52cf82 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTable.java @@ -0,0 +1,59 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.connector.catalog.CatalogManager; +import org.apache.spark.sql.connector.catalog.Identifier; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkTable extends CatalogTestBase { + + @BeforeEach + public void createTable() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testTableEquality() throws NoSuchTableException { + CatalogManager catalogManager = spark.sessionState().catalogManager(); + TableCatalog catalog = (TableCatalog) catalogManager.catalog(catalogName); + Identifier identifier = Identifier.of(tableIdent.namespace().levels(), tableIdent.name()); + SparkTable table1 = (SparkTable) catalog.loadTable(identifier); + SparkTable table2 = (SparkTable) catalog.loadTable(identifier); + + // different instances pointing to the same table must be equivalent + assertThat(table1).as("References must be different").isNotSameAs(table2); + assertThat(table1).as("Tables must be equivalent").isEqualTo(table2); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkWriterMetrics.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkWriterMetrics.java new file mode 100644 index 000000000000..06ecc20c2fc3 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkWriterMetrics.java @@ -0,0 +1,67 @@ +/* + * 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.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestWriterMetrics; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.unsafe.types.UTF8String; + +public class TestSparkWriterMetrics extends TestWriterMetrics { + + public TestSparkWriterMetrics(FileFormat fileFormat) { + super(fileFormat); + } + + @Override + protected FileWriterFactory newWriterFactory(Table sourceTable) { + return SparkFileWriterFactory.builderFor(sourceTable) + .dataSchema(sourceTable.schema()) + .dataFileFormat(fileFormat) + .deleteFileFormat(fileFormat) + .positionDeleteRowSchema(sourceTable.schema()) + .build(); + } + + @Override + protected InternalRow toRow(Integer id, String data, boolean boolValue, Long longValue) { + InternalRow row = new GenericInternalRow(3); + row.update(0, id); + row.update(1, UTF8String.fromString(data)); + + InternalRow nested = new GenericInternalRow(2); + nested.update(0, boolValue); + nested.update(1, longValue); + + row.update(2, nested); + return row; + } + + @Override + protected InternalRow toGenericRow(int value, int repeated) { + InternalRow row = new GenericInternalRow(repeated); + for (int i = 0; i < repeated; i++) { + row.update(i, value); + } + return row; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java new file mode 100644 index 000000000000..d55e718ff2d3 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java @@ -0,0 +1,56 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.Arrays; +import org.apache.iceberg.util.JsonUtil; +import org.junit.jupiter.api.Test; + +public class TestStreamingOffset { + + @Test + public void testJsonConversion() { + StreamingOffset[] expected = + new StreamingOffset[] { + new StreamingOffset(System.currentTimeMillis(), 1L, false), + new StreamingOffset(System.currentTimeMillis(), 2L, false), + new StreamingOffset(System.currentTimeMillis(), 3L, false), + new StreamingOffset(System.currentTimeMillis(), 4L, true) + }; + assertThat(Arrays.stream(expected).map(elem -> StreamingOffset.fromJson(elem.json())).toArray()) + .as("StreamingOffsets should match") + .isEqualTo(expected); + } + + @Test + public void testToJson() throws Exception { + StreamingOffset expected = new StreamingOffset(System.currentTimeMillis(), 1L, false); + ObjectNode actual = JsonUtil.mapper().createObjectNode(); + actual.put("version", 1); + actual.put("snapshot_id", expected.snapshotId()); + actual.put("position", 1L); + actual.put("scan_all_files", false); + String expectedJson = expected.json(); + String actualJson = JsonUtil.mapper().writeValueAsString(actual); + assertThat(actualJson).isEqualTo(expectedJson); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructInternalRowVariant.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructInternalRowVariant.java new file mode 100644 index 000000000000..78e7e23b861b --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructInternalRowVariant.java @@ -0,0 +1,182 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.variants.Variant; +import org.apache.iceberg.variants.VariantMetadata; +import org.apache.iceberg.variants.VariantValue; +import org.apache.iceberg.variants.Variants; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.MapData; +import org.apache.spark.sql.types.VariantType$; +import org.apache.spark.unsafe.types.VariantVal; +import org.junit.jupiter.api.Test; + +public class TestStructInternalRowVariant { + + @Test + public void testGetVariantReturnsVariantVal() { + Types.StructType structType = variantStructType(); + GenericRecord rec = newRecord(structType); + Variant variant = sampleVariant(); + rec.set(0, variant); + + InternalRow row = new StructInternalRow(structType).setStruct(rec); + + VariantVal actual = row.getVariant(0); + assertThat(actual).isNotNull(); + + VariantMetadata metadata = + VariantMetadata.from(ByteBuffer.wrap(actual.getMetadata()).order(ByteOrder.LITTLE_ENDIAN)); + assertThat(metadata.dictionarySize()).isEqualTo(1); + assertThat(metadata.get(0)).isEqualTo("k"); + + VariantValue actualValue = + VariantValue.from( + metadata, ByteBuffer.wrap(actual.getValue()).order(ByteOrder.LITTLE_ENDIAN)); + + assertThat(actualValue.asObject().get("k").asPrimitive().get()).isEqualTo("v1"); + } + + @Test + public void testGetVariantNull() { + Types.StructType structType = variantStructType(); + GenericRecord rec = newRecord(structType); + rec.set(0, null); + + InternalRow row = new StructInternalRow(structType).setStruct(rec); + assertThat(row.getVariant(0)).isNull(); + } + + @Test + public void testArrayOfVariant() { + Types.ListType listType = Types.ListType.ofOptional(2, Types.VariantType.get()); + Types.StructType structType = + Types.StructType.of(Types.NestedField.optional(1, "arr", listType)); + + GenericRecord rec = GenericRecord.create(structType); + + Variant v1 = sampleVariant(); + Variant v2 = sampleVariant(); + + List elements = Arrays.asList(v1, v2, null); + rec.set(0, elements); + + InternalRow row = new StructInternalRow(structType).setStruct(rec); + ArrayData arr = row.getArray(0); + + Object firstVar = arr.get(0, VariantType$.MODULE$); + Object secondVar = arr.get(1, VariantType$.MODULE$); + + assertThat(firstVar).isInstanceOf(VariantVal.class); + assertThat(secondVar).isInstanceOf(VariantVal.class); + assertThat(arr.isNullAt(2)).isTrue(); + + assertVariantValEqualsKV((VariantVal) firstVar, "k", "v1"); + assertVariantValEqualsKV((VariantVal) secondVar, "k", "v1"); + } + + @Test + public void testMapWithVariant() { + Types.MapType mapType = + Types.MapType.ofOptional(2, 3, Types.StringType.get(), Types.VariantType.get()); + Types.StructType structType = Types.StructType.of(Types.NestedField.optional(1, "m", mapType)); + + GenericRecord rec = GenericRecord.create(structType); + Map map = Maps.newHashMap(); + map.put("a", sampleVariant()); + map.put("b", sampleVariant()); + rec.set(0, map); + + InternalRow row = new StructInternalRow(structType).setStruct(rec); + MapData mapData = row.getMap(0); + + ArrayData values = mapData.valueArray(); + for (int i = 0; i < values.numElements(); i++) { + Object variant = values.get(i, VariantType$.MODULE$); + assertThat(variant).isInstanceOf(VariantVal.class); + assertVariantValEqualsKV((VariantVal) variant, "k", "v1"); + } + } + + @Test + public void testNestedStructVariant() { + Types.StructType variant = + Types.StructType.of(Types.NestedField.optional(2, "v", Types.VariantType.get())); + Types.StructType structVariant = + Types.StructType.of(Types.NestedField.optional(1, "n", variant)); + + // Case 1: nested struct holds Iceberg Variant + GenericRecord variantStructRec = GenericRecord.create(variant); + variantStructRec.set(0, sampleVariant()); + GenericRecord structRec = GenericRecord.create(structVariant); + structRec.set(0, variantStructRec); + + InternalRow structRow = new StructInternalRow(structVariant).setStruct(structRec); + InternalRow nested = structRow.getStruct(0, 1); + VariantVal variantVal1 = nested.getVariant(0); + assertVariantValEqualsKV(variantVal1, "k", "v1"); + } + + @Test + public void testGetWithVariantType() { + Types.StructType structType = variantStructType(); + GenericRecord rec = newRecord(structType); + rec.set(0, sampleVariant()); + + InternalRow row = new StructInternalRow(structType).setStruct(rec); + Object obj = row.get(0, VariantType$.MODULE$); + assertThat(obj).isInstanceOf(VariantVal.class); + assertVariantValEqualsKV((VariantVal) obj, "k", "v1"); + } + + private static Types.StructType variantStructType() { + return Types.StructType.of(Types.NestedField.optional(1, "a", Types.VariantType.get())); + } + + private static GenericRecord newRecord(Types.StructType structType) { + return GenericRecord.create(structType); + } + + private static Variant sampleVariant() { + VariantMetadata md = Variants.metadata("k"); + org.apache.iceberg.variants.ShreddedObject obj = Variants.object(md); + obj.put("k", Variants.of("v1")); + return Variant.of(md, obj); + } + + private static void assertVariantValEqualsKV(VariantVal vv, String key, String expected) { + VariantMetadata metadata = + VariantMetadata.from(ByteBuffer.wrap(vv.getMetadata()).order(ByteOrder.LITTLE_ENDIAN)); + VariantValue value = + VariantValue.from(metadata, ByteBuffer.wrap(vv.getValue()).order(ByteOrder.LITTLE_ENDIAN)); + assertThat(value.asObject().get(key).asPrimitive().get()).isEqualTo(expected); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java new file mode 100644 index 000000000000..54048bbf218a --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java @@ -0,0 +1,309 @@ +/* + * 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.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.net.InetAddress; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.execution.streaming.MemoryStream; +import org.apache.spark.sql.streaming.DataStreamWriter; +import org.apache.spark.sql.streaming.StreamingQuery; +import org.apache.spark.sql.streaming.StreamingQueryException; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import scala.Option; +import scala.collection.JavaConverters; + +public class TestStructuredStreaming { + + private static final Configuration CONF = new Configuration(); + private static final Schema SCHEMA = + new Schema( + optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); + private static SparkSession spark = null; + + @TempDir private Path temp; + + @BeforeAll + public static void startSpark() { + TestStructuredStreaming.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .config("spark.sql.shuffle.partitions", 4) + .getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestStructuredStreaming.spark; + TestStructuredStreaming.spark = null; + currentSpark.stop(); + } + + @Test + public void testStreamingWriteAppendMode() throws Exception { + File parent = temp.resolve("parquet").toFile(); + File location = new File(parent, "test-table"); + File checkpoint = new File(parent, "checkpoint"); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + List expected = + Lists.newArrayList( + new SimpleRecord(1, "1"), + new SimpleRecord(2, "2"), + new SimpleRecord(3, "3"), + new SimpleRecord(4, "4")); + + MemoryStream inputStream = newMemoryStream(1, spark.sqlContext(), Encoders.INT()); + DataStreamWriter streamWriter = + inputStream + .toDF() + .selectExpr("value AS id", "CAST (value AS STRING) AS data") + .writeStream() + .outputMode("append") + .format("iceberg") + .option("checkpointLocation", checkpoint.toString()) + .option("path", location.toString()); + + try { + // start the original query with checkpointing + StreamingQuery query = streamWriter.start(); + List batch1 = Lists.newArrayList(1, 2); + send(batch1, inputStream); + query.processAllAvailable(); + List batch2 = Lists.newArrayList(3, 4); + send(batch2, inputStream); + query.processAllAvailable(); + query.stop(); + + // remove the last commit to force Spark to reprocess batch #1 + File lastCommitFile = new File(checkpoint + "/commits/1"); + assertThat(lastCommitFile.delete()).as("The commit file must be deleted").isTrue(); + Files.deleteIfExists(Paths.get(checkpoint + "/commits/.1.crc")); + + // restart the query from the checkpoint + StreamingQuery restartedQuery = streamWriter.start(); + restartedQuery.processAllAvailable(); + + // ensure the write was idempotent + Dataset result = spark.read().format("iceberg").load(location.toString()); + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + assertThat(table.snapshots()).as("Number of snapshots should match").hasSize(2); + } finally { + for (StreamingQuery query : spark.streams().active()) { + query.stop(); + } + } + } + + @Test + public void testStreamingWriteCompleteMode() throws Exception { + File parent = temp.resolve("parquet").toFile(); + File location = new File(parent, "test-table"); + File checkpoint = new File(parent, "checkpoint"); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + List expected = + Lists.newArrayList( + new SimpleRecord(2, "1"), new SimpleRecord(3, "2"), new SimpleRecord(1, "3")); + + MemoryStream inputStream = newMemoryStream(1, spark.sqlContext(), Encoders.INT()); + DataStreamWriter streamWriter = + inputStream + .toDF() + .groupBy("value") + .count() + .selectExpr("CAST(count AS INT) AS id", "CAST (value AS STRING) AS data") + .writeStream() + .outputMode("complete") + .format("iceberg") + .option("checkpointLocation", checkpoint.toString()) + .option("path", location.toString()); + + try { + // start the original query with checkpointing + StreamingQuery query = streamWriter.start(); + List batch1 = Lists.newArrayList(1, 2); + send(batch1, inputStream); + query.processAllAvailable(); + List batch2 = Lists.newArrayList(1, 2, 2, 3); + send(batch2, inputStream); + query.processAllAvailable(); + query.stop(); + + // remove the last commit to force Spark to reprocess batch #1 + File lastCommitFile = new File(checkpoint + "/commits/1"); + assertThat(lastCommitFile.delete()).as("The commit file must be deleted").isTrue(); + Files.deleteIfExists(Paths.get(checkpoint + "/commits/.1.crc")); + + // restart the query from the checkpoint + StreamingQuery restartedQuery = streamWriter.start(); + restartedQuery.processAllAvailable(); + + // ensure the write was idempotent + Dataset result = spark.read().format("iceberg").load(location.toString()); + List actual = + result.orderBy("data").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + assertThat(table.snapshots()).as("Number of snapshots should match").hasSize(2); + } finally { + for (StreamingQuery query : spark.streams().active()) { + query.stop(); + } + } + } + + @Test + public void testStreamingWriteCompleteModeWithProjection() throws Exception { + File parent = temp.resolve("parquet").toFile(); + File location = new File(parent, "test-table"); + File checkpoint = new File(parent, "checkpoint"); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + List expected = + Lists.newArrayList( + new SimpleRecord(1, null), new SimpleRecord(2, null), new SimpleRecord(3, null)); + + MemoryStream inputStream = newMemoryStream(1, spark.sqlContext(), Encoders.INT()); + DataStreamWriter streamWriter = + inputStream + .toDF() + .groupBy("value") + .count() + .selectExpr("CAST(count AS INT) AS id") // select only id column + .writeStream() + .outputMode("complete") + .format("iceberg") + .option("checkpointLocation", checkpoint.toString()) + .option("path", location.toString()); + + try { + // start the original query with checkpointing + StreamingQuery query = streamWriter.start(); + List batch1 = Lists.newArrayList(1, 2); + send(batch1, inputStream); + query.processAllAvailable(); + List batch2 = Lists.newArrayList(1, 2, 2, 3); + send(batch2, inputStream); + query.processAllAvailable(); + query.stop(); + + // remove the last commit to force Spark to reprocess batch #1 + File lastCommitFile = new File(checkpoint + "/commits/1"); + assertThat(lastCommitFile.delete()).as("The commit file must be deleted").isTrue(); + Files.deleteIfExists(Paths.get(checkpoint + "/commits/.1.crc")); + + // restart the query from the checkpoint + StreamingQuery restartedQuery = streamWriter.start(); + restartedQuery.processAllAvailable(); + + // ensure the write was idempotent + Dataset result = spark.read().format("iceberg").load(location.toString()); + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + assertThat(table.snapshots()).as("Number of snapshots should match").hasSize(2); + } finally { + for (StreamingQuery query : spark.streams().active()) { + query.stop(); + } + } + } + + @Test + public void testStreamingWriteUpdateMode() throws Exception { + File parent = temp.resolve("parquet").toFile(); + File location = new File(parent, "test-table"); + File checkpoint = new File(parent, "checkpoint"); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + tables.create(SCHEMA, spec, location.toString()); + + MemoryStream inputStream = newMemoryStream(1, spark.sqlContext(), Encoders.INT()); + DataStreamWriter streamWriter = + inputStream + .toDF() + .selectExpr("value AS id", "CAST (value AS STRING) AS data") + .writeStream() + .outputMode("update") + .format("iceberg") + .option("checkpointLocation", checkpoint.toString()) + .option("path", location.toString()); + + try { + StreamingQuery query = streamWriter.start(); + List batch1 = Lists.newArrayList(1, 2); + send(batch1, inputStream); + + assertThatThrownBy(query::processAllAvailable) + .isInstanceOf(StreamingQueryException.class) + .hasMessageContaining("does not support Update mode"); + } finally { + for (StreamingQuery query : spark.streams().active()) { + query.stop(); + } + } + } + + private MemoryStream newMemoryStream(int id, SQLContext sqlContext, Encoder encoder) { + return new MemoryStream<>(id, sqlContext, Option.empty(), encoder); + } + + private void send(List records, MemoryStream stream) { + stream.addData(JavaConverters.asScalaBuffer(records)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java new file mode 100644 index 000000000000..1a726c365ace --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -0,0 +1,942 @@ +/* + * 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.expressions.Expressions.ref; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.IntStream; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.spark.api.java.function.VoidFunction2; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.streaming.DataStreamWriter; +import org.apache.spark.sql.streaming.OutputMode; +import org.apache.spark.sql.streaming.StreamingQuery; +import org.apache.spark.sql.streaming.Trigger; +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; + +@ExtendWith(ParameterizedTestExtension.class) +public final class TestStructuredStreamingRead3 extends CatalogTestBase { + + private Table table; + + private final AtomicInteger microBatches = new AtomicInteger(); + + /** + * test data to be used by multiple writes each write creates a snapshot and writes a list of + * records + */ + private static final List> TEST_DATA_MULTIPLE_SNAPSHOTS = + Lists.newArrayList( + Lists.newArrayList( + new SimpleRecord(1, "one"), new SimpleRecord(2, "two"), new SimpleRecord(3, "three")), + Lists.newArrayList(new SimpleRecord(4, "four"), new SimpleRecord(5, "five")), + Lists.newArrayList(new SimpleRecord(6, "six"), new SimpleRecord(7, "seven"))); + + /** + * test data - to be used for multiple write batches each batch inturn will have multiple + * snapshots + */ + private static final List>> TEST_DATA_MULTIPLE_WRITES_MULTIPLE_SNAPSHOTS = + Lists.newArrayList( + Lists.newArrayList( + Lists.newArrayList( + new SimpleRecord(1, "one"), + new SimpleRecord(2, "two"), + new SimpleRecord(3, "three")), + Lists.newArrayList(new SimpleRecord(4, "four"), new SimpleRecord(5, "five"))), + Lists.newArrayList( + Lists.newArrayList(new SimpleRecord(6, "six"), new SimpleRecord(7, "seven")), + Lists.newArrayList(new SimpleRecord(8, "eight"), new SimpleRecord(9, "nine"))), + Lists.newArrayList( + Lists.newArrayList( + new SimpleRecord(10, "ten"), + new SimpleRecord(11, "eleven"), + new SimpleRecord(12, "twelve")), + Lists.newArrayList( + new SimpleRecord(13, "thirteen"), new SimpleRecord(14, "fourteen")), + Lists.newArrayList( + new SimpleRecord(15, "fifteen"), new SimpleRecord(16, "sixteen")))); + + @BeforeAll + public static void setupSpark() { + // disable AQE as tests assume that writes generate a particular number of files + spark.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), "false"); + } + + @BeforeEach + public void setupTable() { + sql( + "CREATE TABLE %s " + + "(id INT, data STRING) " + + "USING iceberg " + + "PARTITIONED BY (bucket(3, id)) " + + "TBLPROPERTIES ('commit.manifest.min-count-to-merge'='3', 'commit.manifest-merge.enabled'='true')", + tableName); + this.table = validationCatalog.loadTable(tableIdent); + microBatches.set(0); + } + + @AfterEach + public void stopStreams() throws TimeoutException { + for (StreamingQuery query : spark.streams().active()) { + query.stop(); + } + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testReadStreamOnIcebergTableWithMultipleSnapshots() throws Exception { + List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(expected); + + StreamingQuery query = startStream(); + + List actual = rowsAvailable(query); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + } + + @TestTemplate + public void testReadStreamWithMaxFiles1() throws Exception { + appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "1"), + List.of(1L, 2L, 1L, 1L, 1L, 1L)); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "1"), + List.of(1L, 2L, 1L, 1L, 1L, 1L), + Trigger.AvailableNow()); + } + + @TestTemplate + public void testReadStreamWithMaxFiles2() throws Exception { + appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "2"), + List.of(3L, 2L, 2L)); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "2"), + List.of(3L, 2L, 2L), + Trigger.AvailableNow()); + } + + @TestTemplate + public void testReadStreamWithMaxRows1() throws Exception { + appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "1"), + List.of(1L, 2L, 1L, 1L, 1L, 1L)); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "1"), + List.of(1L, 2L, 1L, 1L, 1L, 1L), + Trigger.AvailableNow()); + + // soft limit of 1 is being enforced, the stream is not blocked. + StreamingQuery query = + startStream(ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "1")); + + // check answer correctness only 1 record read the micro-batch will be stuck + List actual = rowsAvailable(query); + assertThat(actual) + .containsExactlyInAnyOrderElementsOf(Iterables.concat(TEST_DATA_MULTIPLE_SNAPSHOTS)); + } + + @TestTemplate + public void testReadStreamWithMaxRows2() throws Exception { + appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "2"), + List.of(3L, 2L, 2L)); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "2"), + List.of(3L, 2L, 2L), + Trigger.AvailableNow()); + + StreamingQuery query = + startStream(ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "2")); + + List actual = rowsAvailable(query); + assertThat(actual) + .containsExactlyInAnyOrderElementsOf(Iterables.concat(TEST_DATA_MULTIPLE_SNAPSHOTS)); + } + + @TestTemplate + public void testReadStreamWithMaxRows4() throws Exception { + appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "4"), List.of(4L, 3L)); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "4"), + List.of(4L, 3L), + Trigger.AvailableNow()); + } + + @TestTemplate + public void testReadStreamWithCompositeReadLimit() throws Exception { + appendDataAsMultipleSnapshots(TEST_DATA_MULTIPLE_SNAPSHOTS); + + assertMicroBatchRecordSizes( + ImmutableMap.of( + SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "1", + SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "2"), + List.of(1L, 2L, 1L, 1L, 1L, 1L)); + + assertMicroBatchRecordSizes( + ImmutableMap.of( + SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "1", + SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "2"), + List.of(1L, 2L, 1L, 1L, 1L, 1L), + Trigger.AvailableNow()); + } + + @TestTemplate + public void testAvailableNowStreamReadShouldNotHangOrReprocessData() throws Exception { + File writerCheckpointFolder = temp.resolve("writer-checkpoint-folder").toFile(); + File writerCheckpoint = new File(writerCheckpointFolder, "writer-checkpoint"); + File output = temp.resolve("junit").toFile(); + + DataStreamWriter querySource = + spark + .readStream() + .format("iceberg") + .load(tableName) + .writeStream() + .option("checkpointLocation", writerCheckpoint.toString()) + .format("parquet") + .trigger(Trigger.AvailableNow()) + .option("path", output.getPath()); + + List expected = Lists.newArrayList(); + for (List> expectedCheckpoint : + TEST_DATA_MULTIPLE_WRITES_MULTIPLE_SNAPSHOTS) { + + // New data was added while the stream was not running + appendDataAsMultipleSnapshots(expectedCheckpoint); + expected.addAll(Lists.newArrayList(Iterables.concat(Iterables.concat(expectedCheckpoint)))); + + try { + StreamingQuery query = querySource.start(); + + // Query should terminate on its own after processing all available data + assertThat(query.awaitTermination(60000)).isTrue(); + + // Check output + List actual = + spark + .read() + .load(output.getPath()) + .as(Encoders.bean(SimpleRecord.class)) + .collectAsList(); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + + // Restarting immediately should not reprocess data + query = querySource.start(); + assertThat(query.awaitTermination(60000)).isTrue(); + assertThat(query.recentProgress().length).isEqualTo(1); + assertThat(query.recentProgress()[0].sources()[0].startOffset()) + .isEqualTo(query.recentProgress()[0].sources()[0].endOffset()); + } finally { + stopStreams(); + } + } + } + + @TestTemplate + public void testTriggerAvailableNowDoesNotProcessNewDataWhileRunning() throws Exception { + List> expectedData = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(expectedData); + + long expectedRecordCount = expectedData.stream().mapToLong(List::size).sum(); + + table.refresh(); + long expectedSnapshotId = table.currentSnapshot().snapshotId(); + + String sinkTable = "availablenow_sink"; + StreamingQuery query = + spark + .readStream() + .option(SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "1") + .format("iceberg") + .load(tableName) + .writeStream() + .format("memory") + .queryName(sinkTable) + .trigger(Trigger.AvailableNow()) + .start(); + + assertThat(query.isActive()).isTrue(); + + // Add new data while the stream is running + List newDataDuringStreamSnap1 = + Lists.newArrayList( + new SimpleRecord(100, "hundred"), + new SimpleRecord(101, "hundred-one"), + new SimpleRecord(102, "hundred-two")); + List newDataDuringStreamSnap2 = + Lists.newArrayList( + new SimpleRecord(200, "two-hundred"), new SimpleRecord(201, "two-hundred-one")); + appendData(newDataDuringStreamSnap1); + appendData(newDataDuringStreamSnap2); + + // Query should terminate on its own after processing all available data till expectedSnapshotId + assertThat(query.awaitTermination(60000)).isTrue(); + + List actualResults = + spark + .sql("SELECT * FROM " + sinkTable) + .as(Encoders.bean(SimpleRecord.class)) + .collectAsList(); + long endOffsetSnapshotId = + StreamingOffset.fromJson(query.lastProgress().sources()[0].endOffset()).snapshotId(); + + // Verify the stream processed only up to the snapshot present when started + assertThat(expectedSnapshotId).isEqualTo(endOffsetSnapshotId); + + // Verify only the initial data was processed + assertThat(actualResults.size()).isEqualTo(expectedRecordCount); + assertThat(actualResults).containsExactlyInAnyOrderElementsOf(Iterables.concat(expectedData)); + } + + @TestTemplate + public void testReadStreamOnIcebergThenAddData() throws Exception { + List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; + + StreamingQuery query = startStream(); + + appendDataAsMultipleSnapshots(expected); + + List actual = rowsAvailable(query); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + } + + @TestTemplate + public void testReadingStreamFromTimestamp() throws Exception { + List dataBeforeTimestamp = + Lists.newArrayList( + new SimpleRecord(-2, "minustwo"), + new SimpleRecord(-1, "minusone"), + new SimpleRecord(0, "zero")); + + appendData(dataBeforeTimestamp); + + table.refresh(); + long streamStartTimestamp = table.currentSnapshot().timestampMillis() + 1; + + StreamingQuery query = + startStream(SparkReadOptions.STREAM_FROM_TIMESTAMP, Long.toString(streamStartTimestamp)); + + List empty = rowsAvailable(query); + assertThat(empty).isEmpty(); + + List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(expected); + + List actual = rowsAvailable(query); + + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + } + + @TestTemplate + public void testReadingStreamFromFutureTimetsamp() throws Exception { + long futureTimestamp = System.currentTimeMillis() + 10000; + + StreamingQuery query = + startStream(SparkReadOptions.STREAM_FROM_TIMESTAMP, Long.toString(futureTimestamp)); + + List actual = rowsAvailable(query); + assertThat(actual).isEmpty(); + + List data = + Lists.newArrayList( + new SimpleRecord(-2, "minustwo"), + new SimpleRecord(-1, "minusone"), + new SimpleRecord(0, "zero")); + + // Perform several inserts that should not show up because the fromTimestamp has not elapsed + IntStream.range(0, 3) + .forEach( + x -> { + appendData(data); + assertThat(rowsAvailable(query)).isEmpty(); + }); + + waitUntilAfter(futureTimestamp); + + // Data appended after the timestamp should appear + appendData(data); + actual = rowsAvailable(query); + assertThat(actual).containsExactlyInAnyOrderElementsOf(data); + } + + @TestTemplate + public void testReadingStreamFromTimestampFutureWithExistingSnapshots() throws Exception { + List dataBeforeTimestamp = + Lists.newArrayList( + new SimpleRecord(1, "one"), new SimpleRecord(2, "two"), new SimpleRecord(3, "three")); + appendData(dataBeforeTimestamp); + + long streamStartTimestamp = System.currentTimeMillis() + 2000; + + // Start the stream with a future timestamp after the current snapshot + StreamingQuery query = + startStream(SparkReadOptions.STREAM_FROM_TIMESTAMP, Long.toString(streamStartTimestamp)); + List actual = rowsAvailable(query); + assertThat(actual).isEmpty(); + + // Stream should contain data added after the timestamp elapses + waitUntilAfter(streamStartTimestamp); + List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(expected); + assertThat(rowsAvailable(query)) + .containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + } + + @TestTemplate + public void testReadingStreamFromTimestampOfExistingSnapshot() throws Exception { + List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; + + // Create an existing snapshot with some data + appendData(expected.get(0)); + table.refresh(); + long firstSnapshotTime = table.currentSnapshot().timestampMillis(); + + // Start stream giving the first Snapshot's time as the start point + StreamingQuery stream = + startStream(SparkReadOptions.STREAM_FROM_TIMESTAMP, Long.toString(firstSnapshotTime)); + + // Append rest of expected data + for (int i = 1; i < expected.size(); i++) { + appendData(expected.get(i)); + } + + List actual = rowsAvailable(stream); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + } + + @TestTemplate + public void testReadingStreamWithExpiredSnapshotFromTimestamp() throws TimeoutException { + List firstSnapshotRecordList = Lists.newArrayList(new SimpleRecord(1, "one")); + + List secondSnapshotRecordList = Lists.newArrayList(new SimpleRecord(2, "two")); + + List thirdSnapshotRecordList = Lists.newArrayList(new SimpleRecord(3, "three")); + + List expectedRecordList = Lists.newArrayList(); + expectedRecordList.addAll(secondSnapshotRecordList); + expectedRecordList.addAll(thirdSnapshotRecordList); + + appendData(firstSnapshotRecordList); + table.refresh(); + long firstSnapshotid = table.currentSnapshot().snapshotId(); + long firstSnapshotCommitTime = table.currentSnapshot().timestampMillis(); + + appendData(secondSnapshotRecordList); + appendData(thirdSnapshotRecordList); + + table.expireSnapshots().expireSnapshotId(firstSnapshotid).commit(); + + StreamingQuery query = + startStream( + SparkReadOptions.STREAM_FROM_TIMESTAMP, String.valueOf(firstSnapshotCommitTime)); + List actual = rowsAvailable(query); + assertThat(actual).containsExactlyInAnyOrderElementsOf(expectedRecordList); + } + + @TestTemplate + public void testResumingStreamReadFromCheckpoint() throws Exception { + File writerCheckpointFolder = temp.resolve("writer-checkpoint-folder").toFile(); + File writerCheckpoint = new File(writerCheckpointFolder, "writer-checkpoint"); + File output = temp.resolve("junit").toFile(); + + DataStreamWriter querySource = + spark + .readStream() + .format("iceberg") + .load(tableName) + .writeStream() + .option("checkpointLocation", writerCheckpoint.toString()) + .format("parquet") + .queryName("checkpoint_test") + .option("path", output.getPath()); + + StreamingQuery startQuery = querySource.start(); + startQuery.processAllAvailable(); + startQuery.stop(); + + List expected = Lists.newArrayList(); + for (List> expectedCheckpoint : + TEST_DATA_MULTIPLE_WRITES_MULTIPLE_SNAPSHOTS) { + // New data was added while the stream was down + appendDataAsMultipleSnapshots(expectedCheckpoint); + expected.addAll(Lists.newArrayList(Iterables.concat(Iterables.concat(expectedCheckpoint)))); + + // Stream starts up again from checkpoint read the newly added data and shut down + StreamingQuery restartedQuery = querySource.start(); + restartedQuery.processAllAvailable(); + restartedQuery.stop(); + + // Read data added by the stream + List actual = + spark.read().load(output.getPath()).as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + } + } + + @TestTemplate + public void testFailReadingCheckpointInvalidSnapshot() throws IOException, TimeoutException { + File writerCheckpointFolder = temp.resolve("writer-checkpoint-folder").toFile(); + File writerCheckpoint = new File(writerCheckpointFolder, "writer-checkpoint"); + File output = temp.resolve("junit").toFile(); + + DataStreamWriter querySource = + spark + .readStream() + .format("iceberg") + .load(tableName) + .writeStream() + .option("checkpointLocation", writerCheckpoint.toString()) + .format("parquet") + .queryName("checkpoint_test") + .option("path", output.getPath()); + + List firstSnapshotRecordList = Lists.newArrayList(new SimpleRecord(1, "one")); + List secondSnapshotRecordList = Lists.newArrayList(new SimpleRecord(2, "two")); + StreamingQuery startQuery = querySource.start(); + + appendData(firstSnapshotRecordList); + table.refresh(); + long firstSnapshotid = table.currentSnapshot().snapshotId(); + startQuery.processAllAvailable(); + startQuery.stop(); + + appendData(secondSnapshotRecordList); + + table.expireSnapshots().expireSnapshotId(firstSnapshotid).commit(); + + StreamingQuery restartedQuery = querySource.start(); + assertThatThrownBy(restartedQuery::processAllAvailable) + .hasCauseInstanceOf(IllegalStateException.class) + .hasMessageContaining( + String.format( + "Cannot load current offset at snapshot %d, the snapshot was expired or removed", + firstSnapshotid)); + } + + @TestTemplate + public void testParquetOrcAvroDataInOneTable() throws Exception { + List parquetFileRecords = + Lists.newArrayList( + new SimpleRecord(1, "one"), new SimpleRecord(2, "two"), new SimpleRecord(3, "three")); + + List orcFileRecords = + Lists.newArrayList(new SimpleRecord(4, "four"), new SimpleRecord(5, "five")); + + List avroFileRecords = + Lists.newArrayList(new SimpleRecord(6, "six"), new SimpleRecord(7, "seven")); + + appendData(parquetFileRecords); + appendData(orcFileRecords, "orc"); + appendData(avroFileRecords, "avro"); + + StreamingQuery query = startStream(); + assertThat(rowsAvailable(query)) + .containsExactlyInAnyOrderElementsOf( + Iterables.concat(parquetFileRecords, orcFileRecords, avroFileRecords)); + } + + @TestTemplate + public void testReadStreamFromEmptyTable() throws Exception { + StreamingQuery stream = startStream(); + List actual = rowsAvailable(stream); + assertThat(actual).isEmpty(); + } + + @TestTemplate + public void testReadStreamWithSnapshotTypeOverwriteErrorsOut() throws Exception { + // upgrade table to version 2 - to facilitate creation of Snapshot of type OVERWRITE. + TableOperations ops = ((BaseTable) table).operations(); + TableMetadata meta = ops.current(); + ops.commit(meta, meta.upgradeToFormatVersion(2)); + + // fill table with some initial data + List> dataAcrossSnapshots = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(dataAcrossSnapshots); + + Schema deleteRowSchema = table.schema().select("data"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + List dataDeletes = + Lists.newArrayList( + dataDelete.copy("data", "one") // id = 1 + ); + + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(File.createTempFile("junit", null, temp.toFile())), + TestHelpers.Row.of(0), + dataDeletes, + deleteRowSchema); + + DataFile dataFile = + DataFiles.builder(table.spec()) + .withPath(File.createTempFile("junit", null, temp.toFile()).getPath()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .withFormat(FileFormat.PARQUET) + .build(); + + table.newRowDelta().addRows(dataFile).addDeletes(eqDeletes).commit(); + + // check pre-condition - that the above Delete file write - actually resulted in snapshot of + // type OVERWRITE + assertThat(table.currentSnapshot().operation()).isEqualTo(DataOperations.OVERWRITE); + + StreamingQuery query = startStream(); + + assertThatThrownBy(query::processAllAvailable) + .cause() + .isInstanceOf(IllegalStateException.class) + .hasMessageStartingWith("Cannot process overwrite snapshot"); + } + + @TestTemplate + public void testReadStreamWithSnapshotTypeRewriteDataFilesIgnoresReplace() throws Exception { + // fill table with some data + List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(expected); + + makeRewriteDataFiles(); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "1"), + List.of(1L, 2L, 1L, 1L, 1L, 1L)); + } + + @TestTemplate + public void testReadStreamWithSnapshotTypeRewriteDataFilesIgnoresReplaceMaxRows() + throws Exception { + // fill table with some data + List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(expected); + + makeRewriteDataFiles(); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, "4"), List.of(4L, 3L)); + } + + @TestTemplate + public void testReadStreamWithSnapshotTypeRewriteDataFilesIgnoresReplaceMaxFilesAndRows() + throws Exception { + // fill table with some data + List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(expected); + + makeRewriteDataFiles(); + + assertMicroBatchRecordSizes( + ImmutableMap.of( + SparkReadOptions.STREAMING_MAX_ROWS_PER_MICRO_BATCH, + "4", + SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, + "1"), + List.of(1L, 2L, 1L, 1L, 1L, 1L)); + } + + @TestTemplate + public void testReadStreamWithSnapshotType2RewriteDataFilesIgnoresReplace() throws Exception { + // fill table with some data + List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(expected); + + makeRewriteDataFiles(); + makeRewriteDataFiles(); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "1"), + List.of(1L, 2L, 1L, 1L, 1L, 1L)); + } + + @TestTemplate + public void testReadStreamWithSnapshotTypeRewriteDataFilesIgnoresReplaceFollowedByAppend() + throws Exception { + // fill table with some data + List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(expected); + + makeRewriteDataFiles(); + + appendDataAsMultipleSnapshots(expected); + + assertMicroBatchRecordSizes( + ImmutableMap.of(SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "1"), + List.of(1L, 2L, 1L, 1L, 1L, 1L, 1L, 2L, 1L, 1L, 1L, 1L)); + } + + @TestTemplate + public void testReadStreamWithSnapshotTypeReplaceIgnoresReplace() throws Exception { + // fill table with some data + List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(expected); + + // this should create a snapshot with type Replace. + table.rewriteManifests().clusterBy(f -> 1).commit(); + + // check pre-condition + assertThat(table.currentSnapshot().operation()).isEqualTo(DataOperations.REPLACE); + + StreamingQuery query = startStream(); + List actual = rowsAvailable(query); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + } + + @TestTemplate + public void testReadStreamWithSnapshotTypeDeleteErrorsOut() throws Exception { + table.updateSpec().removeField("id_bucket").addField(ref("id")).commit(); + + // fill table with some data + List> dataAcrossSnapshots = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(dataAcrossSnapshots); + + // this should create a snapshot with type delete. + table.newDelete().deleteFromRowFilter(Expressions.equal("id", 4)).commit(); + + // check pre-condition - that the above delete operation on table resulted in Snapshot of Type + // DELETE. + assertThat(table.currentSnapshot().operation()).isEqualTo(DataOperations.DELETE); + + StreamingQuery query = startStream(); + + assertThatThrownBy(query::processAllAvailable) + .cause() + .isInstanceOf(IllegalStateException.class) + .hasMessageStartingWith("Cannot process delete snapshot"); + } + + @TestTemplate + public void testReadStreamWithSnapshotTypeDeleteAndSkipDeleteOption() throws Exception { + table.updateSpec().removeField("id_bucket").addField(ref("id")).commit(); + + // fill table with some data + List> dataAcrossSnapshots = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(dataAcrossSnapshots); + + // this should create a snapshot with type delete. + table.newDelete().deleteFromRowFilter(Expressions.equal("id", 4)).commit(); + + // check pre-condition - that the above delete operation on table resulted in Snapshot of Type + // DELETE. + assertThat(table.currentSnapshot().operation()).isEqualTo(DataOperations.DELETE); + + StreamingQuery query = startStream(SparkReadOptions.STREAMING_SKIP_DELETE_SNAPSHOTS, "true"); + assertThat(rowsAvailable(query)) + .containsExactlyInAnyOrderElementsOf(Iterables.concat(dataAcrossSnapshots)); + } + + @TestTemplate + public void testReadStreamWithSnapshotTypeDeleteAndSkipOverwriteOption() throws Exception { + table.updateSpec().removeField("id_bucket").addField(ref("id")).commit(); + + // fill table with some data + List> dataAcrossSnapshots = TEST_DATA_MULTIPLE_SNAPSHOTS; + appendDataAsMultipleSnapshots(dataAcrossSnapshots); + + DataFile dataFile = + DataFiles.builder(table.spec()) + .withPath(File.createTempFile("junit", null, temp.toFile()).getPath()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .withFormat(FileFormat.PARQUET) + .build(); + + // this should create a snapshot with type overwrite. + table + .newOverwrite() + .addFile(dataFile) + .overwriteByRowFilter(Expressions.greaterThan("id", 4)) + .commit(); + + // check pre-condition - that the above delete operation on table resulted in Snapshot of Type + // OVERWRITE. + assertThat(table.currentSnapshot().operation()).isEqualTo(DataOperations.OVERWRITE); + + StreamingQuery query = startStream(SparkReadOptions.STREAMING_SKIP_OVERWRITE_SNAPSHOTS, "true"); + assertThat(rowsAvailable(query)) + .containsExactlyInAnyOrderElementsOf(Iterables.concat(dataAcrossSnapshots)); + } + + /** + * We are testing that all the files in a rewrite snapshot are skipped Create a rewrite data files + * snapshot using existing files. + */ + public void makeRewriteDataFiles() { + table.refresh(); + + // we are testing that all the files in a rewrite snapshot are skipped + // create a rewrite data files snapshot using existing files + RewriteFiles rewrite = table.newRewrite(); + Iterable it = table.snapshots(); + for (Snapshot snapshot : it) { + if (snapshot.operation().equals(DataOperations.APPEND)) { + Iterable datafiles = snapshot.addedDataFiles(table.io()); + for (DataFile datafile : datafiles) { + rewrite.addFile(datafile); + rewrite.deleteFile(datafile); + } + } + } + rewrite.commit(); + } + + /** + * appends each list as a Snapshot on the iceberg table at the given location. accepts a list of + * lists - each list representing data per snapshot. + */ + private void appendDataAsMultipleSnapshots(List> data) { + for (List l : data) { + appendData(l); + } + } + + private void appendData(List data) { + appendData(data, "parquet"); + } + + private void appendData(List data, String format) { + Dataset df = spark.createDataFrame(data, SimpleRecord.class); + df.select("id", "data") + .write() + .format("iceberg") + .option("write-format", format) + .mode("append") + .save(tableName); + } + + private static final String MEMORY_TABLE = "_stream_view_mem"; + + private StreamingQuery startStream(Map options) throws TimeoutException { + return spark + .readStream() + .options(options) + .format("iceberg") + .load(tableName) + .writeStream() + .options(options) + .format("memory") + .queryName(MEMORY_TABLE) + .outputMode(OutputMode.Append()) + .start(); + } + + private StreamingQuery startStream() throws TimeoutException { + return startStream(Collections.emptyMap()); + } + + private StreamingQuery startStream(String key, String value) throws TimeoutException { + return startStream( + ImmutableMap.of(key, value, SparkReadOptions.STREAMING_MAX_FILES_PER_MICRO_BATCH, "1")); + } + + private void assertMicroBatchRecordSizes( + Map options, List expectedMicroBatchRecordSize) + throws TimeoutException { + assertMicroBatchRecordSizes(options, expectedMicroBatchRecordSize, Trigger.ProcessingTime(0L)); + } + + private void assertMicroBatchRecordSizes( + Map options, List expectedMicroBatchRecordSize, Trigger trigger) + throws TimeoutException { + Dataset ds = spark.readStream().options(options).format("iceberg").load(tableName); + + List syncList = Collections.synchronizedList(Lists.newArrayList()); + ds.writeStream() + .options(options) + .trigger(trigger) + .foreachBatch( + (VoidFunction2, Long>) + (dataset, batchId) -> { + syncList.add(dataset.count()); + }) + .start() + .processAllAvailable(); + + stopStreams(); + assertThat(syncList).containsExactlyInAnyOrderElementsOf(expectedMicroBatchRecordSize); + } + + private List rowsAvailable(StreamingQuery query) { + query.processAllAvailable(); + return spark + .sql("select * from " + MEMORY_TABLE) + .as(Encoders.bean(SimpleRecord.class)) + .collectAsList(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java new file mode 100644 index 000000000000..b54bb315c543 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTables.java @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source; + +import java.io.File; +import java.util.Map; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.Files; +import org.apache.iceberg.LocationProviders; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.io.OutputFile; +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.Maps; + +// TODO: Use the copy of this from core. +class TestTables { + private TestTables() {} + + static TestTable create(File temp, String name, Schema schema, PartitionSpec spec) { + return create(temp, name, schema, spec, ImmutableMap.of()); + } + + static TestTable create( + File temp, String name, Schema schema, PartitionSpec spec, Map properties) { + TestTableOperations ops = new TestTableOperations(name); + if (ops.current() != null) { + throw new AlreadyExistsException("Table %s already exists at location: %s", name, temp); + } + ops.commit(null, TableMetadata.newTableMetadata(schema, spec, temp.toString(), properties)); + return new TestTable(ops, name); + } + + static TestTable load(String name) { + TestTableOperations ops = new TestTableOperations(name); + if (ops.current() == null) { + return null; + } + return new TestTable(ops, name); + } + + static boolean drop(String name) { + synchronized (METADATA) { + return METADATA.remove(name) != null; + } + } + + static class TestTable extends BaseTable { + private final TestTableOperations ops; + + private TestTable(TestTableOperations ops, String name) { + super(ops, name); + this.ops = ops; + } + + @Override + public TestTableOperations operations() { + return ops; + } + } + + private static final Map METADATA = Maps.newHashMap(); + + static void clearTables() { + synchronized (METADATA) { + METADATA.clear(); + } + } + + static TableMetadata readMetadata(String tableName) { + synchronized (METADATA) { + return METADATA.get(tableName); + } + } + + static void replaceMetadata(String tableName, TableMetadata metadata) { + synchronized (METADATA) { + METADATA.put(tableName, metadata); + } + } + + static class TestTableOperations implements TableOperations { + + private final String tableName; + private TableMetadata current = null; + private long lastSnapshotId = 0; + private int failCommits = 0; + + TestTableOperations(String tableName) { + this.tableName = tableName; + refresh(); + if (current != null) { + for (Snapshot snap : current.snapshots()) { + this.lastSnapshotId = Math.max(lastSnapshotId, snap.snapshotId()); + } + } else { + this.lastSnapshotId = 0; + } + } + + void failCommits(int numFailures) { + this.failCommits = numFailures; + } + + @Override + public TableMetadata current() { + return current; + } + + @Override + public TableMetadata refresh() { + synchronized (METADATA) { + this.current = METADATA.get(tableName); + } + return current; + } + + @Override + public void commit(TableMetadata base, TableMetadata metadata) { + if (base != current) { + throw new CommitFailedException("Cannot commit changes based on stale metadata"); + } + synchronized (METADATA) { + refresh(); + if (base == current) { + if (failCommits > 0) { + this.failCommits -= 1; + throw new CommitFailedException("Injected failure"); + } + METADATA.put(tableName, metadata); + this.current = metadata; + } else { + throw new CommitFailedException( + "Commit failed: table was updated at %d", base.lastUpdatedMillis()); + } + } + } + + @Override + public FileIO io() { + return new LocalFileIO(); + } + + @Override + public LocationProvider locationProvider() { + Preconditions.checkNotNull( + current, "Current metadata should not be null when locationProvider is called"); + return LocationProviders.locationsFor(current.location(), current.properties()); + } + + @Override + public String metadataFileLocation(String fileName) { + return new File(new File(current.location(), "metadata"), fileName).getAbsolutePath(); + } + + @Override + public long newSnapshotId() { + long nextSnapshotId = lastSnapshotId + 1; + this.lastSnapshotId = nextSnapshotId; + return nextSnapshotId; + } + } + + static class LocalFileIO implements FileIO { + + @Override + public InputFile newInputFile(String path) { + return Files.localInput(path); + } + + @Override + public OutputFile newOutputFile(String path) { + return Files.localOutput(new File(path)); + } + + @Override + public void deleteFile(String path) { + if (!new File(path).delete()) { + throw new RuntimeIOException("Failed to delete file: " + path); + } + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java new file mode 100644 index 000000000000..79781a8fc314 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java @@ -0,0 +1,229 @@ +/* + * 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.Files.localOutput; +import static org.apache.iceberg.data.FileHelpers.encrypt; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.time.LocalDateTime; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericFileWriterFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.TestBase; +import org.apache.iceberg.spark.data.GenericsHelpers; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.AfterAll; +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; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestTimestampWithoutZone extends TestBase { + private static final Configuration CONF = new Configuration(); + private static final HadoopTables TABLES = new HadoopTables(CONF); + + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(3, "data", Types.StringType.get())); + + private static SparkSession spark = null; + + @BeforeAll + public static void startSpark() { + TestTimestampWithoutZone.spark = SparkSession.builder().master("local[2]").getOrCreate(); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestTimestampWithoutZone.spark; + TestTimestampWithoutZone.spark = null; + currentSpark.stop(); + } + + @TempDir private Path temp; + + @Parameter(index = 0) + private FileFormat fileFormat; + + @Parameter(index = 1) + private boolean vectorized; + + @Parameters(name = "format = {0}, vectorized = {1}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.PARQUET, false}, + {FileFormat.PARQUET, true}, + {FileFormat.AVRO, false} + }; + } + + private File parent = null; + private File unpartitioned = null; + private List records = null; + + @BeforeEach + public void writeUnpartitionedTable() throws IOException { + this.parent = temp.resolve("TestTimestampWithoutZone").toFile(); + this.unpartitioned = new File(parent, "unpartitioned"); + File dataFolder = new File(unpartitioned, "data"); + assertThat(dataFolder.mkdirs()).as("Mkdir should succeed").isTrue(); + + Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), unpartitioned.toString()); + Schema tableSchema = table.schema(); // use the table schema because ids are reassigned + + File testFile = new File(dataFolder, fileFormat.addExtension(UUID.randomUUID().toString())); + + // create records using the table's schema + this.records = testRecords(tableSchema); + + DataWriter writer = + new GenericFileWriterFactory.Builder(table) + .dataFileFormat(fileFormat) + .build() + .newDataWriter(encrypt(localOutput(testFile)), PartitionSpec.unpartitioned(), null); + try (writer) { + writer.write(records); + } + + DataFile file = writer.toDataFile(); + + table.newAppend().appendFile(file).commit(); + } + + @TestTemplate + public void testUnpartitionedTimestampWithoutZone() { + assertEqualsSafe(SCHEMA.asStruct(), records, read(unpartitioned.toString(), vectorized)); + } + + @TestTemplate + public void testUnpartitionedTimestampWithoutZoneProjection() { + Schema projection = SCHEMA.select("id", "ts"); + assertEqualsSafe( + projection.asStruct(), + records.stream().map(r -> projectFlat(projection, r)).collect(Collectors.toList()), + read(unpartitioned.toString(), vectorized, "id", "ts")); + } + + @TestTemplate + public void testUnpartitionedTimestampWithoutZoneAppend() { + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(unpartitioned.toString()) + .write() + .format("iceberg") + .mode(SaveMode.Append) + .save(unpartitioned.toString()); + + assertEqualsSafe( + SCHEMA.asStruct(), + Stream.concat(records.stream(), records.stream()).collect(Collectors.toList()), + read(unpartitioned.toString(), vectorized)); + } + + private static Record projectFlat(Schema projection, Record record) { + Record result = GenericRecord.create(projection); + List fields = projection.asStruct().fields(); + for (int i = 0; i < fields.size(); i += 1) { + Types.NestedField field = fields.get(i); + result.set(i, record.getField(field.name())); + } + return result; + } + + public static void assertEqualsSafe( + Types.StructType struct, List expected, List actual) { + assertThat(actual).as("Number of results should match expected").hasSameSizeAs(expected); + for (int i = 0; i < expected.size(); i += 1) { + GenericsHelpers.assertEqualsSafe(struct, expected.get(i), actual.get(i)); + } + } + + private List testRecords(Schema schema) { + return Lists.newArrayList( + record(schema, 0L, parseToLocal("2017-12-22T09:20:44.294658"), "junction"), + record(schema, 1L, parseToLocal("2017-12-22T07:15:34.582910"), "alligator"), + record(schema, 2L, parseToLocal("2017-12-22T06:02:09.243857"), "forrest"), + record(schema, 3L, parseToLocal("2017-12-22T03:10:11.134509"), "clapping"), + record(schema, 4L, parseToLocal("2017-12-22T00:34:00.184671"), "brush"), + record(schema, 5L, parseToLocal("2017-12-21T22:20:08.935889"), "trap"), + record(schema, 6L, parseToLocal("2017-12-21T21:55:30.589712"), "element"), + record(schema, 7L, parseToLocal("2017-12-21T17:31:14.532797"), "limited"), + record(schema, 8L, parseToLocal("2017-12-21T15:21:51.237521"), "global"), + record(schema, 9L, parseToLocal("2017-12-21T15:02:15.230570"), "goldfish")); + } + + private static List read(String table, boolean vectorized) { + return read(table, vectorized, "*"); + } + + private static List read( + String table, boolean vectorized, String select0, String... selectN) { + Dataset dataset = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .load(table) + .select(select0, selectN); + return dataset.collectAsList(); + } + + private static LocalDateTime parseToLocal(String timestamp) { + return LocalDateTime.parse(timestamp); + } + + private static Record record(Schema schema, Object... values) { + Record rec = GenericRecord.create(schema); + for (int i = 0; i < values.length; i += 1) { + rec.set(i, values[i]); + } + return rec; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java new file mode 100644 index 000000000000..e2b5d8920e9f --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java @@ -0,0 +1,304 @@ +/* + * 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.spark.SparkSchemaUtil.convert; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.net.InetAddress; +import java.nio.ByteBuffer; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.hadoop.HadoopTables; +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.Maps; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.InternalRow; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestWriteMetricsConfig { + + private static final Configuration CONF = new Configuration(); + private static final Schema SIMPLE_SCHEMA = + new Schema( + optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); + private static final Schema COMPLEX_SCHEMA = + new Schema( + required(1, "longCol", Types.IntegerType.get()), + optional(2, "strCol", Types.StringType.get()), + required( + 3, + "record", + Types.StructType.of( + required(4, "id", Types.IntegerType.get()), + required(5, "data", Types.StringType.get())))); + + @TempDir private Path temp; + + private static SparkSession spark = null; + private static JavaSparkContext sc = null; + + @BeforeAll + public static void startSpark() { + TestWriteMetricsConfig.spark = + SparkSession.builder() + .master("local[2]") + .config("spark.driver.host", InetAddress.getLoopbackAddress().getHostAddress()) + .getOrCreate(); + TestWriteMetricsConfig.sc = JavaSparkContext.fromSparkContext(spark.sparkContext()); + } + + @AfterAll + public static void stopSpark() { + SparkSession currentSpark = TestWriteMetricsConfig.spark; + TestWriteMetricsConfig.spark = null; + TestWriteMetricsConfig.sc = null; + currentSpark.stop(); + } + + @Test + public void testFullMetricsCollectionForParquet() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map properties = Maps.newHashMap(); + properties.put(TableProperties.DEFAULT_WRITE_METRICS_MODE, "full"); + Table table = tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation); + + List expectedRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset df = spark.createDataFrame(expectedRecords, SimpleRecord.class); + df.select("id", "data") + .coalesce(1) + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, "parquet") + .mode(SaveMode.Append) + .save(tableLocation); + + for (FileScanTask task : table.newScan().includeColumnStats().planFiles()) { + DataFile file = task.file(); + assertThat(file.nullValueCounts()).hasSize(2); + assertThat(file.valueCounts()).hasSize(2); + assertThat(file.lowerBounds()).hasSize(2); + assertThat(file.upperBounds()).hasSize(2); + } + } + + @Test + public void testCountMetricsCollectionForParquet() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map properties = Maps.newHashMap(); + properties.put(TableProperties.DEFAULT_WRITE_METRICS_MODE, "counts"); + Table table = tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation); + + List expectedRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset df = spark.createDataFrame(expectedRecords, SimpleRecord.class); + df.select("id", "data") + .coalesce(1) + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, "parquet") + .mode(SaveMode.Append) + .save(tableLocation); + + for (FileScanTask task : table.newScan().includeColumnStats().planFiles()) { + DataFile file = task.file(); + assertThat(file.nullValueCounts()).hasSize(2); + assertThat(file.valueCounts()).hasSize(2); + assertThat(file.lowerBounds()).isEmpty(); + assertThat(file.upperBounds()).isEmpty(); + } + } + + @Test + public void testNoMetricsCollectionForParquet() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map properties = Maps.newHashMap(); + properties.put(TableProperties.DEFAULT_WRITE_METRICS_MODE, "none"); + Table table = tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation); + + List expectedRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset df = spark.createDataFrame(expectedRecords, SimpleRecord.class); + df.select("id", "data") + .coalesce(1) + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, "parquet") + .mode(SaveMode.Append) + .save(tableLocation); + + for (FileScanTask task : table.newScan().includeColumnStats().planFiles()) { + DataFile file = task.file(); + assertThat(file.nullValueCounts()).isEmpty(); + assertThat(file.valueCounts()).isEmpty(); + assertThat(file.lowerBounds()).isEmpty(); + assertThat(file.upperBounds()).isEmpty(); + } + } + + @Test + public void testCustomMetricCollectionForParquet() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map properties = Maps.newHashMap(); + properties.put(TableProperties.DEFAULT_WRITE_METRICS_MODE, "counts"); + properties.put("write.metadata.metrics.column.id", "full"); + Table table = tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation); + + List expectedRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset df = spark.createDataFrame(expectedRecords, SimpleRecord.class); + df.select("id", "data") + .coalesce(1) + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, "parquet") + .mode(SaveMode.Append) + .save(tableLocation); + + Schema schema = table.schema(); + Types.NestedField id = schema.findField("id"); + for (FileScanTask task : table.newScan().includeColumnStats().planFiles()) { + DataFile file = task.file(); + assertThat(file.nullValueCounts()).hasSize(2); + assertThat(file.valueCounts()).hasSize(2); + assertThat(file.lowerBounds()).hasSize(1).containsKey(id.fieldId()); + assertThat(file.upperBounds()).hasSize(1).containsKey(id.fieldId()); + } + } + + @Test + public void testBadCustomMetricCollectionForParquet() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map properties = Maps.newHashMap(); + properties.put(TableProperties.DEFAULT_WRITE_METRICS_MODE, "counts"); + properties.put("write.metadata.metrics.column.ids", "full"); + + assertThatThrownBy(() -> tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Invalid metrics config, could not find column ids from table prop write.metadata.metrics.column.ids in schema table"); + } + + @Test + public void testCustomMetricCollectionForNestedParquet() { + String tableLocation = temp.resolve("iceberg-table").toFile().toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.builderFor(COMPLEX_SCHEMA).identity("strCol").build(); + Map properties = Maps.newHashMap(); + properties.put(TableProperties.DEFAULT_WRITE_METRICS_MODE, "none"); + properties.put("write.metadata.metrics.column.longCol", "counts"); + properties.put("write.metadata.metrics.column.record.id", "full"); + properties.put("write.metadata.metrics.column.record.data", "truncate(2)"); + Table table = tables.create(COMPLEX_SCHEMA, spec, properties, tableLocation); + + Iterable rows = RandomData.generateSpark(COMPLEX_SCHEMA, 10, 0); + JavaRDD rdd = sc.parallelize(Lists.newArrayList(rows)); + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + + Dataset df = + ((org.apache.spark.sql.classic.SparkSession) spark) + .internalCreateDataFrame(JavaRDD.toRDD(rdd), convert(COMPLEX_SCHEMA), false); + + df.coalesce(1) + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, "parquet") + .mode(SaveMode.Append) + .save(tableLocation); + + Schema schema = table.schema(); + Types.NestedField longCol = schema.findField("longCol"); + Types.NestedField recordId = schema.findField("record.id"); + Types.NestedField recordData = schema.findField("record.data"); + for (FileScanTask task : table.newScan().includeColumnStats().planFiles()) { + DataFile file = task.file(); + + Map nullValueCounts = file.nullValueCounts(); + assertThat(nullValueCounts) + .hasSize(3) + .containsKeys(longCol.fieldId(), recordId.fieldId(), recordData.fieldId()); + + Map valueCounts = file.valueCounts(); + assertThat(valueCounts) + .hasSize(3) + .containsKeys(longCol.fieldId(), recordId.fieldId(), recordData.fieldId()); + + Map lowerBounds = file.lowerBounds(); + assertThat(lowerBounds).hasSize(2).containsKey(recordId.fieldId()); + + ByteBuffer recordDataLowerBound = lowerBounds.get(recordData.fieldId()); + assertThat(ByteBuffers.toByteArray(recordDataLowerBound)).hasSize(2); + + Map upperBounds = file.upperBounds(); + assertThat(upperBounds).hasSize(2).containsKey(recordId.fieldId()); + + ByteBuffer recordDataUpperBound = upperBounds.get(recordData.fieldId()); + assertThat(ByteBuffers.toByteArray(recordDataUpperBound)).hasSize(2); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ThreeColumnRecord.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ThreeColumnRecord.java new file mode 100644 index 000000000000..554557df416c --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/ThreeColumnRecord.java @@ -0,0 +1,83 @@ +/* + * 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.Objects; + +public class ThreeColumnRecord { + private Integer c1; + private String c2; + private String c3; + + public ThreeColumnRecord() {} + + public ThreeColumnRecord(Integer c1, String c2, String c3) { + this.c1 = c1; + this.c2 = c2; + this.c3 = c3; + } + + public Integer getC1() { + return c1; + } + + public void setC1(Integer c1) { + this.c1 = c1; + } + + public String getC2() { + return c2; + } + + public void setC2(String c2) { + this.c2 = c2; + } + + public String getC3() { + return c3; + } + + public void setC3(String c3) { + this.c3 = c3; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ThreeColumnRecord that = (ThreeColumnRecord) o; + return Objects.equals(c1, that.c1) + && Objects.equals(c2, that.c2) + && Objects.equals(c3, that.c3); + } + + @Override + public int hashCode() { + return Objects.hash(c1, c2, c3); + } + + @Override + public String toString() { + return "ThreeColumnRecord{" + "c1=" + c1 + ", c2='" + c2 + '\'' + ", c3='" + c3 + '\'' + '}'; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java new file mode 100644 index 000000000000..c2fa8dc0ce24 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/PartitionedWritesTestBase.java @@ -0,0 +1,282 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.functions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class PartitionedWritesTestBase extends CatalogTestBase { + + @BeforeEach + public void createTables() { + sql( + "CREATE TABLE %s (id bigint, data string) USING iceberg PARTITIONED BY (truncate(id, 3))", + tableName); + sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testInsertAppend() { + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Rows before insert") + .isEqualTo(3L); + + sql("INSERT INTO %s VALUES (4, 'd'), (5, 'e')", commitTarget()); + + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 5 rows after insert") + .isEqualTo(5L); + + List expected = + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d"), row(5L, "e")); + + assertEquals( + "Row data should match expected", + expected, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testInsertOverwrite() { + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Rows before overwrite") + .isEqualTo(3L); + + // 4 and 5 replace 3 in the partition (id - (id % 3)) = 3 + sql("INSERT OVERWRITE %s VALUES (4, 'd'), (5, 'e')", commitTarget()); + + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 4 rows after overwrite") + .isEqualTo(4L); + + List expected = + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(4L, "d"), row(5L, "e")); + + assertEquals( + "Row data should match expected", + expected, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testDataFrameV2Append() throws NoSuchTableException { + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); + + List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); + Dataset ds = spark.createDataFrame(data, SimpleRecord.class); + + ds.writeTo(commitTarget()).append(); + + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 5 rows after insert") + .isEqualTo(5L); + + List expected = + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d"), row(5L, "e")); + + assertEquals( + "Row data should match expected", + expected, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testDataFrameV2DynamicOverwrite() throws NoSuchTableException { + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); + + List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); + Dataset ds = spark.createDataFrame(data, SimpleRecord.class); + + ds.writeTo(commitTarget()).overwritePartitions(); + + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 4 rows after overwrite") + .isEqualTo(4L); + + List expected = + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(4L, "d"), row(5L, "e")); + + assertEquals( + "Row data should match expected", + expected, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testDataFrameV2Overwrite() throws NoSuchTableException { + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); + + List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); + Dataset ds = spark.createDataFrame(data, SimpleRecord.class); + + ds.writeTo(commitTarget()).overwrite(functions.col("id").$less(3)); + + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows after overwrite") + .isEqualTo(3L); + + List expected = ImmutableList.of(row(3L, "c"), row(4L, "d"), row(5L, "e")); + + assertEquals( + "Row data should match expected", + expected, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testViewsReturnRecentResults() { + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); + + Dataset query = spark.sql("SELECT * FROM " + commitTarget() + " WHERE id = 1"); + query.createOrReplaceTempView("tmp"); + + assertEquals( + "View should have expected rows", ImmutableList.of(row(1L, "a")), sql("SELECT * FROM tmp")); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", commitTarget()); + + assertEquals( + "View should have expected rows", + ImmutableList.of(row(1L, "a"), row(1L, "a")), + sql("SELECT * FROM tmp")); + } + + // Asserts whether the given table .partitions table has the expected rows. Note that the output + // row should have spec_id and it is sorted by spec_id and selectPartitionColumns. + protected void assertPartitionMetadata( + String tableName, List expected, String... selectPartitionColumns) { + String[] fullyQualifiedCols = + Arrays.stream(selectPartitionColumns).map(s -> "partition." + s).toArray(String[]::new); + Dataset actualPartitionRows = + spark + .read() + .format("iceberg") + .load(tableName + ".partitions") + .select("spec_id", fullyQualifiedCols) + .orderBy("spec_id", fullyQualifiedCols); + + assertEquals( + "There are 3 partitions, one with the original spec ID and two with the new one", + expected, + rowsToJava(actualPartitionRows.collectAsList())); + } + + @TestTemplate + public void testWriteWithOutputSpec() throws NoSuchTableException { + Table table = validationCatalog.loadTable(tableIdent); + + // Drop all records in table to have a fresh start. + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + + final int originalSpecId = table.spec().specId(); + table.updateSpec().addField("data").commit(); + + // Refresh this when using SparkCatalog since otherwise the new spec would not be caught. + sql("REFRESH TABLE %s", tableName); + + // By default, we write to the current spec. + List data = ImmutableList.of(new SimpleRecord(10, "a")); + spark.createDataFrame(data, SimpleRecord.class).toDF().writeTo(tableName).append(); + + List expected = ImmutableList.of(row(10L, "a", table.spec().specId())); + assertEquals( + "Rows must match", + expected, + sql("SELECT id, data, _spec_id FROM %s WHERE id >= 10 ORDER BY id", tableName)); + + // Output spec ID should be respected when present. + data = ImmutableList.of(new SimpleRecord(11, "b"), new SimpleRecord(12, "c")); + spark + .createDataFrame(data, SimpleRecord.class) + .toDF() + .writeTo(tableName) + .option("output-spec-id", Integer.toString(originalSpecId)) + .append(); + + expected = + ImmutableList.of( + row(10L, "a", table.spec().specId()), + row(11L, "b", originalSpecId), + row(12L, "c", originalSpecId)); + assertEquals( + "Rows must match", + expected, + sql("SELECT id, data, _spec_id FROM %s WHERE id >= 10 ORDER BY id", tableName)); + + // Verify that the actual partitions are written with the correct spec ID. + // Two of the partitions should have the original spec ID and one should have the new one. + // TODO: WAP branch does not support reading partitions table, skip this check for now. + expected = + ImmutableList.of( + row(originalSpecId, 9L, null), + row(originalSpecId, 12L, null), + row(table.spec().specId(), 9L, "a")); + assertPartitionMetadata(tableName, expected, "id_trunc", "data"); + + // Even the default spec ID should be followed when present. + data = ImmutableList.of(new SimpleRecord(13, "d")); + spark + .createDataFrame(data, SimpleRecord.class) + .toDF() + .writeTo(tableName) + .option("output-spec-id", Integer.toString(table.spec().specId())) + .append(); + + expected = + ImmutableList.of( + row(10L, "a", table.spec().specId()), + row(11L, "b", originalSpecId), + row(12L, "c", originalSpecId), + row(13L, "d", table.spec().specId())); + assertEquals( + "Rows must match", + expected, + sql("SELECT id, data, _spec_id FROM %s WHERE id >= 10 ORDER BY id", tableName)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java new file mode 100644 index 000000000000..ce0a0f26a096 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java @@ -0,0 +1,866 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.TestBase; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.execution.ExplainMode; +import org.apache.spark.sql.functions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestAggregatePushDown extends CatalogTestBase { + + @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.sql.iceberg.aggregate_pushdown", "true") + .enableHiveSupport() + .getOrCreate(); + + TestBase.catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + + try { + catalog.createNamespace(Namespace.of("default")); + } catch (AlreadyExistsException ignored) { + // the default namespace already exists. ignore the create error + } + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testDifferentDataTypesAggregatePushDownInPartitionedTable() { + testDifferentDataTypesAggregatePushDown(true); + } + + @TestTemplate + public void testDifferentDataTypesAggregatePushDownInNonPartitionedTable() { + testDifferentDataTypesAggregatePushDown(false); + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + private void testDifferentDataTypesAggregatePushDown(boolean hasPartitionCol) { + String createTable; + if (hasPartitionCol) { + createTable = + "CREATE TABLE %s (id LONG, int_data INT, boolean_data BOOLEAN, float_data FLOAT, double_data DOUBLE, " + + "decimal_data DECIMAL(14, 2), binary_data binary) USING iceberg PARTITIONED BY (id)"; + } else { + createTable = + "CREATE TABLE %s (id LONG, int_data INT, boolean_data BOOLEAN, float_data FLOAT, double_data DOUBLE, " + + "decimal_data DECIMAL(14, 2), binary_data binary) USING iceberg"; + } + + sql(createTable, tableName); + sql( + "INSERT INTO TABLE %s VALUES " + + "(1, null, false, null, null, 11.11, X'1111')," + + " (1, null, true, 2.222, 2.222222, 22.22, X'2222')," + + " (2, 33, false, 3.333, 3.333333, 33.33, X'3333')," + + " (2, 44, true, null, 4.444444, 44.44, X'4444')," + + " (3, 55, false, 5.555, 5.555555, 55.55, X'5555')," + + " (3, null, true, null, 6.666666, 66.66, null) ", + tableName); + + String select = + "SELECT count(*), max(id), min(id), count(id), " + + "max(int_data), min(int_data), count(int_data), " + + "max(boolean_data), min(boolean_data), count(boolean_data), " + + "max(float_data), min(float_data), count(float_data), " + + "max(double_data), min(double_data), count(double_data), " + + "max(decimal_data), min(decimal_data), count(decimal_data), " + + "max(binary_data), min(binary_data), count(binary_data) FROM %s"; + + List explain = sql("EXPLAIN " + select, tableName); + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates = false; + if (explainString.contains("count(*)") + && explainString.contains("max(id)") + && explainString.contains("min(id)") + && explainString.contains("count(id)") + && explainString.contains("max(int_data)") + && explainString.contains("min(int_data)") + && explainString.contains("count(int_data)") + && explainString.contains("max(boolean_data)") + && explainString.contains("min(boolean_data)") + && explainString.contains("count(boolean_data)") + && explainString.contains("max(float_data)") + && explainString.contains("min(float_data)") + && explainString.contains("count(float_data)") + && explainString.contains("max(double_data)") + && explainString.contains("min(double_data)") + && explainString.contains("count(double_data)") + && explainString.contains("max(decimal_data)") + && explainString.contains("min(decimal_data)") + && explainString.contains("count(decimal_data)") + && explainString.contains("max(binary_data)") + && explainString.contains("min(binary_data)") + && explainString.contains("count(binary_data)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); + + List actual = sql(select, tableName); + List expected = Lists.newArrayList(); + expected.add( + new Object[] { + 6L, + 3L, + 1L, + 6L, + 55, + 33, + 3L, + true, + false, + 6L, + 5.555f, + 2.222f, + 3L, + 6.666666, + 2.222222, + 5L, + new BigDecimal("66.66"), + new BigDecimal("11.11"), + 6L, + new byte[] {85, 85}, + new byte[] {17, 17}, + 5L + }); + assertEquals("min/max/count push down", expected, actual); + } + + @TestTemplate + public void testDateAndTimestampWithPartition() { + sql( + "CREATE TABLE %s (id bigint, data string, d date, ts timestamp) USING iceberg PARTITIONED BY (id)", + tableName); + sql( + "INSERT INTO %s VALUES (1, '1', date('2021-11-10'), null)," + + "(1, '2', date('2021-11-11'), timestamp('2021-11-11 22:22:22')), " + + "(2, '3', date('2021-11-12'), timestamp('2021-11-12 22:22:22')), " + + "(2, '4', date('2021-11-13'), timestamp('2021-11-13 22:22:22')), " + + "(3, '5', null, timestamp('2021-11-14 22:22:22')), " + + "(3, '6', date('2021-11-14'), null)", + tableName); + String select = "SELECT max(d), min(d), count(d), max(ts), min(ts), count(ts) FROM %s"; + + List explain = sql("EXPLAIN " + select, tableName); + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates = false; + if (explainString.contains("max(d)") + && explainString.contains("min(d)") + && explainString.contains("count(d)") + && explainString.contains("max(ts)") + && explainString.contains("min(ts)") + && explainString.contains("count(ts)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); + + List actual = sql(select, tableName); + List expected = Lists.newArrayList(); + expected.add( + new Object[] { + Date.valueOf("2021-11-14"), + Date.valueOf("2021-11-10"), + 5L, + Timestamp.valueOf("2021-11-14 22:22:22.0"), + Timestamp.valueOf("2021-11-11 22:22:22.0"), + 4L + }); + assertEquals("min/max/count push down", expected, actual); + } + + @TestTemplate + public void testAggregateNotPushDownIfOneCantPushDown() { + sql("CREATE TABLE %s (id LONG, data DOUBLE) USING iceberg", tableName); + sql( + "INSERT INTO TABLE %s VALUES (1, 1111), (1, 2222), (2, 3333), (2, 4444), (3, 5555), (3, 6666) ", + tableName); + String select = "SELECT COUNT(data), SUM(data) FROM %s"; + + List explain = sql("EXPLAIN " + select, tableName); + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates = false; + if (explainString.contains("count(data)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); + + List actual = sql(select, tableName); + List expected = Lists.newArrayList(); + expected.add(new Object[] {6L, 23331.0}); + assertEquals("expected and actual should equal", expected, actual); + } + + @TestTemplate + public void testAggregatePushDownWithMetricsMode() { + sql("CREATE TABLE %s (id LONG, data DOUBLE) USING iceberg", tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + tableName, TableProperties.DEFAULT_WRITE_METRICS_MODE, "none"); + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + tableName, TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "id", "counts"); + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + tableName, TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "data", "none"); + sql( + "INSERT INTO TABLE %s VALUES (1, 1111), (1, 2222), (2, 3333), (2, 4444), (3, 5555), (3, 6666)", + tableName); + + String select1 = "SELECT COUNT(data) FROM %s"; + + List explain1 = sql("EXPLAIN " + select1, tableName); + String explainString1 = explain1.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates = false; + if (explainString1.contains("count(data)")) { + explainContainsPushDownAggregates = true; + } + + // count(data) is not pushed down because the metrics mode is `none` + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); + + List actual1 = sql(select1, tableName); + List expected1 = Lists.newArrayList(); + expected1.add(new Object[] {6L}); + assertEquals("expected and actual should equal", expected1, actual1); + + String select2 = "SELECT COUNT(id) FROM %s"; + List explain2 = sql("EXPLAIN " + select2, tableName); + String explainString2 = explain2.get(0)[0].toString().toLowerCase(Locale.ROOT); + if (explainString2.contains("count(id)")) { + explainContainsPushDownAggregates = true; + } + + // count(id) is pushed down because the metrics mode is `counts` + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); + + List actual2 = sql(select2, tableName); + List expected2 = Lists.newArrayList(); + expected2.add(new Object[] {6L}); + assertEquals("expected and actual should equal", expected2, actual2); + + String select3 = "SELECT COUNT(id), MAX(id) FROM %s"; + explainContainsPushDownAggregates = false; + List explain3 = sql("EXPLAIN " + select3, tableName); + String explainString3 = explain3.get(0)[0].toString().toLowerCase(Locale.ROOT); + if (explainString3.contains("count(id)")) { + explainContainsPushDownAggregates = true; + } + + // COUNT(id), MAX(id) are not pushed down because MAX(id) is not pushed down (metrics mode is + // `counts`) + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); + + List actual3 = sql(select3, tableName); + List expected3 = Lists.newArrayList(); + expected3.add(new Object[] {6L, 3L}); + assertEquals("expected and actual should equal", expected3, actual3); + } + + @TestTemplate + public void testAggregateNotPushDownForStringType() { + sql("CREATE TABLE %s (id LONG, data STRING) USING iceberg", tableName); + sql( + "INSERT INTO TABLE %s VALUES (1, '1111'), (1, '2222'), (2, '3333'), (2, '4444'), (3, '5555'), (3, '6666') ", + tableName); + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + tableName, TableProperties.DEFAULT_WRITE_METRICS_MODE, "truncate(16)"); + + String select1 = "SELECT MAX(id), MAX(data) FROM %s"; + + List explain1 = sql("EXPLAIN " + select1, tableName); + String explainString1 = explain1.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates = false; + if (explainString1.contains("max(id)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); + + List actual1 = sql(select1, tableName); + List expected1 = Lists.newArrayList(); + expected1.add(new Object[] {3L, "6666"}); + assertEquals("expected and actual should equal", expected1, actual1); + + String select2 = "SELECT COUNT(data) FROM %s"; + List explain2 = sql("EXPLAIN " + select2, tableName); + String explainString2 = explain2.get(0)[0].toString().toLowerCase(Locale.ROOT); + if (explainString2.contains("count(data)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); + + List actual2 = sql(select2, tableName); + List expected2 = Lists.newArrayList(); + expected2.add(new Object[] {6L}); + assertEquals("expected and actual should equal", expected2, actual2); + + explainContainsPushDownAggregates = false; + sql( + "ALTER TABLE %s SET TBLPROPERTIES('%s' '%s')", + tableName, TableProperties.DEFAULT_WRITE_METRICS_MODE, "full"); + String select3 = "SELECT count(data), max(data) FROM %s"; + List explain3 = sql("EXPLAIN " + select3, tableName); + String explainString3 = explain3.get(0)[0].toString().toLowerCase(Locale.ROOT); + if (explainString3.contains("count(data)") && explainString3.contains("max(data)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); + + List actual3 = sql(select3, tableName); + List expected3 = Lists.newArrayList(); + expected3.add(new Object[] {6L, "6666"}); + assertEquals("expected and actual should equal", expected3, actual3); + } + + @TestTemplate + public void testAggregatePushDownWithDataFilter() { + testAggregatePushDownWithFilter(false); + } + + @TestTemplate + public void testAggregatePushDownWithPartitionFilter() { + testAggregatePushDownWithFilter(true); + } + + private void testAggregatePushDownWithFilter(boolean partitionFilerOnly) { + String createTable; + if (!partitionFilerOnly) { + createTable = "CREATE TABLE %s (id LONG, data INT) USING iceberg"; + } else { + createTable = "CREATE TABLE %s (id LONG, data INT) USING iceberg PARTITIONED BY (id)"; + } + + sql(createTable, tableName); + + sql( + "INSERT INTO TABLE %s VALUES" + + " (1, 11)," + + " (1, 22)," + + " (2, 33)," + + " (2, 44)," + + " (3, 55)," + + " (3, 66) ", + tableName); + + String select = "SELECT MIN(data) FROM %s WHERE id > 1"; + + List explain = sql("EXPLAIN " + select, tableName); + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates = false; + if (explainString.contains("min(data)")) { + explainContainsPushDownAggregates = true; + } + + if (!partitionFilerOnly) { + // Filters are not completely pushed down, we can't push down aggregates + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); + } else { + // Filters are not completely pushed down, we can push down aggregates + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); + } + + List actual = sql(select, tableName); + List expected = Lists.newArrayList(); + expected.add(new Object[] {33}); + assertEquals("expected and actual should equal", expected, actual); + } + + @TestTemplate + public void testAggregateWithComplexType() { + sql("CREATE TABLE %s (id INT, complex STRUCT) USING iceberg", tableName); + sql( + "INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", 3, \"c2\", \"v1\"))," + + "(2, named_struct(\"c1\", 2, \"c2\", \"v2\")), (3, null)", + tableName); + String select1 = "SELECT count(complex), count(id) FROM %s"; + List explain = sql("EXPLAIN " + select1, tableName); + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates = false; + if (explainString.contains("count(complex)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("count not pushed down for complex types") + .isFalse(); + + List actual = sql(select1, tableName); + List expected = Lists.newArrayList(); + expected.add(new Object[] {2L, 3L}); + assertEquals("count not push down", actual, expected); + + String select2 = "SELECT max(complex) FROM %s"; + explain = sql("EXPLAIN " + select2, tableName); + explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + explainContainsPushDownAggregates = false; + if (explainString.contains("max(complex)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("max not pushed down for complex types") + .isFalse(); + } + + @TestTemplate + public void testAggregationPushdownStructInteger() { + sql("CREATE TABLE %s (id BIGINT, struct_with_int STRUCT) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", NULL))", tableName); + sql("INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", 2))", tableName); + sql("INSERT INTO TABLE %s VALUES (3, named_struct(\"c1\", 3))", tableName); + + String query = "SELECT COUNT(%s), MAX(%s), MIN(%s) FROM %s"; + String aggField = "struct_with_int.c1"; + assertAggregates(sql(query, aggField, aggField, aggField, tableName), 2L, 3L, 2L); + assertExplainContains( + sql("EXPLAIN " + query, aggField, aggField, aggField, tableName), + "count(struct_with_int.c1)", + "max(struct_with_int.c1)", + "min(struct_with_int.c1)"); + } + + @TestTemplate + public void testAggregationPushdownNestedStruct() { + sql( + "CREATE TABLE %s (id BIGINT, struct_with_int STRUCT>>>) USING iceberg", + tableName); + sql( + "INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", named_struct(\"c2\", named_struct(\"c3\", named_struct(\"c4\", NULL)))))", + tableName); + sql( + "INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", named_struct(\"c2\", named_struct(\"c3\", named_struct(\"c4\", 2)))))", + tableName); + sql( + "INSERT INTO TABLE %s VALUES (3, named_struct(\"c1\", named_struct(\"c2\", named_struct(\"c3\", named_struct(\"c4\", 3)))))", + tableName); + + String query = "SELECT COUNT(%s), MAX(%s), MIN(%s) FROM %s"; + String aggField = "struct_with_int.c1.c2.c3.c4"; + + assertAggregates(sql(query, aggField, aggField, aggField, tableName), 2L, 3L, 2L); + + assertExplainContains( + sql("EXPLAIN " + query, aggField, aggField, aggField, tableName), + "count(struct_with_int.c1.c2.c3.c4)", + "max(struct_with_int.c1.c2.c3.c4)", + "min(struct_with_int.c1.c2.c3.c4)"); + } + + @TestTemplate + public void testAggregationPushdownStructTimestamp() { + sql( + "CREATE TABLE %s (id BIGINT, struct_with_ts STRUCT) USING iceberg", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", NULL))", tableName); + sql( + "INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", timestamp('2023-01-30T22:22:22Z')))", + tableName); + sql( + "INSERT INTO TABLE %s VALUES (3, named_struct(\"c1\", timestamp('2023-01-30T22:23:23Z')))", + tableName); + + String query = "SELECT COUNT(%s), MAX(%s), MIN(%s) FROM %s"; + String aggField = "struct_with_ts.c1"; + + assertAggregates( + sql(query, aggField, aggField, aggField, tableName), + 2L, + new Timestamp(1675117403000L), + new Timestamp(1675117342000L)); + + assertExplainContains( + sql("EXPLAIN " + query, aggField, aggField, aggField, tableName), + "count(struct_with_ts.c1)", + "max(struct_with_ts.c1)", + "min(struct_with_ts.c1)"); + } + + @TestTemplate + public void testAggregationPushdownOnBucketedColumn() { + sql( + "CREATE TABLE %s (id BIGINT, struct_with_int STRUCT) USING iceberg PARTITIONED BY (bucket(8, id))", + tableName); + + sql("INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", NULL))", tableName); + sql("INSERT INTO TABLE %s VALUES (null, named_struct(\"c1\", 2))", tableName); + sql("INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", 3))", tableName); + + String query = "SELECT COUNT(%s), MAX(%s), MIN(%s) FROM %s"; + String aggField = "id"; + assertAggregates(sql(query, aggField, aggField, aggField, tableName), 2L, 2L, 1L); + assertExplainContains( + sql("EXPLAIN " + query, aggField, aggField, aggField, tableName), + "count(id)", + "max(id)", + "min(id)"); + } + + private void assertAggregates( + List actual, Object expectedCount, Object expectedMax, Object expectedMin) { + Object actualCount = actual.get(0)[0]; + Object actualMax = actual.get(0)[1]; + Object actualMin = actual.get(0)[2]; + + assertThat(actualCount).as("Expected and actual count should equal").isEqualTo(expectedCount); + assertThat(actualMax).as("Expected and actual max should equal").isEqualTo(expectedMax); + assertThat(actualMin).as("Expected and actual min should equal").isEqualTo(expectedMin); + } + + private void assertExplainContains(List explain, String... expectedFragments) { + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + Arrays.stream(expectedFragments) + .forEach( + fragment -> + assertThat(explainString) + .as("Expected to find plan fragment in explain plan") + .contains(fragment)); + } + + @TestTemplate + public void testAggregatePushDownInDeleteCopyOnWrite() { + sql("CREATE TABLE %s (id LONG, data INT) USING iceberg", tableName); + sql( + "INSERT INTO TABLE %s VALUES (1, 1111), (1, 2222), (2, 3333), (2, 4444), (3, 5555), (3, 6666) ", + tableName); + sql("DELETE FROM %s WHERE data = 1111", tableName); + String select = "SELECT max(data), min(data), count(data) FROM %s"; + + List explain = sql("EXPLAIN " + select, tableName); + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates = false; + if (explainString.contains("max(data)") + && explainString.contains("min(data)") + && explainString.contains("count(data)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("min/max/count pushed down for deleted") + .isTrue(); + + List actual = sql(select, tableName); + List expected = Lists.newArrayList(); + expected.add(new Object[] {6666, 2222, 5L}); + assertEquals("min/max/count push down", expected, actual); + } + + @TestTemplate + public void testAggregatePushDownForTimeTravel() { + sql("CREATE TABLE %s (id LONG, data INT) USING iceberg", tableName); + sql( + "INSERT INTO TABLE %s VALUES (1, 1111), (1, 2222), (2, 3333), (2, 4444), (3, 5555), (3, 6666) ", + tableName); + + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + List expected1 = sql("SELECT count(id) FROM %s", tableName); + + sql("INSERT INTO %s VALUES (4, 7777), (5, 8888)", tableName); + List expected2 = sql("SELECT count(id) FROM %s", tableName); + + List explain1 = + sql("EXPLAIN SELECT count(id) FROM %s VERSION AS OF %s", tableName, snapshotId); + String explainString1 = explain1.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates1 = false; + if (explainString1.contains("count(id)")) { + explainContainsPushDownAggregates1 = true; + } + assertThat(explainContainsPushDownAggregates1).as("count pushed down").isTrue(); + + List actual1 = + sql("SELECT count(id) FROM %s VERSION AS OF %s", tableName, snapshotId); + assertEquals("count push down", expected1, actual1); + + List explain2 = sql("EXPLAIN SELECT count(id) FROM %s", tableName); + String explainString2 = explain2.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates2 = false; + if (explainString2.contains("count(id)")) { + explainContainsPushDownAggregates2 = true; + } + + assertThat(explainContainsPushDownAggregates2).as("count pushed down").isTrue(); + + List actual2 = sql("SELECT count(id) FROM %s", tableName); + assertEquals("count push down", expected2, actual2); + } + + @TestTemplate + public void testAllNull() { + sql("CREATE TABLE %s (id int, data int) USING iceberg PARTITIONED BY (id)", tableName); + sql( + "INSERT INTO %s VALUES (1, null)," + + "(1, null), " + + "(2, null), " + + "(2, null), " + + "(3, null), " + + "(3, null)", + tableName); + String select = "SELECT count(*), max(data), min(data), count(data) FROM %s"; + + List explain = sql("EXPLAIN " + select, tableName); + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates = false; + if (explainString.contains("max(data)") + && explainString.contains("min(data)") + && explainString.contains("count(data)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); + + List actual = sql(select, tableName); + List expected = Lists.newArrayList(); + expected.add(new Object[] {6L, null, null, 0L}); + assertEquals("min/max/count push down", expected, actual); + } + + @TestTemplate + public void testAllNaN() { + sql("CREATE TABLE %s (id int, data float) USING iceberg PARTITIONED BY (id)", tableName); + sql( + "INSERT INTO %s VALUES (1, float('nan'))," + + "(1, float('nan')), " + + "(2, float('nan')), " + + "(2, float('nan')), " + + "(3, float('nan')), " + + "(3, float('nan'))", + tableName); + String select = "SELECT count(*), max(data), min(data), count(data) FROM %s"; + + List explain = sql("EXPLAIN " + select, tableName); + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates = false; + if (explainString.contains("max(data)") + || explainString.contains("min(data)") + || explainString.contains("count(data)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); + + List actual = sql(select, tableName); + List expected = Lists.newArrayList(); + expected.add(new Object[] {6L, Float.NaN, Float.NaN, 6L}); + assertEquals("expected and actual should equal", expected, actual); + } + + @TestTemplate + public void testNaN() { + sql("CREATE TABLE %s (id int, data float) USING iceberg PARTITIONED BY (id)", tableName); + sql( + "INSERT INTO %s VALUES (1, float('nan'))," + + "(1, float('nan')), " + + "(2, 2), " + + "(2, float('nan')), " + + "(3, float('nan')), " + + "(3, 1)", + tableName); + String select = "SELECT count(*), max(data), min(data), count(data) FROM %s"; + + List explain = sql("EXPLAIN " + select, tableName); + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates = false; + if (explainString.contains("max(data)") + || explainString.contains("min(data)") + || explainString.contains("count(data)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("explain should not contain the pushed down aggregates") + .isFalse(); + + List actual = sql(select, tableName); + List expected = Lists.newArrayList(); + expected.add(new Object[] {6L, Float.NaN, 1.0F, 6L}); + assertEquals("expected and actual should equal", expected, actual); + } + + @TestTemplate + public void testInfinity() { + sql( + "CREATE TABLE %s (id int, data1 float, data2 double, data3 double) USING iceberg PARTITIONED BY (id)", + tableName); + sql( + "INSERT INTO %s VALUES (1, float('-infinity'), double('infinity'), 1.23), " + + "(1, float('-infinity'), double('infinity'), -1.23), " + + "(1, float('-infinity'), double('infinity'), double('infinity')), " + + "(1, float('-infinity'), double('infinity'), 2.23), " + + "(1, float('-infinity'), double('infinity'), double('-infinity')), " + + "(1, float('-infinity'), double('infinity'), -2.23)", + tableName); + String select = + "SELECT count(*), max(data1), min(data1), count(data1), max(data2), min(data2), count(data2), max(data3), min(data3), count(data3) FROM %s"; + + List explain = sql("EXPLAIN " + select, tableName); + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + boolean explainContainsPushDownAggregates = false; + if (explainString.contains("max(data1)") + && explainString.contains("min(data1)") + && explainString.contains("count(data1)") + && explainString.contains("max(data2)") + && explainString.contains("min(data2)") + && explainString.contains("count(data2)") + && explainString.contains("max(data3)") + && explainString.contains("min(data3)") + && explainString.contains("count(data3)")) { + explainContainsPushDownAggregates = true; + } + + assertThat(explainContainsPushDownAggregates) + .as("explain should contain the pushed down aggregates") + .isTrue(); + + List actual = sql(select, tableName); + List expected = Lists.newArrayList(); + expected.add( + new Object[] { + 6L, + Float.NEGATIVE_INFINITY, + Float.NEGATIVE_INFINITY, + 6L, + Double.POSITIVE_INFINITY, + Double.POSITIVE_INFINITY, + 6L, + Double.POSITIVE_INFINITY, + Double.NEGATIVE_INFINITY, + 6L + }); + assertEquals("min/max/count push down", expected, actual); + } + + @TestTemplate + public void testAggregatePushDownForIncrementalScan() { + sql("CREATE TABLE %s (id LONG, data INT) USING iceberg", tableName); + sql( + "INSERT INTO TABLE %s VALUES (1, 1111), (1, 2222), (2, 3333), (2, 4444), (3, 5555), (3, 6666) ", + tableName); + long snapshotId1 = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + sql("INSERT INTO %s VALUES (4, 7777), (5, 8888)", tableName); + long snapshotId2 = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + sql("INSERT INTO %s VALUES (6, -7777), (7, 8888)", tableName); + long snapshotId3 = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + sql("INSERT INTO %s VALUES (8, 7777), (9, 9999)", tableName); + + Dataset pushdownDs = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.START_SNAPSHOT_ID, snapshotId2) + .option(SparkReadOptions.END_SNAPSHOT_ID, snapshotId3) + .load(tableName) + .agg(functions.min("data"), functions.max("data"), functions.count("data")); + String explain1 = pushdownDs.queryExecution().explainString(ExplainMode.fromString("simple")); + assertThat(explain1).contains("LocalTableScan", "min(data)", "max(data)", "count(data)"); + + List expected1 = Lists.newArrayList(); + expected1.add(new Object[] {-7777, 8888, 2L}); + assertEquals("min/max/count push down", expected1, rowsToJava(pushdownDs.collectAsList())); + + Dataset unboundedPushdownDs = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.START_SNAPSHOT_ID, snapshotId1) + .load(tableName) + .agg(functions.min("data"), functions.max("data"), functions.count("data")); + String explain2 = + unboundedPushdownDs.queryExecution().explainString(ExplainMode.fromString("simple")); + assertThat(explain2).contains("LocalTableScan", "min(data)", "max(data)", "count(data)"); + + List expected2 = Lists.newArrayList(); + expected2.add(new Object[] {-7777, 9999, 6L}); + assertEquals( + "min/max/count push down", expected2, rowsToJava(unboundedPushdownDs.collectAsList())); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java new file mode 100644 index 000000000000..6ba7e01c402d --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java @@ -0,0 +1,333 @@ +/* + * 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.sql; + +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_REST; +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 org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.spark.SparkException; +import org.apache.spark.sql.AnalysisException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestAlterTable extends CatalogTestBase { + private final TableIdentifier renamedIdent = + TableIdentifier.of(Namespace.of("default"), "table2"); + + @BeforeEach + public void createTable() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s2", tableName); + } + + @TestTemplate + public void testAddColumnNotNull() { + assertThatThrownBy(() -> sql("ALTER TABLE %s ADD COLUMN c3 INT NOT NULL", tableName)) + .isInstanceOf(SparkException.class) + .hasMessage( + "Unsupported table change: Incompatible change: cannot add required column: c3"); + } + + @TestTemplate + public void testAddColumn() { + sql( + "ALTER TABLE %s ADD COLUMN point struct AFTER id", + tableName); + + Types.StructType expectedSchema = + Types.StructType.of( + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional( + 3, + "point", + Types.StructType.of( + NestedField.required(4, "x", Types.DoubleType.get()), + NestedField.required(5, "y", Types.DoubleType.get()))), + NestedField.optional(2, "data", Types.StringType.get())); + + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); + + sql("ALTER TABLE %s ADD COLUMN point.z double COMMENT 'May be null' FIRST", tableName); + + Types.StructType expectedSchema2 = + Types.StructType.of( + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional( + 3, + "point", + Types.StructType.of( + NestedField.optional(6, "z", Types.DoubleType.get(), "May be null"), + NestedField.required(4, "x", Types.DoubleType.get()), + NestedField.required(5, "y", Types.DoubleType.get()))), + NestedField.optional(2, "data", Types.StringType.get())); + + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema2); + } + + @TestTemplate + public void testAddColumnWithArray() { + sql("ALTER TABLE %s ADD COLUMN data2 array>", tableName); + // use the implicit column name 'element' to access member of array and add column d to struct. + sql("ALTER TABLE %s ADD COLUMN data2.element.d int", tableName); + Types.StructType expectedSchema = + Types.StructType.of( + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional(2, "data", Types.StringType.get()), + NestedField.optional( + 3, + "data2", + Types.ListType.ofOptional( + 4, + Types.StructType.of( + NestedField.optional(5, "a", Types.IntegerType.get()), + NestedField.optional(6, "b", Types.IntegerType.get()), + NestedField.optional(7, "c", Types.IntegerType.get()), + NestedField.optional(8, "d", Types.IntegerType.get()))))); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); + } + + @TestTemplate + public void testAddColumnWithMap() { + sql("ALTER TABLE %s ADD COLUMN data2 map, struct>", tableName); + // use the implicit column name 'key' and 'value' to access member of map. + // add column to value struct column + sql("ALTER TABLE %s ADD COLUMN data2.value.c int", tableName); + Types.StructType expectedSchema = + Types.StructType.of( + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional(2, "data", Types.StringType.get()), + NestedField.optional( + 3, + "data2", + Types.MapType.ofOptional( + 4, + 5, + Types.StructType.of(NestedField.optional(6, "x", Types.IntegerType.get())), + Types.StructType.of( + NestedField.optional(7, "a", Types.IntegerType.get()), + NestedField.optional(8, "b", Types.IntegerType.get()), + NestedField.optional(9, "c", Types.IntegerType.get()))))); + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); + + // should not allow changing map key column + assertThatThrownBy(() -> sql("ALTER TABLE %s ADD COLUMN data2.key.y int", tableName)) + .isInstanceOf(SparkException.class) + .hasMessageStartingWith("Unsupported table change: Cannot add fields to map keys:"); + } + + @TestTemplate + public void testAddColumnWithDefaultValuesUnsupported() { + assertThatThrownBy( + () -> sql("ALTER TABLE %s ADD COLUMN col_with_default int DEFAULT 123", tableName)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageStartingWith( + "Cannot add column col_with_default since setting default values in Spark is currently unsupported"); + } + + @TestTemplate + public void testDropColumn() { + sql("ALTER TABLE %s DROP COLUMN data", tableName); + + Types.StructType expectedSchema = + Types.StructType.of(NestedField.required(1, "id", Types.LongType.get())); + + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); + } + + @TestTemplate + public void testRenameColumn() { + sql("ALTER TABLE %s RENAME COLUMN id TO row_id", tableName); + + Types.StructType expectedSchema = + Types.StructType.of( + NestedField.required(1, "row_id", Types.LongType.get()), + NestedField.optional(2, "data", Types.StringType.get())); + + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); + } + + @TestTemplate + public void testAlterColumnComment() { + sql("ALTER TABLE %s ALTER COLUMN id COMMENT 'Record id'", tableName); + + Types.StructType expectedSchema = + Types.StructType.of( + NestedField.required(1, "id", Types.LongType.get(), "Record id"), + NestedField.optional(2, "data", Types.StringType.get())); + + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); + } + + @TestTemplate + public void testAlterColumnType() { + sql("ALTER TABLE %s ADD COLUMN count int", tableName); + sql("ALTER TABLE %s ALTER COLUMN count TYPE bigint", tableName); + + Types.StructType expectedSchema = + Types.StructType.of( + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional(2, "data", Types.StringType.get()), + NestedField.optional(3, "count", Types.LongType.get())); + + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); + } + + @TestTemplate + public void testAlterColumnDropNotNull() { + sql("ALTER TABLE %s ALTER COLUMN id DROP NOT NULL", tableName); + + Types.StructType expectedSchema = + Types.StructType.of( + NestedField.optional(1, "id", Types.LongType.get()), + NestedField.optional(2, "data", Types.StringType.get())); + + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); + } + + @TestTemplate + public void testAlterColumnSetNotNull() { + // no-op changes are allowed + sql("ALTER TABLE %s ALTER COLUMN id SET NOT NULL", tableName); + + Types.StructType expectedSchema = + Types.StructType.of( + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional(2, "data", Types.StringType.get())); + + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); + + assertThatThrownBy(() -> sql("ALTER TABLE %s ALTER COLUMN data SET NOT NULL", tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Cannot change nullable column to non-nullable: data"); + } + + @TestTemplate + public void testAlterColumnPositionAfter() { + sql("ALTER TABLE %s ADD COLUMN count int", tableName); + sql("ALTER TABLE %s ALTER COLUMN count AFTER id", tableName); + + Types.StructType expectedSchema = + Types.StructType.of( + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional(3, "count", Types.IntegerType.get()), + NestedField.optional(2, "data", Types.StringType.get())); + + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); + } + + @TestTemplate + public void testAlterColumnPositionFirst() { + sql("ALTER TABLE %s ADD COLUMN count int", tableName); + sql("ALTER TABLE %s ALTER COLUMN count FIRST", tableName); + + Types.StructType expectedSchema = + Types.StructType.of( + NestedField.optional(3, "count", Types.IntegerType.get()), + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional(2, "data", Types.StringType.get())); + + assertThat(validationCatalog.loadTable(tableIdent).schema().asStruct()) + .as("Schema should match expected") + .isEqualTo(expectedSchema); + } + + @TestTemplate + public void testTableRename() { + assumeThat(catalogConfig.get(ICEBERG_CATALOG_TYPE)) + .as( + "need to fix https://github.com/apache/iceberg/issues/11154 before enabling this for the REST catalog") + .isNotEqualTo(ICEBERG_CATALOG_TYPE_REST); + assumeThat(validationCatalog) + .as("Hadoop catalog does not support rename") + .isNotInstanceOf(HadoopCatalog.class); + + assertThat(validationCatalog.tableExists(tableIdent)).as("Initial name should exist").isTrue(); + assertThat(validationCatalog.tableExists(renamedIdent)) + .as("New name should not exist") + .isFalse(); + + sql("ALTER TABLE %s RENAME TO %s2", tableName, tableName); + + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Initial name should not exist") + .isFalse(); + assertThat(validationCatalog.tableExists(renamedIdent)).as("New name should exist").isTrue(); + } + + @TestTemplate + public void testSetTableProperties() { + sql("ALTER TABLE %s SET TBLPROPERTIES ('prop'='value')", tableName); + + assertThat(validationCatalog.loadTable(tableIdent).properties()) + .as("Should have the new table property") + .containsEntry("prop", "value"); + + sql("ALTER TABLE %s UNSET TBLPROPERTIES ('prop')", tableName); + + assertThat(validationCatalog.loadTable(tableIdent).properties()) + .as("Should not have the removed table property") + .doesNotContainKey("prop"); + + String[] reservedProperties = new String[] {"sort-order", "identifier-fields"}; + for (String reservedProp : reservedProperties) { + assertThatThrownBy( + () -> sql("ALTER TABLE %s SET TBLPROPERTIES ('%s'='value')", tableName, reservedProp)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageStartingWith( + "Cannot specify the '%s' because it's a reserved table property", reservedProp); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java new file mode 100644 index 000000000000..3dee6e1e1d54 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java @@ -0,0 +1,120 @@ +/* + * 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.sql; + +import static org.apache.iceberg.Files.localInput; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.UnitestKMS; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.types.Types; +import org.apache.parquet.crypto.ParquetCryptoRuntimeException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestCTASEncryption extends CatalogTestBase { + private static Map appendCatalogEncryptionProperties(Map props) { + Map newProps = Maps.newHashMap(); + newProps.putAll(props); + newProps.put(CatalogProperties.ENCRYPTION_KMS_IMPL, UnitestKMS.class.getCanonicalName()); + return newProps; + } + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + appendCatalogEncryptionProperties(SparkCatalogConfig.HIVE.properties()) + } + }; + } + + @BeforeEach + public void createTables() { + sql("CREATE TABLE %s (id bigint, data string, float float) USING iceberg ", tableName + "1"); + sql( + "INSERT INTO %s VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', float('NaN'))", + tableName + "1"); + + sql( + "CREATE TABLE %s USING iceberg " + + "TBLPROPERTIES ( " + + "'encryption.key-id'='%s', 'format-version'='3')" + + " AS SELECT * from %s", + tableName, UnitestKMS.MASTER_KEY_NAME1, tableName + "1"); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s", tableName + "1"); + } + + @TestTemplate + public void testSelect() { + List expected = + ImmutableList.of(row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", Float.NaN)); + + assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); + } + + @TestTemplate + public void testDirectDataFileRead() { + List dataFileTable = + sql("SELECT file_path FROM %s.%s", tableName, MetadataTableType.ALL_DATA_FILES); + List dataFiles = + Streams.concat(dataFileTable.stream()) + .map(row -> (String) row[0]) + .collect(Collectors.toList()); + + if (dataFiles.isEmpty()) { + throw new RuntimeException("No data files found for table " + tableName); + } + + Schema schema = new Schema(optional(0, "id", Types.IntegerType.get())); + for (String filePath : dataFiles) { + assertThatThrownBy( + () -> + Parquet.read(localInput(filePath)) + .project(schema) + .callInit() + .build() + .iterator() + .next()) + .isInstanceOf(ParquetCryptoRuntimeException.class) + .hasMessageContaining("Trying to read file with encrypted footer. No keys available"); + } + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java new file mode 100644 index 000000000000..7fb1fda3364f --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java @@ -0,0 +1,457 @@ +/* + * 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.sql; + +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_REST; +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.nio.file.Files; +import java.util.UUID; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.NestedField; +import org.apache.iceberg.types.Types.StructType; +import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestCreateTable extends CatalogTestBase { + + @AfterEach + public void dropTestTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testTransformIgnoreCase() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + sql( + "CREATE TABLE IF NOT EXISTS %s (id BIGINT NOT NULL, ts timestamp) " + + "USING iceberg partitioned by (HOURS(ts))", + tableName); + assertThat(validationCatalog.tableExists(tableIdent)).as("Table should already exist").isTrue(); + sql( + "CREATE TABLE IF NOT EXISTS %s (id BIGINT NOT NULL, ts timestamp) " + + "USING iceberg partitioned by (hours(ts))", + tableName); + assertThat(validationCatalog.tableExists(tableIdent)).as("Table should already exist").isTrue(); + } + + @TestTemplate + public void testTransformSingularForm() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + sql( + "CREATE TABLE IF NOT EXISTS %s (id BIGINT NOT NULL, ts timestamp) " + + "USING iceberg partitioned by (hour(ts))", + tableName); + assertThat(validationCatalog.tableExists(tableIdent)).as("Table should exist").isTrue(); + } + + @TestTemplate + public void testTransformPluralForm() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + sql( + "CREATE TABLE IF NOT EXISTS %s (id BIGINT NOT NULL, ts timestamp) " + + "USING iceberg partitioned by (hours(ts))", + tableName); + assertThat(validationCatalog.tableExists(tableIdent)).as("Table should exist").isTrue(); + } + + @TestTemplate + public void testCreateTable() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + sql("CREATE TABLE %s (id BIGINT NOT NULL, data STRING) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table).as("Should load the new table").isNotNull(); + + StructType expectedSchema = + StructType.of( + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional(2, "data", Types.StringType.get())); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); + assertThat(table.spec().fields()).as("Should not be partitioned").isEmpty(); + assertThat(table.properties()).doesNotContainKey(TableProperties.DEFAULT_FILE_FORMAT); + } + + @TestTemplate + public void testCreateTablePartitionedByUUID() { + assertThat(validationCatalog.tableExists(tableIdent)).isFalse(); + Schema schema = new Schema(1, Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); + PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("uuid", 16).build(); + validationCatalog.createTable(tableIdent, schema, spec); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table).isNotNull(); + + StructType expectedSchema = + StructType.of(Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); + assertThat(table.schema().asStruct()).isEqualTo(expectedSchema); + assertThat(table.spec().fields()).hasSize(1); + + String uuid = UUID.randomUUID().toString(); + + sql("INSERT INTO %s VALUES('%s')", tableName, uuid); + + assertThat(sql("SELECT uuid FROM %s", tableName)).singleElement().isEqualTo(row(uuid)); + } + + @TestTemplate + public void testCreateTableInRootNamespace() { + assumeThat(catalogName) + .as("Hadoop has no default namespace configured") + .isEqualTo("testhadoop"); + + try { + sql("CREATE TABLE %s.table (id bigint) USING iceberg", catalogName); + } finally { + sql("DROP TABLE IF EXISTS %s.table", catalogName); + } + } + + @TestTemplate + public void testCreateTableUsingParquet() { + assumeThat(catalogName) + .as("Not working with session catalog because Spark will not use v2 for a Parquet table") + .isNotEqualTo("spark_catalog"); + + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + sql("CREATE TABLE %s (id BIGINT NOT NULL, data STRING) USING parquet", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table).as("Should load the new table").isNotNull(); + + StructType expectedSchema = + StructType.of( + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional(2, "data", Types.StringType.get())); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); + assertThat(table.spec().fields()).as("Should not be partitioned").isEmpty(); + assertThat(table.properties()).containsEntry(TableProperties.DEFAULT_FILE_FORMAT, "parquet"); + + assertThatThrownBy( + () -> + sql( + "CREATE TABLE %s.default.fail (id BIGINT NOT NULL, data STRING) USING crocodile", + catalogName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unsupported format in USING: crocodile"); + } + + @TestTemplate + public void testCreateTablePartitionedBy() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL, created_at TIMESTAMP, category STRING, data STRING) " + + "USING iceberg " + + "PARTITIONED BY (category, bucket(8, id), days(created_at))", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table).as("Should load the new table").isNotNull(); + + StructType expectedSchema = + StructType.of( + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional(2, "created_at", Types.TimestampType.withZone()), + NestedField.optional(3, "category", Types.StringType.get()), + NestedField.optional(4, "data", Types.StringType.get())); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); + + PartitionSpec expectedSpec = + PartitionSpec.builderFor(new Schema(expectedSchema.fields())) + .identity("category") + .bucket("id", 8) + .day("created_at") + .build(); + assertThat(table.spec()).as("Should be partitioned correctly").isEqualTo(expectedSpec); + assertThat(table.properties()).doesNotContainKey(TableProperties.DEFAULT_FILE_FORMAT); + } + + @TestTemplate + public void testCreateTableColumnComments() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL COMMENT 'Unique identifier', data STRING COMMENT 'Data value') " + + "USING iceberg", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table).as("Should load the new table").isNotNull(); + + StructType expectedSchema = + StructType.of( + NestedField.required(1, "id", Types.LongType.get(), "Unique identifier"), + NestedField.optional(2, "data", Types.StringType.get(), "Data value")); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); + assertThat(table.spec().fields()).as("Should not be partitioned").isEmpty(); + assertThat(table.properties()).doesNotContainKey(TableProperties.DEFAULT_FILE_FORMAT); + } + + @TestTemplate + public void testCreateTableComment() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL, data STRING) " + + "USING iceberg " + + "COMMENT 'Table doc'", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table).as("Should load the new table").isNotNull(); + + StructType expectedSchema = + StructType.of( + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional(2, "data", Types.StringType.get())); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); + assertThat(table.spec().fields()).as("Should not be partitioned").isEmpty(); + assertThat(table.properties()) + .doesNotContainKey(TableProperties.DEFAULT_FILE_FORMAT) + .containsEntry(TableCatalog.PROP_COMMENT, "Table doc"); + } + + @TestTemplate + public void testCreateTableLocation() throws Exception { + assumeThat(validationCatalog) + .as("Cannot set custom locations for Hadoop catalog tables") + .isNotInstanceOf(HadoopCatalog.class); + + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + File tableLocation = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableLocation.delete()).isTrue(); + + String location = "file:" + tableLocation; + + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL, data STRING) " + + "USING iceberg " + + "LOCATION '%s'", + tableName, location); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table).as("Should load the new table").isNotNull(); + + StructType expectedSchema = + StructType.of( + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional(2, "data", Types.StringType.get())); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); + assertThat(table.spec().fields()).as("Should not be partitioned").isEmpty(); + assertThat(table.properties()).doesNotContainKey(TableProperties.DEFAULT_FILE_FORMAT); + assertThat(table.location()).as("Should have a custom table location").isEqualTo(location); + } + + @TestTemplate + public void testCreateTableProperties() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL, data STRING) " + + "USING iceberg " + + "TBLPROPERTIES (p1=2, p2='x')", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table).as("Should load the new table").isNotNull(); + + StructType expectedSchema = + StructType.of( + NestedField.required(1, "id", Types.LongType.get()), + NestedField.optional(2, "data", Types.StringType.get())); + assertThat(table.schema().asStruct()) + .as("Should have the expected schema") + .isEqualTo(expectedSchema); + assertThat(table.spec().fields()).as("Should not be partitioned").isEmpty(); + assertThat(table.properties()).containsEntry("p1", "2").containsEntry("p2", "x"); + } + + @TestTemplate + public void testCreateTableCommitProperties() { + assumeThat(catalogConfig.get(ICEBERG_CATALOG_TYPE)) + .as( + "need to fix https://github.com/apache/iceberg/issues/11554 before enabling this for the REST catalog") + .isNotEqualTo(ICEBERG_CATALOG_TYPE_REST); + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + assertThatThrownBy( + () -> + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL, data STRING) " + + "USING iceberg " + + "TBLPROPERTIES ('commit.retry.num-retries'='x', p2='x')", + tableName)) + .isInstanceOf(ValidationException.class) + .hasMessage("Table property commit.retry.num-retries must have integer value"); + + assertThatThrownBy( + () -> + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL, data STRING) " + + "USING iceberg " + + "TBLPROPERTIES ('commit.retry.max-wait-ms'='-1')", + tableName)) + .isInstanceOf(ValidationException.class) + .hasMessage("Table property commit.retry.max-wait-ms must have non negative integer value"); + + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL, data STRING) " + + "USING iceberg " + + "TBLPROPERTIES ('commit.retry.num-retries'='1', 'commit.retry.max-wait-ms'='3000')", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.properties()) + .containsEntry(TableProperties.COMMIT_NUM_RETRIES, "1") + .containsEntry(TableProperties.COMMIT_MAX_RETRY_WAIT_MS, "3000"); + } + + @TestTemplate + public void testCreateTableWithFormatV2ThroughTableProperty() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL, data STRING) " + + "USING iceberg " + + "TBLPROPERTIES ('format-version'='2')", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(((BaseTable) table).operations().current().formatVersion()) + .as("should create table using format v2") + .isEqualTo(2); + } + + @TestTemplate + public void testUpgradeTableWithFormatV2ThroughTableProperty() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL, data STRING) " + + "USING iceberg " + + "TBLPROPERTIES ('format-version'='1')", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + TableOperations ops = ((BaseTable) table).operations(); + assertThat(ops.refresh().formatVersion()) + .as("should create table using format v1") + .isEqualTo(1); + + sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='2')", tableName); + assertThat(ops.refresh().formatVersion()) + .as("should update table to use format v2") + .isEqualTo(2); + } + + @TestTemplate + public void testDowngradeTableToFormatV1ThroughTablePropertyFails() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + sql( + "CREATE TABLE %s " + + "(id BIGINT NOT NULL, data STRING) " + + "USING iceberg " + + "TBLPROPERTIES ('format-version'='2')", + tableName); + + Table table = validationCatalog.loadTable(tableIdent); + TableOperations ops = ((BaseTable) table).operations(); + assertThat(ops.refresh().formatVersion()) + .as("should create table using format v2") + .isEqualTo(2); + + assertThatThrownBy( + () -> sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='1')", tableName)) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot downgrade v2 table to v1"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java new file mode 100644 index 000000000000..6cb0ca5a6611 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTableAsSelect.java @@ -0,0 +1,451 @@ +/* + * 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.sql; + +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.lit; +import static org.apache.spark.sql.functions.when; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestCreateTableAsSelect extends CatalogTestBase { + + @Parameter(index = 3) + private String sourceName; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, sourceName = {3}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties(), + SparkCatalogConfig.HIVE.catalogName() + ".default.source" + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + SparkCatalogConfig.HADOOP.catalogName() + ".default.source" + }, + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties(), + "default.source" + } + }; + } + + @BeforeEach + public void createTableIfNotExists() { + sql( + "CREATE TABLE IF NOT EXISTS %s (id bigint NOT NULL, data string) " + + "USING iceberg PARTITIONED BY (truncate(id, 3))", + sourceName); + sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c'), (4, 'd'), (5, 'e')", sourceName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testUnpartitionedCTAS() { + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", tableName, sourceName); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + Table ctasTable = validationCatalog.loadTable(tableIdent); + + assertThat(ctasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(ctasTable.spec().fields()).as("Should be an unpartitioned table").isEmpty(); + assertEquals( + "Should have rows matching the source table", + sql("SELECT * FROM %s ORDER BY id", sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testPartitionedCTAS() { + sql( + "CREATE TABLE %s USING iceberg PARTITIONED BY (id) AS SELECT * FROM %s ORDER BY id", + tableName, sourceName); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + PartitionSpec expectedSpec = PartitionSpec.builderFor(expectedSchema).identity("id").build(); + + Table ctasTable = validationCatalog.loadTable(tableIdent); + + assertThat(ctasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(ctasTable.spec()).as("Should be partitioned by id").isEqualTo(expectedSpec); + assertEquals( + "Should have rows matching the source table", + sql("SELECT * FROM %s ORDER BY id", sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testCTASWriteDistributionModeRespected() { + sql( + "CREATE TABLE %s USING iceberg PARTITIONED BY (bucket(2, id)) AS SELECT * FROM %s", + tableName, sourceName); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + PartitionSpec expectedSpec = PartitionSpec.builderFor(expectedSchema).bucket("id", 2).build(); + + Table ctasTable = validationCatalog.loadTable(tableIdent); + + assertThat(ctasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(ctasTable.spec()).as("Should be partitioned by id").isEqualTo(expectedSpec); + assertEquals( + "Should have rows matching the source table", + sql("SELECT * FROM %s ORDER BY id", sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testRTAS() { + sql( + "CREATE TABLE %s USING iceberg TBLPROPERTIES ('prop1'='val1', 'prop2'='val2')" + + "AS SELECT * FROM %s", + tableName, sourceName); + + assertEquals( + "Should have rows matching the source table", + sql("SELECT * FROM %s ORDER BY id", sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + sql( + "REPLACE TABLE %s USING iceberg PARTITIONED BY (part) TBLPROPERTIES ('prop1'='newval1', 'prop3'='val3') AS " + + "SELECT id, data, CASE WHEN (id %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " + + "FROM %s ORDER BY 3, 1", + tableName, sourceName); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "part", Types.StringType.get())); + + PartitionSpec expectedSpec = + PartitionSpec.builderFor(expectedSchema).identity("part").withSpecId(1).build(); + + Table rtasTable = validationCatalog.loadTable(tableIdent); + + // the replacement table has a different schema and partition spec than the original + assertThat(rtasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(rtasTable.spec()).as("Should be partitioned by part").isEqualTo(expectedSpec); + + assertEquals( + "Should have rows matching the source table", + sql( + "SELECT id, data, CASE WHEN (id %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " + + "FROM %s ORDER BY id", + sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + assertThat(rtasTable.snapshots()).as("Table should have expected snapshots").hasSize(2); + assertThat(rtasTable.properties()) + .containsEntry("prop1", "newval1") + .containsEntry("prop2", "val2") + .containsEntry("prop3", "val3"); + } + + @TestTemplate + public void testCreateRTAS() { + sql( + "CREATE OR REPLACE TABLE %s USING iceberg PARTITIONED BY (part) AS " + + "SELECT id, data, CASE WHEN (id %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " + + "FROM %s ORDER BY 3, 1", + tableName, sourceName); + + assertEquals( + "Should have rows matching the source table", + sql( + "SELECT id, data, CASE WHEN (id %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " + + "FROM %s ORDER BY id", + sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + sql( + "CREATE OR REPLACE TABLE %s USING iceberg PARTITIONED BY (part) AS " + + "SELECT 2 * id as id, data, CASE WHEN ((2 * id) %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " + + "FROM %s ORDER BY 3, 1", + tableName, sourceName); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "part", Types.StringType.get())); + + PartitionSpec expectedSpec = + PartitionSpec.builderFor(expectedSchema) + .identity("part") + .withSpecId(0) // the spec is identical and should be reused + .build(); + + Table rtasTable = validationCatalog.loadTable(tableIdent); + + // the replacement table has a different schema and partition spec than the original + assertThat(rtasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(rtasTable.spec()).as("Should be partitioned by part").isEqualTo(expectedSpec); + + assertEquals( + "Should have rows matching the source table", + sql( + "SELECT 2 * id, data, CASE WHEN ((2 * id) %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " + + "FROM %s ORDER BY id", + sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + assertThat(rtasTable.snapshots()).as("Table should have expected snapshots").hasSize(2); + } + + @TestTemplate + public void testDataFrameV2Create() throws Exception { + spark.table(sourceName).writeTo(tableName).using("iceberg").create(); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + Table ctasTable = validationCatalog.loadTable(tableIdent); + + assertThat(ctasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(ctasTable.spec().fields()).as("Should be an unpartitioned table").isEmpty(); + assertEquals( + "Should have rows matching the source table", + sql("SELECT * FROM %s ORDER BY id", sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testDataFrameV2Replace() throws Exception { + spark.table(sourceName).writeTo(tableName).using("iceberg").create(); + + assertEquals( + "Should have rows matching the source table", + sql("SELECT * FROM %s ORDER BY id", sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + spark + .table(sourceName) + .select( + col("id"), + col("data"), + when(col("id").mod(lit(2)).equalTo(lit(0)), lit("even")).otherwise("odd").as("part")) + .orderBy("part", "id") + .writeTo(tableName) + .partitionedBy(col("part")) + .using("iceberg") + .replace(); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "part", Types.StringType.get())); + + PartitionSpec expectedSpec = + PartitionSpec.builderFor(expectedSchema).identity("part").withSpecId(1).build(); + + Table rtasTable = validationCatalog.loadTable(tableIdent); + + // the replacement table has a different schema and partition spec than the original + assertThat(rtasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(rtasTable.spec()).as("Should be partitioned by part").isEqualTo(expectedSpec); + + assertEquals( + "Should have rows matching the source table", + sql( + "SELECT id, data, CASE WHEN (id %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " + + "FROM %s ORDER BY id", + sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + assertThat(rtasTable.snapshots()).as("Table should have expected snapshots").hasSize(2); + } + + @TestTemplate + public void testDataFrameV2CreateOrReplace() { + spark + .table(sourceName) + .select( + col("id"), + col("data"), + when(col("id").mod(lit(2)).equalTo(lit(0)), lit("even")).otherwise("odd").as("part")) + .orderBy("part", "id") + .writeTo(tableName) + .partitionedBy(col("part")) + .using("iceberg") + .createOrReplace(); + + assertEquals( + "Should have rows matching the source table", + sql( + "SELECT id, data, CASE WHEN (id %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " + + "FROM %s ORDER BY id", + sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + spark + .table(sourceName) + .select(col("id").multiply(lit(2)).as("id"), col("data")) + .select( + col("id"), + col("data"), + when(col("id").mod(lit(2)).equalTo(lit(0)), lit("even")).otherwise("odd").as("part")) + .orderBy("part", "id") + .writeTo(tableName) + .partitionedBy(col("part")) + .using("iceberg") + .createOrReplace(); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "part", Types.StringType.get())); + + PartitionSpec expectedSpec = + PartitionSpec.builderFor(expectedSchema) + .identity("part") + .withSpecId(0) // the spec is identical and should be reused + .build(); + + Table rtasTable = validationCatalog.loadTable(tableIdent); + + // the replacement table has a different schema and partition spec than the original + assertThat(rtasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + assertThat(rtasTable.spec()).as("Should be partitioned by part").isEqualTo(expectedSpec); + + assertEquals( + "Should have rows matching the source table", + sql( + "SELECT 2 * id, data, CASE WHEN ((2 * id) %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " + + "FROM %s ORDER BY id", + sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + assertThat(rtasTable.snapshots()).as("Table should have expected snapshots").hasSize(2); + } + + @TestTemplate + public void testCreateRTASWithPartitionSpecChanging() { + sql( + "CREATE OR REPLACE TABLE %s USING iceberg PARTITIONED BY (part) AS " + + "SELECT id, data, CASE WHEN (id %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " + + "FROM %s ORDER BY 3, 1", + tableName, sourceName); + + Table rtasTable = validationCatalog.loadTable(tableIdent); + + assertEquals( + "Should have rows matching the source table", + sql( + "SELECT id, data, CASE WHEN (id %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " + + "FROM %s ORDER BY id", + sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + // Change the partitioning of the table + rtasTable.updateSpec().removeField("part").commit(); // Spec 1 + + sql( + "CREATE OR REPLACE TABLE %s USING iceberg PARTITIONED BY (part, id) AS " + + "SELECT 2 * id as id, data, CASE WHEN ((2 * id) %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " + + "FROM %s ORDER BY 3, 1", + tableName, sourceName); + + Schema expectedSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "part", Types.StringType.get())); + + PartitionSpec expectedSpec = + PartitionSpec.builderFor(expectedSchema) + .identity("part") + .identity("id") + .withSpecId(2) // The Spec is new + .build(); + + assertThat(rtasTable.spec()).as("Should be partitioned by part and id").isEqualTo(expectedSpec); + + // the replacement table has a different schema and partition spec than the original + assertThat(rtasTable.schema().asStruct()) + .as("Should have expected nullable schema") + .isEqualTo(expectedSchema.asStruct()); + + assertEquals( + "Should have rows matching the source table", + sql( + "SELECT 2 * id, data, CASE WHEN ((2 * id) %% 2) = 0 THEN 'even' ELSE 'odd' END AS part " + + "FROM %s ORDER BY id", + sourceName), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + assertThat(rtasTable.snapshots()).as("Table should have expected snapshots").hasSize(2); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java new file mode 100644 index 000000000000..bd4a41593c34 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java @@ -0,0 +1,165 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestDeleteFrom extends CatalogTestBase { + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testDeleteFromUnpartitionedTable() throws NoSuchTableException { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.coalesce(1).writeTo(tableName).append(); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + sql("DELETE FROM %s WHERE id < 2", tableName); + + assertEquals( + "Should have no rows after successful delete", + ImmutableList.of(row(2L, "b"), row(3L, "c")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + sql("DELETE FROM %s WHERE id < 4", tableName); + + assertEquals( + "Should have no rows after successful delete", + ImmutableList.of(), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testDeleteFromTableAtSnapshot() throws NoSuchTableException { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.coalesce(1).writeTo(tableName).append(); + + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + 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"); + } + + @TestTemplate + public void testDeleteFromPartitionedTable() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id bigint, data string) " + + "USING iceberg " + + "PARTITIONED BY (truncate(id, 2))", + tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset df = spark.createDataFrame(records, SimpleRecord.class); + df.coalesce(1).writeTo(tableName).append(); + + assertEquals( + "Should have 3 rows in 2 partitions", + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + sql("DELETE FROM %s WHERE id > 2", tableName); + assertEquals( + "Should have two rows in the second partition", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + sql("DELETE FROM %s WHERE id < 2", tableName); + + assertEquals( + "Should have two rows in the second partition", + ImmutableList.of(row(2L, "b")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testDeleteFromWhereFalse() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 1 snapshot").hasSize(1); + + sql("DELETE FROM %s WHERE false", tableName); + + table.refresh(); + + assertThat(table.snapshots()).as("Delete should not produce a new snapshot").hasSize(1); + } + + @TestTemplate + public void testTruncate() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).as("Should have 1 snapshot").hasSize(1); + + sql("TRUNCATE TABLE %s", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java new file mode 100644 index 000000000000..07faae52749b --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java @@ -0,0 +1,158 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.spark.CatalogTestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestDropTable extends CatalogTestBase { + + @BeforeEach + public void createTable() { + sql("CREATE TABLE %s (id INT, name STRING) USING iceberg", tableName); + sql("INSERT INTO %s VALUES (1, 'test')", tableName); + } + + @AfterEach + public void removeTable() throws IOException { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testDropTable() throws IOException { + dropTableInternal(); + } + + @TestTemplate + public void testDropTableGCDisabled() throws IOException { + sql("ALTER TABLE %s SET TBLPROPERTIES (gc.enabled = false)", tableName); + dropTableInternal(); + } + + private void dropTableInternal() throws IOException { + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "test")), + sql("SELECT * FROM %s", tableName)); + + List manifestAndFiles = manifestsAndFiles(); + assertThat(manifestAndFiles).as("There should be 2 files for manifests and files").hasSize(2); + assertThat(checkFilesExist(manifestAndFiles, true)).as("All files should exist").isTrue(); + + sql("DROP TABLE %s", tableName); + assertThat(validationCatalog.tableExists(tableIdent)).as("Table should not exist").isFalse(); + + if (catalogName.equals("testhadoop")) { + // HadoopCatalog drop table without purge will delete the base table location. + assertThat(checkFilesExist(manifestAndFiles, false)) + .as("All files should be deleted") + .isTrue(); + } else { + assertThat(checkFilesExist(manifestAndFiles, true)) + .as("All files should not be deleted") + .isTrue(); + } + } + + @TestTemplate + public void testPurgeTable() throws IOException { + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "test")), + sql("SELECT * FROM %s", tableName)); + + List manifestAndFiles = manifestsAndFiles(); + assertThat(manifestAndFiles).as("There should be 2 files for manifests and files").hasSize(2); + assertThat(checkFilesExist(manifestAndFiles, true)).as("All files should exist").isTrue(); + + sql("DROP TABLE %s PURGE", tableName); + assertThat(validationCatalog.tableExists(tableIdent)).as("Table should not exist").isFalse(); + assertThat(checkFilesExist(manifestAndFiles, false)).as("All files should be deleted").isTrue(); + } + + @TestTemplate + public void testPurgeTableGCDisabled() throws IOException { + sql("ALTER TABLE %s SET TBLPROPERTIES (gc.enabled = false)", tableName); + + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1, "test")), + sql("SELECT * FROM %s", tableName)); + + List manifestAndFiles = manifestsAndFiles(); + assertThat(manifestAndFiles).as("There should be 2 files for manifests and files").hasSize(2); + assertThat(checkFilesExist(manifestAndFiles, true)).as("All files should exist").isTrue(); + + assertThatThrownBy(() -> sql("DROP TABLE %s PURGE", tableName)) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Cannot purge table: GC is disabled (deleting files may corrupt other tables"); + + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not been dropped") + .isTrue(); + assertThat(checkFilesExist(manifestAndFiles, true)) + .as("All files should not be deleted") + .isTrue(); + } + + private List manifestsAndFiles() { + List files = sql("SELECT file_path FROM %s.%s", tableName, MetadataTableType.FILES); + List manifests = + sql("SELECT path FROM %s.%s", tableName, MetadataTableType.MANIFESTS); + return Streams.concat(files.stream(), manifests.stream()) + .map(row -> (String) row[0]) + .collect(Collectors.toList()); + } + + private boolean checkFilesExist(List files, boolean shouldExist) throws IOException { + boolean mask = !shouldExist; + if (files.isEmpty()) { + return mask; + } + + FileSystem fs = new Path(files.get(0)).getFileSystem(hiveConf); + return files.stream() + .allMatch( + file -> { + try { + return fs.exists(new Path(file)) ^ mask; + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java new file mode 100644 index 000000000000..a984c4c826d2 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java @@ -0,0 +1,703 @@ +/* + * 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.sql; + +import static org.apache.iceberg.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.assertj.core.api.Assertions.assertThat; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.sql.Timestamp; +import java.time.Instant; +import java.util.List; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.variants.ShreddedObject; +import org.apache.iceberg.variants.VariantMetadata; +import org.apache.iceberg.variants.Variants; +import org.apache.spark.sql.execution.SparkPlan; +import org.apache.spark.unsafe.types.VariantVal; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestFilterPushDown extends TestBaseWithCatalog { + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, planningMode = {0}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + LOCAL + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + DISTRIBUTED + } + }; + } + + @Parameter(index = 3) + private PlanningMode planningMode; + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS tmp_view"); + } + + @TestTemplate + public void testFilterPushdownWithDecimalValues() { + sql( + "CREATE TABLE %s (id INT, salary DECIMAL(10, 2), dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep)", + tableName); + configurePlanningMode(planningMode); + + sql("INSERT INTO %s VALUES (1, 100.01, 'd1')", tableName); + sql("INSERT INTO %s VALUES (2, 100.05, 'd1')", tableName); + + checkFilters( + "dep = 'd1' AND salary > 100.03" /* query predicate */, + "isnotnull(salary) AND (salary > 100.03)" /* Spark post scan filter */, + "dep IS NOT NULL, salary IS NOT NULL, dep = 'd1', salary > 100.03" /* Iceberg scan filters */, + ImmutableList.of(row(2, new BigDecimal("100.05"), "d1"))); + } + + @TestTemplate + public void testFilterPushdownWithIdentityTransform() { + sql( + "CREATE TABLE %s (id INT, salary INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep)", + tableName); + configurePlanningMode(planningMode); + + sql("INSERT INTO %s VALUES (1, 100, 'd1')", tableName); + sql("INSERT INTO %s VALUES (2, 200, 'd2')", tableName); + sql("INSERT INTO %s VALUES (3, 300, 'd3')", tableName); + sql("INSERT INTO %s VALUES (4, 400, 'd4')", tableName); + sql("INSERT INTO %s VALUES (5, 500, 'd5')", tableName); + sql("INSERT INTO %s VALUES (6, 600, null)", tableName); + + checkOnlyIcebergFilters( + "dep IS NULL" /* query predicate */, + "dep IS NULL" /* Iceberg scan filters */, + ImmutableList.of(row(6, 600, null))); + + checkOnlyIcebergFilters( + "dep IS NOT NULL" /* query predicate */, + "dep IS NOT NULL" /* Iceberg scan filters */, + ImmutableList.of( + row(1, 100, "d1"), + row(2, 200, "d2"), + row(3, 300, "d3"), + row(4, 400, "d4"), + row(5, 500, "d5"))); + + checkOnlyIcebergFilters( + "dep = 'd3'" /* query predicate */, + "dep IS NOT NULL, dep = 'd3'" /* Iceberg scan filters */, + ImmutableList.of(row(3, 300, "d3"))); + + checkOnlyIcebergFilters( + "dep > 'd3'" /* query predicate */, + "dep IS NOT NULL, dep > 'd3'" /* Iceberg scan filters */, + ImmutableList.of(row(4, 400, "d4"), row(5, 500, "d5"))); + + checkOnlyIcebergFilters( + "dep >= 'd5'" /* query predicate */, + "dep IS NOT NULL, dep >= 'd5'" /* Iceberg scan filters */, + ImmutableList.of(row(5, 500, "d5"))); + + checkOnlyIcebergFilters( + "dep < 'd2'" /* query predicate */, + "dep IS NOT NULL, dep < 'd2'" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"))); + + checkOnlyIcebergFilters( + "dep <= 'd2'" /* query predicate */, + "dep IS NOT NULL, dep <= 'd2'" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"), row(2, 200, "d2"))); + + checkOnlyIcebergFilters( + "dep <=> 'd3'" /* query predicate */, + "dep = 'd3'" /* Iceberg scan filters */, + ImmutableList.of(row(3, 300, "d3"))); + + checkOnlyIcebergFilters( + "dep IN (null, 'd1')" /* query predicate */, + "dep IN ('d1')" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"))); + + checkOnlyIcebergFilters( + "dep NOT IN ('d2', 'd4')" /* query predicate */, + "(dep IS NOT NULL AND dep NOT IN ('d2', 'd4'))" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"), row(3, 300, "d3"), row(5, 500, "d5"))); + + checkOnlyIcebergFilters( + "dep = 'd1' AND dep IS NOT NULL" /* query predicate */, + "dep = 'd1', dep IS NOT NULL" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"))); + + checkOnlyIcebergFilters( + "dep = 'd1' OR dep = 'd2' OR dep = 'd3'" /* query predicate */, + "((dep = 'd1' OR dep = 'd2') OR dep = 'd3')" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"), row(2, 200, "d2"), row(3, 300, "d3"))); + + checkFilters( + "dep = 'd1' AND id = 1" /* query predicate */, + "isnotnull(id) AND (id = 1)" /* Spark post scan filter */, + "dep IS NOT NULL, id IS NOT NULL, dep = 'd1', id = 1" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"))); + + checkFilters( + "dep = 'd2' OR id = 1" /* query predicate */, + "(dep = d2) OR (id = 1)" /* Spark post scan filter */, + "(dep = 'd2' OR id = 1)" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"), row(2, 200, "d2"))); + + checkFilters( + "dep LIKE 'd1%' AND id = 1" /* query predicate */, + "isnotnull(id) AND (id = 1)" /* Spark post scan filter */, + "dep IS NOT NULL, id IS NOT NULL, dep LIKE 'd1%', id = 1" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"))); + + checkFilters( + "dep NOT LIKE 'd5%' AND (id = 1 OR id = 5)" /* query predicate */, + "(id = 1) OR (id = 5)" /* Spark post scan filter */, + "dep IS NOT NULL, NOT (dep LIKE 'd5%'), (id = 1 OR id = 5)" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"))); + + checkFilters( + "dep LIKE '%d5' AND id IN (1, 5)" /* query predicate */, + "EndsWith(dep, d5) AND id IN (1,5)" /* Spark post scan filter */, + "dep IS NOT NULL, id IN (1, 5)" /* Iceberg scan filters */, + ImmutableList.of(row(5, 500, "d5"))); + } + + @TestTemplate + public void testFilterPushdownWithHoursTransform() { + sql( + "CREATE TABLE %s (id INT, price INT, t TIMESTAMP)" + + "USING iceberg " + + "PARTITIONED BY (hours(t))", + tableName); + configurePlanningMode(planningMode); + + sql("INSERT INTO %s VALUES (1, 100, TIMESTAMP '2021-06-30T01:00:00.000Z')", tableName); + sql("INSERT INTO %s VALUES (2, 200, TIMESTAMP '2021-06-30T02:00:00.000Z')", tableName); + sql("INSERT INTO %s VALUES (3, 300, null)", tableName); + + withDefaultTimeZone( + "UTC", + () -> { + checkOnlyIcebergFilters( + "t IS NULL" /* query predicate */, + "t IS NULL" /* Iceberg scan filters */, + ImmutableList.of(row(3, 300, null))); + + // strict/inclusive projections for t < TIMESTAMP '2021-06-30T02:00:00.000Z' are equal, + // so this filter selects entire partitions and can be pushed down completely + checkOnlyIcebergFilters( + "t < TIMESTAMP '2021-06-30T02:00:00.000Z'" /* query predicate */, + "t IS NOT NULL, t < 1625018400000000" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, timestamp("2021-06-30T01:00:00.0Z")))); + + // strict/inclusive projections for t < TIMESTAMP '2021-06-30T01:00:00.001Z' differ, + // so this filter does NOT select entire partitions and can't be pushed down completely + checkFilters( + "t < TIMESTAMP '2021-06-30T01:00:00.001Z'" /* query predicate */, + "t < 2021-06-30 01:00:00.001" /* Spark post scan filter */, + "t IS NOT NULL, t < 1625014800001000" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, timestamp("2021-06-30T01:00:00.0Z")))); + + // strict/inclusive projections for t <= TIMESTAMP '2021-06-30T01:00:00.000Z' differ, + // so this filter does NOT select entire partitions and can't be pushed down completely + checkFilters( + "t <= TIMESTAMP '2021-06-30T01:00:00.000Z'" /* query predicate */, + "t <= 2021-06-30 01:00:00" /* Spark post scan filter */, + "t IS NOT NULL, t <= 1625014800000000" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, timestamp("2021-06-30T01:00:00.0Z")))); + }); + } + + @TestTemplate + public void testFilterPushdownWithDaysTransform() { + sql( + "CREATE TABLE %s (id INT, price INT, t TIMESTAMP)" + + "USING iceberg " + + "PARTITIONED BY (days(t))", + tableName); + configurePlanningMode(planningMode); + + sql("INSERT INTO %s VALUES (1, 100, TIMESTAMP '2021-06-15T01:00:00.000Z')", tableName); + sql("INSERT INTO %s VALUES (2, 200, TIMESTAMP '2021-06-30T02:00:00.000Z')", tableName); + sql("INSERT INTO %s VALUES (3, 300, TIMESTAMP '2021-07-15T10:00:00.000Z')", tableName); + sql("INSERT INTO %s VALUES (4, 400, null)", tableName); + + withDefaultTimeZone( + "UTC", + () -> { + checkOnlyIcebergFilters( + "t IS NULL" /* query predicate */, + "t IS NULL" /* Iceberg scan filters */, + ImmutableList.of(row(4, 400, null))); + + // strict/inclusive projections for t < TIMESTAMP '2021-07-05T00:00:00.000Z' are equal, + // so this filter selects entire partitions and can be pushed down completely + checkOnlyIcebergFilters( + "t < TIMESTAMP '2021-07-05T00:00:00.000Z'" /* query predicate */, + "t IS NOT NULL, t < 1625443200000000" /* Iceberg scan filters */, + ImmutableList.of( + row(1, 100, timestamp("2021-06-15T01:00:00.000Z")), + row(2, 200, timestamp("2021-06-30T02:00:00.000Z")))); + + // strict/inclusive projections for t < TIMESTAMP '2021-06-30T03:00:00.000Z' differ, + // so this filter does NOT select entire partitions and can't be pushed down completely + checkFilters( + "t < TIMESTAMP '2021-06-30T03:00:00.000Z'" /* query predicate */, + "t < 2021-06-30 03:00:00" /* Spark post scan filter */, + "t IS NOT NULL, t < 1625022000000000" /* Iceberg scan filters */, + ImmutableList.of( + row(1, 100, timestamp("2021-06-15T01:00:00.000Z")), + row(2, 200, timestamp("2021-06-30T02:00:00.000Z")))); + }); + } + + @TestTemplate + public void testFilterPushdownWithMonthsTransform() { + sql( + "CREATE TABLE %s (id INT, price INT, t TIMESTAMP)" + + "USING iceberg " + + "PARTITIONED BY (months(t))", + tableName); + configurePlanningMode(planningMode); + + sql("INSERT INTO %s VALUES (1, 100, TIMESTAMP '2021-06-30T01:00:00.000Z')", tableName); + sql("INSERT INTO %s VALUES (2, 200, TIMESTAMP '2021-06-30T02:00:00.000Z')", tableName); + sql("INSERT INTO %s VALUES (3, 300, TIMESTAMP '2021-07-15T10:00:00.000Z')", tableName); + sql("INSERT INTO %s VALUES (4, 400, null)", tableName); + + withDefaultTimeZone( + "UTC", + () -> { + checkOnlyIcebergFilters( + "t IS NULL" /* query predicate */, + "t IS NULL" /* Iceberg scan filters */, + ImmutableList.of(row(4, 400, null))); + + // strict/inclusive projections for t < TIMESTAMP '2021-07-01T00:00:00.000Z' are equal, + // so this filter selects entire partitions and can be pushed down completely + checkOnlyIcebergFilters( + "t < TIMESTAMP '2021-07-01T00:00:00.000Z'" /* query predicate */, + "t IS NOT NULL, t < 1625097600000000" /* Iceberg scan filters */, + ImmutableList.of( + row(1, 100, timestamp("2021-06-30T01:00:00.000Z")), + row(2, 200, timestamp("2021-06-30T02:00:00.000Z")))); + + // strict/inclusive projections for t < TIMESTAMP '2021-06-30T03:00:00.000Z' differ, + // so this filter does NOT select entire partitions and can't be pushed down completely + checkFilters( + "t < TIMESTAMP '2021-06-30T03:00:00.000Z'" /* query predicate */, + "t < 2021-06-30 03:00:00" /* Spark post scan filter */, + "t IS NOT NULL, t < 1625022000000000" /* Iceberg scan filters */, + ImmutableList.of( + row(1, 100, timestamp("2021-06-30T01:00:00.000Z")), + row(2, 200, timestamp("2021-06-30T02:00:00.000Z")))); + }); + } + + @TestTemplate + public void testFilterPushdownWithYearsTransform() { + sql( + "CREATE TABLE %s (id INT, price INT, t TIMESTAMP)" + + "USING iceberg " + + "PARTITIONED BY (years(t))", + tableName); + configurePlanningMode(planningMode); + + sql("INSERT INTO %s VALUES (1, 100, TIMESTAMP '2021-06-30T01:00:00.000Z')", tableName); + sql("INSERT INTO %s VALUES (2, 200, TIMESTAMP '2021-06-30T02:00:00.000Z')", tableName); + sql("INSERT INTO %s VALUES (2, 200, TIMESTAMP '2022-09-25T02:00:00.000Z')", tableName); + sql("INSERT INTO %s VALUES (3, 300, null)", tableName); + + withDefaultTimeZone( + "UTC", + () -> { + checkOnlyIcebergFilters( + "t IS NULL" /* query predicate */, + "t IS NULL" /* Iceberg scan filters */, + ImmutableList.of(row(3, 300, null))); + + // strict/inclusive projections for t < TIMESTAMP '2022-01-01T00:00:00.000Z' are equal, + // so this filter selects entire partitions and can be pushed down completely + checkOnlyIcebergFilters( + "t < TIMESTAMP '2022-01-01T00:00:00.000Z'" /* query predicate */, + "t IS NOT NULL, t < 1640995200000000" /* Iceberg scan filters */, + ImmutableList.of( + row(1, 100, timestamp("2021-06-30T01:00:00.000Z")), + row(2, 200, timestamp("2021-06-30T02:00:00.000Z")))); + + // strict/inclusive projections for t < TIMESTAMP '2021-06-30T03:00:00.000Z' differ, + // so this filter does NOT select entire partitions and can't be pushed down completely + checkFilters( + "t < TIMESTAMP '2021-06-30T03:00:00.000Z'" /* query predicate */, + "t < 2021-06-30 03:00:00" /* Spark post scan filter */, + "t IS NOT NULL, t < 1625022000000000" /* Iceberg scan filters */, + ImmutableList.of( + row(1, 100, timestamp("2021-06-30T01:00:00.000Z")), + row(2, 200, timestamp("2021-06-30T02:00:00.000Z")))); + }); + } + + @TestTemplate + public void testFilterPushdownWithBucketTransform() { + sql( + "CREATE TABLE %s (id INT, salary INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep, bucket(8, id))", + tableName); + configurePlanningMode(planningMode); + + sql("INSERT INTO %s VALUES (1, 100, 'd1')", tableName); + sql("INSERT INTO %s VALUES (2, 200, 'd2')", tableName); + + checkFilters( + "dep = 'd1' AND id = 1" /* query predicate */, + "id = 1" /* Spark post scan filter */, + "dep IS NOT NULL, id IS NOT NULL, dep = 'd1'" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"))); + } + + @TestTemplate + public void testFilterPushdownWithTruncateTransform() { + sql( + "CREATE TABLE %s (id INT, salary INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (truncate(1, dep))", + tableName); + configurePlanningMode(planningMode); + + sql("INSERT INTO %s VALUES (1, 100, 'd1')", tableName); + sql("INSERT INTO %s VALUES (2, 200, 'd2')", tableName); + sql("INSERT INTO %s VALUES (3, 300, 'a3')", tableName); + + checkOnlyIcebergFilters( + "dep LIKE 'd%'" /* query predicate */, + "dep IS NOT NULL, dep LIKE 'd%'" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"), row(2, 200, "d2"))); + + checkFilters( + "dep = 'd1'" /* query predicate */, + "dep = d1" /* Spark post scan filter */, + "dep IS NOT NULL" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"))); + } + + @TestTemplate + public void testFilterPushdownWithSpecEvolutionAndIdentityTransforms() { + sql( + "CREATE TABLE %s (id INT, salary INT, dep STRING, sub_dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep)", + tableName); + configurePlanningMode(planningMode); + + sql("INSERT INTO %s VALUES (1, 100, 'd1', 'sd1')", tableName); + + // the filter can be pushed completely because all specs include identity(dep) + checkOnlyIcebergFilters( + "dep = 'd1'" /* query predicate */, + "dep IS NOT NULL, dep = 'd1'" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1", "sd1"))); + + Table table = validationCatalog.loadTable(tableIdent); + + table.updateSpec().addField("sub_dep").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO %s VALUES (2, 200, 'd2', 'sd2')", tableName); + + // the filter can be pushed completely because all specs include identity(dep) + checkOnlyIcebergFilters( + "dep = 'd1'" /* query predicate */, + "dep IS NOT NULL, dep = 'd1'" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1", "sd1"))); + + table.updateSpec().removeField("sub_dep").removeField("dep").commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO %s VALUES (3, 300, 'd3', 'sd3')", tableName); + + // the filter can't be pushed completely because not all specs include identity(dep) + checkFilters( + "dep = 'd1'" /* query predicate */, + "isnotnull(dep) AND (dep = d1)" /* Spark post scan filter */, + "dep IS NOT NULL, dep = 'd1'" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1", "sd1"))); + } + + @TestTemplate + public void testFilterPushdownWithSpecEvolutionAndTruncateTransform() { + sql( + "CREATE TABLE %s (id INT, salary INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (truncate(2, dep))", + tableName); + configurePlanningMode(planningMode); + + sql("INSERT INTO %s VALUES (1, 100, 'd1')", tableName); + + // the filter can be pushed completely because the current spec supports it + checkOnlyIcebergFilters( + "dep LIKE 'd1%'" /* query predicate */, + "dep IS NOT NULL, dep LIKE 'd1%'" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"))); + + Table table = validationCatalog.loadTable(tableIdent); + table + .updateSpec() + .removeField(Expressions.truncate("dep", 2)) + .addField(Expressions.truncate("dep", 1)) + .commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO %s VALUES (2, 200, 'd2')", tableName); + + // the filter can be pushed completely because both specs support it + checkOnlyIcebergFilters( + "dep LIKE 'd%'" /* query predicate */, + "dep IS NOT NULL, dep LIKE 'd%'" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"), row(2, 200, "d2"))); + + // the filter can't be pushed completely because the second spec is truncate(dep, 1) and + // the predicate literal is d1, which is two chars + checkFilters( + "dep LIKE 'd1%' AND id = 1" /* query predicate */, + "(isnotnull(id) AND StartsWith(dep, d1)) AND (id = 1)" /* Spark post scan filter */, + "dep IS NOT NULL, id IS NOT NULL, dep LIKE 'd1%', id = 1" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, "d1"))); + } + + @TestTemplate + public void testFilterPushdownWithSpecEvolutionAndTimeTransforms() { + sql( + "CREATE TABLE %s (id INT, price INT, t TIMESTAMP)" + + "USING iceberg " + + "PARTITIONED BY (hours(t))", + tableName); + configurePlanningMode(planningMode); + + withDefaultTimeZone( + "UTC", + () -> { + sql("INSERT INTO %s VALUES (1, 100, TIMESTAMP '2021-06-30T01:00:00.000Z')", tableName); + + // the filter can be pushed completely because the current spec supports it + checkOnlyIcebergFilters( + "t < TIMESTAMP '2021-07-01T00:00:00.000Z'" /* query predicate */, + "t IS NOT NULL, t < 1625097600000000" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100, timestamp("2021-06-30T01:00:00.000Z")))); + + Table table = validationCatalog.loadTable(tableIdent); + table + .updateSpec() + .removeField(Expressions.hour("t")) + .addField(Expressions.month("t")) + .commit(); + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO %s VALUES (2, 200, TIMESTAMP '2021-05-30T01:00:00.000Z')", tableName); + + // the filter can be pushed completely because both specs support it + checkOnlyIcebergFilters( + "t < TIMESTAMP '2021-06-01T00:00:00.000Z'" /* query predicate */, + "t IS NOT NULL, t < 1622505600000000" /* Iceberg scan filters */, + ImmutableList.of(row(2, 200, timestamp("2021-05-30T01:00:00.000Z")))); + }); + } + + @TestTemplate + public void testFilterPushdownWithSpecialFloatingPointPartitionValues() { + sql( + "CREATE TABLE %s (id INT, salary DOUBLE)" + "USING iceberg " + "PARTITIONED BY (salary)", + tableName); + configurePlanningMode(planningMode); + + sql("INSERT INTO %s VALUES (1, 100.5)", tableName); + sql("INSERT INTO %s VALUES (2, double('NaN'))", tableName); + sql("INSERT INTO %s VALUES (3, double('infinity'))", tableName); + sql("INSERT INTO %s VALUES (4, double('-infinity'))", tableName); + + checkOnlyIcebergFilters( + "salary = 100.5" /* query predicate */, + "salary IS NOT NULL, salary = 100.5" /* Iceberg scan filters */, + ImmutableList.of(row(1, 100.5))); + + checkOnlyIcebergFilters( + "salary = double('NaN')" /* query predicate */, + "salary IS NOT NULL, is_nan(salary)" /* Iceberg scan filters */, + ImmutableList.of(row(2, Double.NaN))); + + checkOnlyIcebergFilters( + "salary != double('NaN')" /* query predicate */, + "salary IS NOT NULL, NOT (is_nan(salary))" /* Iceberg scan filters */, + ImmutableList.of( + row(1, 100.5), row(3, Double.POSITIVE_INFINITY), row(4, Double.NEGATIVE_INFINITY))); + + checkOnlyIcebergFilters( + "salary = double('infinity')" /* query predicate */, + "salary IS NOT NULL, salary = Infinity" /* Iceberg scan filters */, + ImmutableList.of(row(3, Double.POSITIVE_INFINITY))); + + checkOnlyIcebergFilters( + "salary = double('-infinity')" /* query predicate */, + "salary IS NOT NULL, salary = -Infinity" /* Iceberg scan filters */, + ImmutableList.of(row(4, Double.NEGATIVE_INFINITY))); + } + + @TestTemplate + public void testVariantExtractFiltering() { + sql( + "CREATE TABLE %s (id BIGINT, data VARIANT) USING iceberg TBLPROPERTIES" + + "('format-version'='3')", + tableName); + configurePlanningMode(planningMode); + + sql( + "INSERT INTO %s VALUES " + + "(1, parse_json('{\"field\": \"foo\", \"num\": 25}')), " + + "(2, parse_json('{\"field\": \"bar\", \"num\": 30}')), " + + "(3, parse_json('{\"field\": \"baz\", \"num\": 35}')), " + + "(4, null)", + tableName); + + withDefaultTimeZone( + "UTC", + () -> { + checkFilters( + "try_variant_get(data, '$.num', 'int') IS NOT NULL", + "isnotnull(data) AND isnotnull(try_variant_get(data, $.num, IntegerType, false, Some(UTC)))", + "data IS NOT NULL", + ImmutableList.of( + row(1L, toSparkVariantRow("foo", 25)), + row(2L, toSparkVariantRow("bar", 30)), + row(3L, toSparkVariantRow("baz", 35)))); + + checkFilters( + "try_variant_get(data, '$.num', 'int') IS NULL", + "isnull(try_variant_get(data, $.num, IntegerType, false, Some(UTC)))", + "", + ImmutableList.of(row(4L, null))); + + checkFilters( + "try_variant_get(data, '$.num', 'int') > 30", + "isnotnull(data) AND (try_variant_get(data, $.num, IntegerType, false, Some(UTC)) > 30)", + "data IS NOT NULL", + ImmutableList.of(row(3L, toSparkVariantRow("baz", 35)))); + + checkFilters( + "try_variant_get(data, '$.num', 'int') = 30", + "isnotnull(data) AND (try_variant_get(data, $.num, IntegerType, false, Some(UTC)) = 30)", + "data IS NOT NULL", + ImmutableList.of(row(2L, toSparkVariantRow("bar", 30)))); + + checkFilters( + "try_variant_get(data, '$.num', 'int') IN (25, 35)", + "try_variant_get(data, $.num, IntegerType, false, Some(UTC)) IN (25,35)", + "", + ImmutableList.of( + row(1L, toSparkVariantRow("foo", 25)), row(3L, toSparkVariantRow("baz", 35)))); + + checkFilters( + "try_variant_get(data, '$.num', 'int') != 25", + "isnotnull(data) AND NOT (try_variant_get(data, $.num, IntegerType, false, Some(UTC)) = 25)", + "data IS NOT NULL", + ImmutableList.of( + row(2L, toSparkVariantRow("bar", 30)), row(3L, toSparkVariantRow("baz", 35)))); + }); + } + + private void checkOnlyIcebergFilters( + String predicate, String icebergFilters, List expectedRows) { + + checkFilters(predicate, null, icebergFilters, expectedRows); + } + + private void checkFilters( + String predicate, String sparkFilter, String icebergFilters, List expectedRows) { + + Action check = + () -> { + assertEquals( + "Rows must match", + expectedRows, + sql("SELECT * FROM %s WHERE %s ORDER BY id", tableName, predicate)); + }; + SparkPlan sparkPlan = executeAndKeepPlan(check); + String planAsString = sparkPlan.toString().replaceAll("#(\\d+L?)", ""); + + if (sparkFilter != null) { + assertThat(planAsString) + .as("Post scan filter should match") + .containsAnyOf("Filter (" + sparkFilter + ")", "Filter " + sparkFilter); + } else { + assertThat(planAsString).as("Should be no post scan filter").doesNotContain("Filter ("); + } + + assertThat(planAsString) + .as("Pushed filters must match") + .contains("[filters=" + icebergFilters + ","); + } + + private Timestamp timestamp(String timestampAsString) { + return Timestamp.from(Instant.parse(timestampAsString)); + } + + private VariantVal toSparkVariantRow(String field, int num) { + VariantMetadata metadata = Variants.metadata("field", "num"); + + ShreddedObject obj = Variants.object(metadata); + obj.put("field", Variants.of(field)); + obj.put("num", Variants.of(num)); + + ByteBuffer metadataBuffer = + ByteBuffer.allocate(metadata.sizeInBytes()).order(ByteOrder.LITTLE_ENDIAN); + metadata.writeTo(metadataBuffer, 0); + + ByteBuffer valueBuffer = ByteBuffer.allocate(obj.sizeInBytes()).order(ByteOrder.LITTLE_ENDIAN); + obj.writeTo(valueBuffer, 0); + + return new VariantVal( + ByteBuffers.toByteArray(valueBuffer), ByteBuffers.toByteArray(metadataBuffer)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java new file mode 100644 index 000000000000..d702f65e640b --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java @@ -0,0 +1,302 @@ +/* + * 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.sql; + +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.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NamespaceNotEmptyException; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestNamespaceSQL extends CatalogTestBase { + private static final Namespace NS = Namespace.of("db"); + + @Parameter(index = 3) + private String fullNamespace; + + @Parameter(index = 4) + private boolean isHadoopCatalog; + + @Parameters( + name = + "catalogName = {0}, implementation = {1}, config = {2}, fullNameSpace = {3}, isHadoopCatalog = {4}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties(), + SparkCatalogConfig.HIVE.catalogName() + "." + NS.toString(), + false + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + SparkCatalogConfig.HADOOP.catalogName() + "." + NS, + true + }, + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties(), + NS.toString(), + false + } + }; + } + + @AfterEach + public void cleanNamespaces() { + sql("DROP TABLE IF EXISTS %s.table", fullNamespace); + sql("DROP NAMESPACE IF EXISTS %s", fullNamespace); + } + + @TestTemplate + public void testCreateNamespace() { + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); + + sql("CREATE NAMESPACE %s", fullNamespace); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); + } + + @TestTemplate + public void testDefaultNamespace() { + assumeThat(isHadoopCatalog).as("Hadoop has no default namespace configured").isFalse(); + + sql("USE %s", catalogName); + + assertThat(sql("SHOW CURRENT NAMESPACE")) + .singleElement() + .satisfies( + ns -> { + assertThat(ns).containsExactly(catalogName, "default"); + }); + } + + @TestTemplate + public void testDropEmptyNamespace() { + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); + + sql("CREATE NAMESPACE %s", fullNamespace); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); + + sql("DROP NAMESPACE %s", fullNamespace); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should have been dropped") + .isFalse(); + } + + @TestTemplate + public void testDropNonEmptyNamespace() { + assumeThat(catalogName).as("Session catalog has flaky behavior").isNotEqualTo("spark_catalog"); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); + + sql("CREATE NAMESPACE %s", fullNamespace); + sql("CREATE TABLE %s.table (id bigint) USING iceberg", fullNamespace); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); + assertThat(validationCatalog.tableExists(TableIdentifier.of(NS, "table"))) + .as("Table should exist") + .isTrue(); + + assertThatThrownBy(() -> sql("DROP NAMESPACE %s", fullNamespace)) + .isInstanceOf(NamespaceNotEmptyException.class) + .hasMessageStartingWith("Namespace db is not empty."); + + sql("DROP TABLE %s.table", fullNamespace); + } + + @TestTemplate + public void testListTables() { + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); + + sql("CREATE NAMESPACE %s", fullNamespace); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); + + List rows = sql("SHOW TABLES IN %s", fullNamespace); + assertThat(rows).as("Should not list any tables").isEmpty(); + + sql("CREATE TABLE %s.table (id bigint) USING iceberg", fullNamespace); + + assertThat(sql("SHOW TABLES IN %s", fullNamespace)) + .singleElement() + .satisfies( + row -> { + assertThat(row).containsExactly("db", "table", false); + }); + } + + @TestTemplate + public void testListNamespace() { + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); + + sql("CREATE NAMESPACE %s", fullNamespace); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); + + List namespaces = sql("SHOW NAMESPACES IN %s", catalogName); + + if (isHadoopCatalog) { + assertThat(namespaces) + .singleElement() + .satisfies( + ns -> { + assertThat(ns).containsExactly("db"); + }); + } else { + assertThat(namespaces).as("Should have 2 namespaces").hasSize(2); + Set namespaceNames = + namespaces.stream().map(arr -> arr[0].toString()).collect(Collectors.toSet()); + assertThat(namespaceNames) + .as("Should have default and db namespaces") + .containsExactlyInAnyOrder("default", "db"); + } + + List nestedNamespaces = sql("SHOW NAMESPACES IN %s", fullNamespace); + assertThat(nestedNamespaces).as("Should not have nested namespaces").isEmpty(); + } + + @TestTemplate + public void testCreateNamespaceWithMetadata() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); + + sql("CREATE NAMESPACE %s WITH PROPERTIES ('prop'='value')", fullNamespace); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); + + Map nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(NS); + + assertThat(nsMetadata).containsEntry("prop", "value"); + } + + @TestTemplate + public void testCreateNamespaceWithComment() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); + + sql("CREATE NAMESPACE %s COMMENT 'namespace doc'", fullNamespace); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); + + Map nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(NS); + + assertThat(nsMetadata).containsEntry("comment", "namespace doc"); + } + + @TestTemplate + public void testCreateNamespaceWithLocation() throws Exception { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); + + File location = File.createTempFile("junit", null, temp.toFile()); + assertThat(location.delete()).isTrue(); + + sql("CREATE NAMESPACE %s LOCATION '%s'", fullNamespace, location); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); + + Map nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(NS); + + assertThat(nsMetadata).containsEntry("location", "file:" + location.getPath()); + } + + @TestTemplate + public void testSetProperties() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should not already exist") + .isFalse(); + + sql("CREATE NAMESPACE %s", fullNamespace); + + assertThat(validationNamespaceCatalog.namespaceExists(NS)) + .as("Namespace should exist") + .isTrue(); + + Map defaultMetadata = validationNamespaceCatalog.loadNamespaceMetadata(NS); + assertThat(defaultMetadata) + .as("Default metadata should not have custom property") + .doesNotContainKey("prop"); + + sql("ALTER NAMESPACE %s SET PROPERTIES ('prop'='value')", fullNamespace); + + Map nsMetadata = validationNamespaceCatalog.loadNamespaceMetadata(NS); + + assertThat(nsMetadata).containsEntry("prop", "value"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java new file mode 100644 index 000000000000..800d17dd4559 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWrites.java @@ -0,0 +1,21 @@ +/* + * 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.sql; + +public class TestPartitionedWrites extends PartitionedWritesTestBase {} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesAsSelect.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesAsSelect.java new file mode 100644 index 000000000000..fde0f0a39a9f --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesAsSelect.java @@ -0,0 +1,158 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.stream.IntStream; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.spark.IcebergSpark; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.spark.sql.types.DataTypes; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestPartitionedWritesAsSelect extends TestBaseWithCatalog { + + @Parameter(index = 3) + private String targetTable; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, targetTable = {3}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + SparkCatalogConfig.HADOOP.catalogName() + ".default.target_table" + }, + }; + } + + @BeforeEach + public void createTables() { + sql( + "CREATE TABLE %s (id bigint, data string, category string, ts timestamp) USING iceberg", + tableName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s", targetTable); + } + + @TestTemplate + public void testInsertAsSelectAppend() { + insertData(3); + List expected = currentData(); + + sql( + "CREATE TABLE %s (id bigint, data string, category string, ts timestamp)" + + "USING iceberg PARTITIONED BY (days(ts), category)", + targetTable); + + sql( + "INSERT INTO %s SELECT id, data, category, ts FROM %s ORDER BY ts,category", + targetTable, tableName); + assertThat(scalarSql("SELECT count(*) FROM %s", targetTable)) + .as("Should have 15 rows after insert") + .isEqualTo(3 * 5L); + + assertEquals( + "Row data should match expected", + expected, + sql("SELECT * FROM %s ORDER BY id", targetTable)); + } + + @TestTemplate + public void testInsertAsSelectWithBucket() { + insertData(3); + List expected = currentData(); + + sql( + "CREATE TABLE %s (id bigint, data string, category string, ts timestamp)" + + "USING iceberg PARTITIONED BY (bucket(8, data))", + targetTable); + + IcebergSpark.registerBucketUDF(spark, "iceberg_bucket8", DataTypes.StringType, 8); + sql( + "INSERT INTO %s SELECT id, data, category, ts FROM %s ORDER BY iceberg_bucket8(data)", + targetTable, tableName); + assertThat(scalarSql("SELECT count(*) FROM %s", targetTable)) + .as("Should have 15 rows after insert") + .isEqualTo(3 * 5L); + + assertEquals( + "Row data should match expected", + expected, + sql("SELECT * FROM %s ORDER BY id", targetTable)); + } + + @TestTemplate + public void testInsertAsSelectWithTruncate() { + insertData(3); + List expected = currentData(); + + sql( + "CREATE TABLE %s (id bigint, data string, category string, ts timestamp)" + + "USING iceberg PARTITIONED BY (truncate(data, 4), truncate(id, 4))", + targetTable); + + IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_string4", DataTypes.StringType, 4); + IcebergSpark.registerTruncateUDF(spark, "iceberg_truncate_long4", DataTypes.LongType, 4); + sql( + "INSERT INTO %s SELECT id, data, category, ts FROM %s " + + "ORDER BY iceberg_truncate_string4(data),iceberg_truncate_long4(id)", + targetTable, tableName); + assertThat(scalarSql("SELECT count(*) FROM %s", targetTable)) + .as("Should have 15 rows after insert") + .isEqualTo(3 * 5L); + + assertEquals( + "Row data should match expected", + expected, + sql("SELECT * FROM %s ORDER BY id", targetTable)); + } + + private void insertData(int repeatCounter) { + IntStream.range(0, repeatCounter) + .forEach( + i -> { + sql( + "INSERT INTO %s VALUES (13, '1', 'bgd16', timestamp('2021-11-10 11:20:10'))," + + "(21, '2', 'bgd13', timestamp('2021-11-10 11:20:10')), " + + "(12, '3', 'bgd14', timestamp('2021-11-10 11:20:10'))," + + "(222, '3', 'bgd15', timestamp('2021-11-10 11:20:10'))," + + "(45, '4', 'bgd16', timestamp('2021-11-10 11:20:10'))", + tableName); + }); + } + + private List currentData() { + return rowsToJava(spark.sql("SELECT * FROM " + tableName + " order by id").collectAsList()); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToBranch.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToBranch.java new file mode 100644 index 000000000000..0a90d3ffec75 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToBranch.java @@ -0,0 +1,49 @@ +/* + * 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.sql; + +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestPartitionedWritesToBranch extends PartitionedWritesTestBase { + + private static final String BRANCH = "test"; + + @BeforeEach + @Override + public void createTables() { + super.createTables(); + Table table = validationCatalog.loadTable(tableIdent); + table.manageSnapshots().createBranch(BRANCH, table.currentSnapshot().snapshotId()).commit(); + sql("REFRESH TABLE " + tableName); + } + + @Override + protected String commitTarget() { + return String.format("%s.branch_%s", tableName, BRANCH); + } + + @Override + protected String selectTarget() { + return String.format("%s VERSION AS OF '%s'", tableName, BRANCH); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java new file mode 100644 index 000000000000..45268b78f893 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java @@ -0,0 +1,91 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import java.util.UUID; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.spark.SparkSQLProperties; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestPartitionedWritesToWapBranch extends PartitionedWritesTestBase { + + private static final String BRANCH = "test"; + + @BeforeEach + @Override + public void createTables() { + spark.conf().set(SparkSQLProperties.WAP_BRANCH, BRANCH); + sql( + "CREATE TABLE %s (id bigint, data string) USING iceberg PARTITIONED BY (truncate(id, 3)) OPTIONS (%s = 'true')", + tableName, TableProperties.WRITE_AUDIT_PUBLISH_ENABLED); + sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); + } + + @AfterEach + @Override + public void removeTables() { + super.removeTables(); + spark.conf().unset(SparkSQLProperties.WAP_BRANCH); + spark.conf().unset(SparkSQLProperties.WAP_ID); + } + + @Override + protected String selectTarget() { + return String.format("%s VERSION AS OF '%s'", tableName, BRANCH); + } + + @TestTemplate + public void testBranchAndWapBranchCannotBothBeSetForWrite() { + Table table = validationCatalog.loadTable(tableIdent); + table.manageSnapshots().createBranch("test2", table.refs().get(BRANCH).snapshotId()).commit(); + sql("REFRESH TABLE " + tableName); + assertThatThrownBy(() -> sql("INSERT INTO %s.branch_test2 VALUES (4, 'd')", tableName)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Cannot write to both branch and WAP branch, but got branch [test2] and WAP branch [%s]", + BRANCH); + } + + @TestTemplate + public void testWapIdAndWapBranchCannotBothBeSetForWrite() { + String wapId = UUID.randomUUID().toString(); + spark.conf().set(SparkSQLProperties.WAP_ID, wapId); + assertThatThrownBy(() -> sql("INSERT INTO %s VALUES (4, 'd')", tableName)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Cannot set both WAP ID and branch, but got ID [%s] and branch [%s]", wapId, BRANCH); + } + + @Override + protected void assertPartitionMetadata( + String tableName, List expected, String... selectPartitionColumns) { + // Cannot read from the .partitions table newly written data into the WAP branch. See + // https://github.com/apache/iceberg/issues/7297 for more details. + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java new file mode 100644 index 000000000000..6becc07b2db2 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestRefreshTable.java @@ -0,0 +1,86 @@ +/* + * 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.sql; + +import java.util.List; +import java.util.Set; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRefreshTable extends CatalogTestBase { + + @BeforeEach + public void createTables() { + sql("CREATE TABLE %s (key int, value int) USING iceberg", tableName); + sql("INSERT INTO %s VALUES (1,1)", tableName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testRefreshCommand() { + // We are not allowed to change the session catalog after it has been initialized, so build a + // new one + if (Set.of( + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.REST.catalogName()) + .contains(catalogName)) { + spark.conf().set("spark.sql.catalog." + catalogName + ".cache-enabled", true); + + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + spark = ((org.apache.spark.sql.classic.SparkSession) spark).cloneSession(); + } + + List originalExpected = ImmutableList.of(row(1, 1)); + List originalActual = sql("SELECT * FROM %s", tableName); + assertEquals("Table should start as expected", originalExpected, originalActual); + + // Modify table outside of spark, it should be cached so Spark should see the same value after + // mutation + Table table = validationCatalog.loadTable(tableIdent); + DataFile file = table.currentSnapshot().addedDataFiles(table.io()).iterator().next(); + table.newDelete().deleteFile(file).commit(); + + List cachedActual = sql("SELECT * FROM %s", tableName); + assertEquals("Cached table should be unchanged", originalExpected, cachedActual); + + // Refresh the Spark catalog, should be empty + sql("REFRESH TABLE %s", tableName); + List refreshedExpected = ImmutableList.of(); + List refreshedActual = sql("SELECT * FROM %s", tableName); + assertEquals("Refreshed table should be empty", refreshedExpected, refreshedActual); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java new file mode 100644 index 000000000000..8b146d925bca --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -0,0 +1,734 @@ +/* + * 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.sql; + +import static org.apache.iceberg.TableProperties.SPLIT_SIZE; +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.sql.Timestamp; +import java.text.SimpleDateFormat; +import java.util.Arrays; +import java.util.Date; +import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Table; +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; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSelect extends CatalogTestBase { + private int scanEventCount = 0; + private ScanEvent lastScanEvent = null; + + @Parameter(index = 3) + private String binaryTableName; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, binaryTableName = {3}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties(), + SparkCatalogConfig.HIVE.catalogName() + ".default.binary_table" + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + SparkCatalogConfig.HADOOP.catalogName() + ".default.binary_table" + }, + { + SparkCatalogConfig.SPARK_SESSION.catalogName(), + SparkCatalogConfig.SPARK_SESSION.implementation(), + SparkCatalogConfig.SPARK_SESSION.properties(), + "default.binary_table" + } + }; + } + + @BeforeEach + public void createTables() { + // register a scan event listener to validate pushdown + Listeners.register( + event -> { + scanEventCount += 1; + lastScanEvent = event; + }, + ScanEvent.class); + + sql("CREATE TABLE %s (id bigint, data string, float float) USING iceberg", tableName); + sql("INSERT INTO %s VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', float('NaN'))", tableName); + + this.scanEventCount = 0; + this.lastScanEvent = null; + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s", binaryTableName); + } + + @TestTemplate + public void testSelect() { + List expected = + ImmutableList.of(row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", Float.NaN)); + + assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); + } + + @TestTemplate + public void testSelectWithSpecifiedTargetSplitSize() { + List expected = + ImmutableList.of(row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", Float.NaN)); + + Table table = validationCatalog.loadTable(tableIdent); + table.updateProperties().set("read.split.target-size", "1024").commit(); + spark.sql("REFRESH TABLE " + tableName); + assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); + + // Query failed when `SPLIT_SIZE` < 0 + table.updateProperties().set(SPLIT_SIZE, "-1").commit(); + spark.sql("REFRESH TABLE " + tableName); + assertThatThrownBy(() -> sql("SELECT * FROM %s", tableName)) + .hasMessageContaining("Split size must be > 0: -1") + .isInstanceOf(IllegalArgumentException.class); + + // Query failed when `SPLIT_SIZE` == 0 + table.updateProperties().set(SPLIT_SIZE, "0").commit(); + spark.sql("REFRESH TABLE " + tableName); + assertThatThrownBy(() -> sql("SELECT * FROM %s", tableName)) + .hasMessageContaining("Split size must be > 0: 0") + .isInstanceOf(IllegalArgumentException.class); + } + + @TestTemplate + public void testSelectRewrite() { + List expected = ImmutableList.of(row(3L, "c", Float.NaN)); + + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT * FROM %s where float = float('NaN')", tableName)); + + assertThat(scanEventCount).as("Should create only one scan").isEqualTo(1); + assertThat(Spark3Util.describe(lastScanEvent.filter())) + .as("Should push down expected filter") + .isEqualTo("(float IS NOT NULL AND is_nan(float))"); + } + + @TestTemplate + public void selectWithLimit() { + Object[] first = row(1L, "a", 1.0F); + Object[] second = row(2L, "b", 2.0F); + Object[] third = row(3L, "c", Float.NaN); + + // verify that LIMIT is properly applied in case SupportsPushDownLimit.isPartiallyPushed() is + // ever overridden in SparkScanBuilder + assertThat(sql("SELECT * FROM %s LIMIT 1", tableName)).containsExactly(first); + assertThat(sql("SELECT * FROM %s LIMIT 2", tableName)).containsExactly(first, second); + assertThat(sql("SELECT * FROM %s LIMIT 3", tableName)).containsExactly(first, second, third); + } + + @TestTemplate + public void testProjection() { + List expected = ImmutableList.of(row(1L), row(2L), row(3L)); + + assertEquals("Should return all expected rows", expected, sql("SELECT id FROM %s", tableName)); + + assertThat(scanEventCount).as("Should create only one scan").isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); + assertThat(lastScanEvent.projection().asStruct()) + .as("Should project only the id column") + .isEqualTo(validationCatalog.loadTable(tableIdent).schema().select("id").asStruct()); + } + + @TestTemplate + public void testExpressionPushdown() { + List expected = ImmutableList.of(row("b")); + + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT data FROM %s WHERE id = 2", tableName)); + + assertThat(scanEventCount).as("Should create only one scan").isEqualTo(1); + assertThat(Spark3Util.describe(lastScanEvent.filter())) + .as("Should push down expected filter") + .isEqualTo("(id IS NOT NULL AND id = 2)"); + assertThat(lastScanEvent.projection().asStruct()) + .as("Should project only id and data columns") + .isEqualTo( + validationCatalog.loadTable(tableIdent).schema().select("id", "data").asStruct()); + } + + @TestTemplate + public void testMetadataTables() { + assertEquals( + "Snapshot metadata table", + ImmutableList.of(row(ANY, ANY, null, "append", ANY, ANY)), + sql("SELECT * FROM %s.snapshots", tableName)); + } + + @TestTemplate + public void testSnapshotInTableName() { + // get the snapshot ID of the last write and get the current row set as expected + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + List expected = sql("SELECT * FROM %s ORDER by id", tableName); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + String prefix = "snapshot_id_"; + // read the table at the snapshot + List actual = sql("SELECT * FROM %s.%s ORDER by id", tableName, prefix + snapshotId); + assertEquals("Snapshot at specific ID, prefix " + prefix, expected, actual); + + // read the table using DataFrameReader option + Dataset df = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SNAPSHOT_ID, snapshotId) + .load(tableName) + .orderBy("id"); + List fromDF = rowsToJava(df.collectAsList()); + assertEquals("Snapshot at specific ID " + snapshotId, expected, fromDF); + } + + @TestTemplate + 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); + List expected = sql("SELECT * FROM %s ORDER by id", tableName); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + String prefix = "at_timestamp_"; + // read the table at the snapshot + List actual = sql("SELECT * FROM %s.%s ORDER by id", tableName, prefix + timestamp); + assertEquals("Snapshot at timestamp, prefix " + prefix, expected, actual); + + // read the table using DataFrameReader option + Dataset df = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) + .load(tableName) + .orderBy("id"); + List fromDF = rowsToJava(df.collectAsList()); + assertEquals("Snapshot at timestamp " + timestamp, expected, fromDF); + } + + @TestTemplate + public void testVersionAsOf() { + // get the snapshot ID of the last write and get the current row set as expected + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + // read the table at the snapshot + List actual1 = + sql("SELECT * FROM %s VERSION AS OF %s ORDER BY id", tableName, snapshotId); + assertEquals("Snapshot at specific ID", expected, actual1); + + // read the table at the snapshot + // HIVE time travel syntax + List actual2 = + sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF %s ORDER BY id", tableName, snapshotId); + assertEquals("Snapshot at specific ID", expected, actual2); + + // read the table using DataFrameReader option: versionAsOf + Dataset df = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VERSION_AS_OF, snapshotId) + .load(tableName) + .orderBy("id"); + List fromDF = rowsToJava(df.collectAsList()); + assertEquals("Snapshot at specific ID " + snapshotId, expected, fromDF); + } + + @TestTemplate + public void testTagReference() { + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createTag("test_tag", snapshotId).commit(); + List expected = sql("SELECT * FROM %s ORDER by id", tableName); + + // create a second snapshot, read the table at the tag + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + List actual1 = + sql("SELECT * FROM %s VERSION AS OF 'test_tag' ORDER by id", tableName); + assertEquals("Snapshot at specific tag reference name", expected, actual1); + + // read the table at the tag + // HIVE time travel syntax + List actual2 = + sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_tag' ORDER by id", tableName); + assertEquals("Snapshot at specific tag reference name", expected, actual2); + + // Spark session catalog does not support extended table names + if (!"spark_catalog".equals(catalogName)) { + // read the table using the "tag_" prefix in the table name + List actual3 = sql("SELECT * FROM %s.tag_test_tag ORDER by id", tableName); + assertEquals("Snapshot at specific tag reference name, prefix", expected, actual3); + } + + // read the table using DataFrameReader option: tag + Dataset df = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.TAG, "test_tag") + .load(tableName) + .orderBy("id"); + List fromDF = rowsToJava(df.collectAsList()); + assertEquals("Snapshot at specific tag reference name", expected, fromDF); + } + + @TestTemplate + public void testUseSnapshotIdForTagReferenceAsOf() { + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId1 = table.currentSnapshot().snapshotId(); + + // create a second snapshot, read the table at the snapshot + List actual = sql("SELECT * FROM %s ORDER by id", tableName); + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + table.refresh(); + long snapshotId2 = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createTag(Long.toString(snapshotId1), snapshotId2).commit(); + + // currently Spark version travel ignores the type of the AS OF + // this means if a tag name matches a snapshot ID, it will always choose snapshotID to travel + // to. + List travelWithStringResult = + sql("SELECT * FROM %s VERSION AS OF '%s' ORDER by id", tableName, snapshotId1); + assertEquals("Snapshot at specific tag reference name", actual, travelWithStringResult); + + List travelWithLongResult = + sql("SELECT * FROM %s VERSION AS OF %s ORDER by id", tableName, snapshotId1); + assertEquals("Snapshot at specific tag reference name", actual, travelWithLongResult); + } + + @TestTemplate + public void testBranchReference() { + Table table = validationCatalog.loadTable(tableIdent); + long snapshotId = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createBranch("test_branch", snapshotId).commit(); + List expected = sql("SELECT * FROM %s ORDER by id", tableName); + + // create a second snapshot, read the table at the branch + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + List actual1 = + sql("SELECT * FROM %s VERSION AS OF 'test_branch' ORDER by id", tableName); + assertEquals("Snapshot at specific branch reference name", expected, actual1); + + // read the table at the branch + // HIVE time travel syntax + List actual2 = + sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_branch' ORDER by id", tableName); + assertEquals("Snapshot at specific branch reference name", expected, actual2); + + // Spark session catalog does not support extended table names + if (!"spark_catalog".equals(catalogName)) { + // read the table using the "branch_" prefix in the table name + List actual3 = sql("SELECT * FROM %s.branch_test_branch ORDER by id", tableName); + assertEquals("Snapshot at specific branch reference name, prefix", expected, actual3); + } + + // read the table using DataFrameReader option: branch + Dataset df = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.BRANCH, "test_branch") + .load(tableName) + .orderBy("id"); + List fromDF = rowsToJava(df.collectAsList()); + assertEquals("Snapshot at specific branch reference name", expected, fromDF); + } + + @TestTemplate + public void readAndWriteWithBranchAfterSchemaChange() { + Table table = validationCatalog.loadTable(tableIdent); + String branchName = "test_branch"; + table.manageSnapshots().createBranch(branchName, table.currentSnapshot().snapshotId()).commit(); + + List expected = + Arrays.asList(row(1L, "a", 1.0f), row(2L, "b", 2.0f), row(3L, "c", Float.NaN)); + assertThat(sql("SELECT * FROM %s", tableName)).containsExactlyElementsOf(expected); + + // change schema on the table and add more data + sql("ALTER TABLE %s DROP COLUMN float", tableName); + sql("ALTER TABLE %s ADD COLUMN new_col date", tableName); + sql( + "INSERT INTO %s VALUES (4, 'd', date('2024-04-04')), (5, 'e', date('2024-05-05'))", + tableName); + + // time-travel query using snapshot id should return the snapshot's schema + long branchSnapshotId = table.refs().get(branchName).snapshotId(); + assertThat(sql("SELECT * FROM %s VERSION AS OF %s ORDER by id", tableName, branchSnapshotId)) + .containsExactlyElementsOf(expected); + + // querying the head of the branch should return the table's schema + assertThat(sql("SELECT * FROM %s VERSION AS OF '%s' ORDER by id", tableName, branchName)) + .containsExactly(row(1L, "a", null), row(2L, "b", null), row(3L, "c", null)); + + if (!"spark_catalog".equals(catalogName)) { + // querying the head of the branch using 'branch_' should return the table's schema + assertThat(sql("SELECT * FROM %s.branch_%s ORDER by id", tableName, branchName)) + .containsExactly(row(1L, "a", null), row(2L, "b", null), row(3L, "c", null)); + } + + // writing to a branch uses the table's schema + sql( + "INSERT INTO %s.branch_%s VALUES (6L, 'f', cast('2023-06-06' as date)), (7L, 'g', cast('2023-07-07' as date))", + tableName, branchName); + + // querying the head of the branch returns the table's schema + assertThat(sql("SELECT * FROM %s VERSION AS OF '%s' ORDER by id", tableName, branchName)) + .containsExactlyInAnyOrder( + row(1L, "a", null), + row(2L, "b", null), + row(3L, "c", null), + row(6L, "f", java.sql.Date.valueOf("2023-06-06")), + row(7L, "g", java.sql.Date.valueOf("2023-07-07"))); + + // using DataFrameReader with the 'branch' option should return the table's schema + Dataset df = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.BRANCH, branchName) + .load(tableName) + .orderBy("id"); + assertThat(rowsToJava(df.collectAsList())) + .containsExactlyInAnyOrder( + row(1L, "a", null), + row(2L, "b", null), + row(3L, "c", null), + row(6L, "f", java.sql.Date.valueOf("2023-06-06")), + row(7L, "g", java.sql.Date.valueOf("2023-07-07"))); + } + + @TestTemplate + 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); + } + + @TestTemplate + public void testTimestampAsOf() { + long snapshotTs = validationCatalog.loadTable(tableIdent).currentSnapshot().timestampMillis(); + long timestamp = waitUntilAfter(snapshotTs + 1000); + waitUntilAfter(timestamp + 1000); + // AS OF expects the timestamp if given in long format will be of seconds precision + long timestampInSeconds = TimeUnit.MILLISECONDS.toSeconds(timestamp); + SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); + String formattedDate = sdf.format(new Date(timestamp)); + + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + // read the table at the timestamp in long format i.e 1656507980463. + List actualWithLongFormat = + sql("SELECT * FROM %s TIMESTAMP AS OF %s ORDER BY id", tableName, timestampInSeconds); + assertEquals("Snapshot at timestamp", expected, actualWithLongFormat); + + // read the table at the timestamp in date format i.e 2022-06-29 18:40:37 + List actualWithDateFormat = + sql("SELECT * FROM %s TIMESTAMP AS OF '%s' ORDER BY id", tableName, formattedDate); + assertEquals("Snapshot at timestamp", expected, actualWithDateFormat); + + // HIVE time travel syntax + // read the table at the timestamp in long format i.e 1656507980463. + List actualWithLongFormatInHiveSyntax = + sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF %s ORDER BY id", tableName, timestampInSeconds); + assertEquals("Snapshot at specific ID", expected, actualWithLongFormatInHiveSyntax); + + // read the table at the timestamp in date format i.e 2022-06-29 18:40:37 + List actualWithDateFormatInHiveSyntax = + sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF '%s' ORDER BY id", tableName, formattedDate); + assertEquals("Snapshot at specific ID", expected, actualWithDateFormatInHiveSyntax); + + // read the table using DataFrameReader option + Dataset df = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.TIMESTAMP_AS_OF, formattedDate) + .load(tableName) + .orderBy("id"); + List fromDF = rowsToJava(df.collectAsList()); + assertEquals("Snapshot at timestamp " + timestamp, expected, fromDF); + } + + @TestTemplate + public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { + // get the snapshot ID of the last write + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + // get a timestamp just after the last write + long timestamp = + validationCatalog.loadTable(tableIdent).currentSnapshot().timestampMillis() + 2; + + String timestampPrefix = "at_timestamp_"; + String snapshotPrefix = "snapshot_id_"; + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + // using snapshot in table identifier and VERSION AS OF + assertThatThrownBy( + () -> { + sql( + "SELECT * FROM %s.%s VERSION AS OF %s", + tableName, snapshotPrefix + snapshotId, snapshotId); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + + // using snapshot in table identifier and TIMESTAMP AS OF + assertThatThrownBy( + () -> { + sql( + "SELECT * FROM %s.%s VERSION AS OF %s", + tableName, timestampPrefix + timestamp, snapshotId); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + + // using timestamp in table identifier and VERSION AS OF + assertThatThrownBy( + () -> { + sql( + "SELECT * FROM %s.%s TIMESTAMP AS OF %s", + tableName, snapshotPrefix + snapshotId, timestamp); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + + // using timestamp in table identifier and TIMESTAMP AS OF + assertThatThrownBy( + () -> { + sql( + "SELECT * FROM %s.%s TIMESTAMP AS OF %s", + tableName, timestampPrefix + timestamp, timestamp); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + } + + @TestTemplate + public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + validationCatalog.loadTable(tableIdent).manageSnapshots().createBranch("b1").commit(); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + // using branch_b1 in the table identifier and VERSION AS OF + 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"); + + // 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"); + } + + @TestTemplate + public void testSpecifySnapshotAndTimestamp() { + // get the snapshot ID of the last write + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + // get a timestamp just after the last write + long timestamp = + validationCatalog.loadTable(tableIdent).currentSnapshot().timestampMillis() + 2; + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + assertThatThrownBy( + () -> { + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SNAPSHOT_ID, snapshotId) + .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) + .load(tableName) + .collectAsList(); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + String.format( + "Can specify only one of snapshot-id (%s), as-of-timestamp (%s)", + snapshotId, timestamp)); + } + + @TestTemplate + public void testBinaryInFilter() { + sql("CREATE TABLE %s (id bigint, binary binary) USING iceberg", binaryTableName); + sql("INSERT INTO %s VALUES (1, X''), (2, X'1111'), (3, X'11')", binaryTableName); + List expected = ImmutableList.of(row(2L, new byte[] {0x11, 0x11})); + + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT id, binary FROM %s where binary > X'11'", binaryTableName)); + } + + @TestTemplate + public void testComplexTypeFilter() { + String complexTypeTableName = tableName("complex_table"); + sql( + "CREATE TABLE %s (id INT, complex STRUCT) USING iceberg", + complexTypeTableName); + sql( + "INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", 3, \"c2\", \"v1\"))", + complexTypeTableName); + sql( + "INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", 2, \"c2\", \"v2\"))", + complexTypeTableName); + + List result = + sql( + "SELECT id FROM %s WHERE complex = named_struct(\"c1\", 3, \"c2\", \"v1\")", + complexTypeTableName); + + assertEquals("Should return all expected rows", ImmutableList.of(row(1)), result); + sql("DROP TABLE IF EXISTS %s", complexTypeTableName); + } + + @TestTemplate + public void testRequiredNestedFieldInOptionalStructFilter() { + String nestedStructTable = tableName("nested_struct_table"); + sql( + "CREATE TABLE %s (id INT NOT NULL, address STRUCT) " + + "USING iceberg", + nestedStructTable); + sql("INSERT INTO %s VALUES (0, NULL)", nestedStructTable); + sql("INSERT INTO %s VALUES (1, STRUCT('123 Main St'))", nestedStructTable); + + List result = + sql("SELECT id FROM %s WHERE address.street IS NULL", nestedStructTable); + + assertEquals("Should return all expected rows", ImmutableList.of(row(0)), result); + sql("DROP TABLE IF EXISTS %s", nestedStructTable); + } + + @TestTemplate + public void simpleTypesInFilter() { + String tableName = tableName("simple_types_table"); + sql( + "CREATE TABLE IF NOT EXISTS %s (id bigint, boolean boolean, integer integer, long long, " + + "float float, double double, string string, date date, timestamp timestamp) USING iceberg", + tableName); + sql( + "INSERT INTO %s VALUES (1, true, 1, 1L, 1.1, 1.3, '1.5', to_date('2021-01-01'), to_timestamp('2021-01-01T00:00:00')), " + + "(2, false, 2, 2L, 2.2, 2.4, '2.6', to_date('2022-02-02'), to_timestamp('2022-02-02T00:00:00')), " + + "(3, true, 3, 3L, 3.3, 3.6, '3.9', to_date('2023-03-03'), to_timestamp('2023-03-03T00:00:00'))", + tableName); + assertThat(sql("SELECT id FROM %s where id > 1", tableName)).containsExactly(row(2L), row(3L)); + assertThat(sql("SELECT id, boolean FROM %s where boolean = true", tableName)) + .containsExactly(row(1L, true), row(3L, true)); + assertThat(sql("SELECT long FROM %s where long > 1", tableName)) + .containsExactly(row(2L), row(3L)); + assertThat(sql("SELECT float FROM %s where float > 1.1f", tableName)) + .containsExactly(row(2.2f), row(3.3f)); + assertThat(sql("SELECT double FROM %s where double > 1.3", tableName)) + .containsExactly(row(2.4d), row(3.6d)); + assertThat(sql("SELECT string FROM %s where string > '1.5'", tableName)) + .containsExactly(row("2.6"), row("3.9")); + java.sql.Date dateOne = java.sql.Date.valueOf("2022-02-02"); + java.sql.Date dateTwo = java.sql.Date.valueOf("2023-03-03"); + assertThat(sql("SELECT date FROM %s where date > to_date('2021-01-01')", tableName)) + .containsExactly(row(dateOne), row(dateTwo)); + assertThat( + sql("SELECT timestamp FROM %s where timestamp > to_timestamp('2021-01-01')", tableName)) + .containsExactly( + row(new Timestamp(dateOne.getTime())), row(new Timestamp(dateTwo.getTime()))); + + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void variantTypeInFilter() { + assumeThat(validationCatalog) + .as("Variant is not supported in Hive catalog") + .isNotInstanceOf(HiveCatalog.class); + + String tableName = tableName("variant_table"); + sql( + "CREATE TABLE %s (id BIGINT, v1 VARIANT, v2 VARIANT) USING iceberg TBLPROPERTIES ('format-version'='3')", + tableName); + + String v1r1 = "{\"a\":5}"; + String v1r2 = "{\"a\":10}"; + String v2r1 = "{\"x\":15}"; + String v2r2 = "{\"x\":20}"; + + sql("INSERT INTO %s SELECT 1, parse_json('%s'), parse_json('%s')", tableName, v1r1, v2r1); + sql("INSERT INTO %s SELECT 2, parse_json('%s'), parse_json('%s')", tableName, v1r2, v2r2); + + assertThat( + sql( + "SELECT id, try_variant_get(v1, '$.a', 'int') FROM %s WHERE try_variant_get(v1, '$.a', 'int') > 5", + tableName)) + .containsExactly(row(2L, 10)); + assertThat( + sql( + "SELECT id, try_variant_get(v2, '$.x', 'int') FROM %s WHERE try_variant_get(v2, '$.x', 'int') < 100", + tableName)) + .containsExactlyInAnyOrder(row(1L, 15), row(2L, 20)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java new file mode 100644 index 000000000000..1e00759d8de8 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java @@ -0,0 +1,341 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.iceberg.spark.functions.BucketFunction; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.AnalysisException; +import org.apache.spark.sql.types.DataTypes; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkBucketFunction extends TestBaseWithCatalog { + @BeforeEach + public void useCatalog() { + sql("USE %s", catalogName); + } + + @TestTemplate + public void testSpecValues() { + assertThat(new BucketFunction.BucketInt(DataTypes.IntegerType).hash(34)) + .as("Spec example: hash(34) = 2017239379") + .isEqualTo(2017239379); + + assertThat(new BucketFunction.BucketLong(DataTypes.IntegerType).hash(34L)) + .as("Spec example: hash(34L) = 2017239379") + .isEqualTo(2017239379); + + assertThat( + new BucketFunction.BucketDecimal(DataTypes.createDecimalType(9, 2)) + .hash(new BigDecimal("14.20"))) + .as("Spec example: hash(decimal2(14.20)) = -500754589") + .isEqualTo(-500754589); + + Literal date = Literal.of("2017-11-16").to(Types.DateType.get()); + assertThat(new BucketFunction.BucketInt(DataTypes.DateType).hash(date.value())) + .as("Spec example: hash(2017-11-16) = -653330422") + .isEqualTo(-653330422); + + Literal timestampVal = + Literal.of("2017-11-16T22:31:08").to(Types.TimestampType.withoutZone()); + assertThat(new BucketFunction.BucketLong(DataTypes.TimestampType).hash(timestampVal.value())) + .as("Spec example: hash(2017-11-16T22:31:08) = -2047944441") + .isEqualTo(-2047944441); + + Literal timestampntzVal = + Literal.of("2017-11-16T22:31:08").to(Types.TimestampType.withoutZone()); + assertThat( + new BucketFunction.BucketLong(DataTypes.TimestampNTZType).hash(timestampntzVal.value())) + .as("Spec example: hash(2017-11-16T22:31:08) = -2047944441") + .isEqualTo(-2047944441); + + assertThat(new BucketFunction.BucketString().hash("iceberg")) + .as("Spec example: hash(\"iceberg\") = 1210000089") + .isEqualTo(1210000089); + + assertThat(new BucketFunction.BucketString().hash("iceberg".getBytes(StandardCharsets.UTF_8))) + .as("Verify that the hash string and hash raw bytes produce the same result") + .isEqualTo(new BucketFunction.BucketString().hash("iceberg")); + + ByteBuffer bytes = ByteBuffer.wrap(new byte[] {0, 1, 2, 3}); + assertThat(new BucketFunction.BucketBinary().hash(bytes)) + .as("Spec example: hash([00 01 02 03]) = -188683207") + .isEqualTo(-188683207); + } + + @TestTemplate + public void testBucketIntegers() { + assertThat(scalarSql("SELECT system.bucket(10, 8Y)")) + .as("Byte type should bucket similarly to integer") + .isEqualTo(3); + assertThat(scalarSql("SELECT system.bucket(10, 8S)")) + .as("Short type should bucket similarly to integer") + .isEqualTo(3); + // Integers + assertThat(scalarSql("SELECT system.bucket(10, 8)")).isEqualTo(3); + assertThat(scalarSql("SELECT system.bucket(100, 34)")).isEqualTo(79); + assertThat(scalarSql("SELECT system.bucket(1, CAST(null AS INT))")).isNull(); + } + + @TestTemplate + public void testBucketDates() { + assertThat(scalarSql("SELECT system.bucket(10, date('1970-01-09'))")).isEqualTo(3); + assertThat(scalarSql("SELECT system.bucket(100, date('1970-02-04'))")).isEqualTo(79); + assertThat(scalarSql("SELECT system.bucket(1, CAST(null AS DATE))")).isNull(); + } + + @TestTemplate + public void testBucketLong() { + assertThat(scalarSql("SELECT system.bucket(100, 34L)")).isEqualTo(79); + assertThat(scalarSql("SELECT system.bucket(100, 0L)")).isEqualTo(76); + assertThat(scalarSql("SELECT system.bucket(100, -34L)")).isEqualTo(97); + assertThat(scalarSql("SELECT system.bucket(2, -1L)")).isEqualTo(0); + assertThat(scalarSql("SELECT system.bucket(2, CAST(null AS LONG))")).isNull(); + } + + @TestTemplate + public void testBucketDecimal() { + assertThat(scalarSql("SELECT system.bucket(64, CAST('12.34' as DECIMAL(9, 2)))")).isEqualTo(56); + assertThat(scalarSql("SELECT system.bucket(18, CAST('12.30' as DECIMAL(9, 2)))")).isEqualTo(13); + assertThat(scalarSql("SELECT system.bucket(16, CAST('12.999' as DECIMAL(9, 3)))")).isEqualTo(2); + assertThat(scalarSql("SELECT system.bucket(32, CAST('0.05' as DECIMAL(5, 2)))")).isEqualTo(21); + assertThat(scalarSql("SELECT system.bucket(128, CAST('0.05' as DECIMAL(9, 2)))")).isEqualTo(85); + assertThat(scalarSql("SELECT system.bucket(18, CAST('0.05' as DECIMAL(9, 2)))")).isEqualTo(3); + + assertThat(scalarSql("SELECT system.bucket(2, CAST(null AS decimal))")) + .as("Null input should return null") + .isNull(); + } + + @TestTemplate + public void testBucketTimestamp() { + assertThat(scalarSql("SELECT system.bucket(100, TIMESTAMP '1997-01-01 00:00:00 UTC+00:00')")) + .isEqualTo(99); + assertThat(scalarSql("SELECT system.bucket(100, TIMESTAMP '1997-01-31 09:26:56 UTC+00:00')")) + .isEqualTo(85); + assertThat(scalarSql("SELECT system.bucket(100, TIMESTAMP '2022-08-08 00:00:00 UTC+00:00')")) + .isEqualTo(62); + assertThat(scalarSql("SELECT system.bucket(2, CAST(null AS timestamp))")).isNull(); + } + + @TestTemplate + public void testBucketString() { + assertThat(scalarSql("SELECT system.bucket(5, 'abcdefg')")).isEqualTo(4); + assertThat(scalarSql("SELECT system.bucket(128, 'abc')")).isEqualTo(122); + assertThat(scalarSql("SELECT system.bucket(64, 'abcde')")).isEqualTo(54); + assertThat(scalarSql("SELECT system.bucket(12, '测试')")).isEqualTo(8); + assertThat(scalarSql("SELECT system.bucket(16, '测试raul试测')")).isEqualTo(1); + assertThat(scalarSql("SELECT system.bucket(16, CAST('测试raul试测' AS varchar(8)))")) + .as("Varchar should work like string") + .isEqualTo(1); + assertThat(scalarSql("SELECT system.bucket(16, CAST('测试raul试测' AS char(8)))")) + .as("Char should work like string") + .isEqualTo(1); + assertThat(scalarSql("SELECT system.bucket(16, '')")) + .as("Should not fail on the empty string") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.bucket(16, CAST(null AS string))")) + .as("Null input should return null as output") + .isNull(); + } + + @TestTemplate + public void testBucketBinary() { + assertThat(scalarSql("SELECT system.bucket(10, X'0102030405060708090a0b0c0d0e0f')")) + .isEqualTo(1); + assertThat(scalarSql("SELECT system.bucket(12, %s)", asBytesLiteral("abcdefg"))).isEqualTo(10); + assertThat(scalarSql("SELECT system.bucket(18, %s)", asBytesLiteral("abc\0\0"))).isEqualTo(13); + assertThat(scalarSql("SELECT system.bucket(48, %s)", asBytesLiteral("abc"))).isEqualTo(42); + assertThat(scalarSql("SELECT system.bucket(16, %s)", asBytesLiteral("测试_"))).isEqualTo(3); + + assertThat(scalarSql("SELECT system.bucket(100, CAST(null AS binary))")) + .as("Null input should return null as output") + .isNull(); + } + + @TestTemplate + public void testNumBucketsAcceptsShortAndByte() { + assertThat(scalarSql("SELECT system.bucket(5S, 1L)")) + .as("Short types should be usable for the number of buckets field") + .isEqualTo(1); + + assertThat(scalarSql("SELECT system.bucket(5Y, 1)")) + .as("Byte types should be allowed for the number of buckets field") + .isEqualTo(1); + } + + @TestTemplate + public void testWrongNumberOfArguments() { + assertThatThrownBy(() -> scalarSql("SELECT system.bucket()")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'bucket' cannot process input: (): Wrong number of inputs (expected numBuckets and value)"); + + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'bucket' cannot process input: (int): Wrong number of inputs (expected numBuckets and value)"); + + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1, 1L, 1)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'bucket' cannot process input: (int, bigint, int): Wrong number of inputs (expected numBuckets and value)"); + } + + @TestTemplate + public void testInvalidTypesCannotBeUsedForNumberOfBuckets() { + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(CAST('12.34' as DECIMAL(9, 2)), 10)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'bucket' cannot process input: (decimal(9,2), int): Expected number of buckets to be tinyint, shortint or int"); + + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(12L, 10)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'bucket' cannot process input: (bigint, int): Expected number of buckets to be tinyint, shortint or int"); + + assertThatThrownBy(() -> scalarSql("SELECT system.bucket('5', 10)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'bucket' cannot process input: (string, int): Expected number of buckets to be tinyint, shortint or int"); + + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(INTERVAL '100-00' YEAR TO MONTH, 10)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'bucket' cannot process input: (interval year to month, int): Expected number of buckets to be tinyint, shortint or int"); + + assertThatThrownBy( + () -> + scalarSql("SELECT system.bucket(CAST('11 23:4:0' AS INTERVAL DAY TO SECOND), 10)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'bucket' cannot process input: (interval day to second, int): Expected number of buckets to be tinyint, shortint or int"); + } + + @TestTemplate + public void testInvalidTypesForBucketColumn() { + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, cast(12.3456 as float))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'bucket' cannot process input: (int, float): Expected column to be date, tinyint, smallint, int, bigint, decimal, timestamp, string, or binary"); + + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, cast(12.3456 as double))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'bucket' cannot process input: (int, double): Expected column to be date, tinyint, smallint, int, bigint, decimal, timestamp, string, or binary"); + + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, true)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Function 'bucket' cannot process input: (int, boolean)"); + + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, map(1, 1))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Function 'bucket' cannot process input: (int, map)"); + + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, array(1L))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Function 'bucket' cannot process input: (int, array)"); + + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, INTERVAL '100-00' YEAR TO MONTH)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'bucket' cannot process input: (int, interval year to month)"); + + assertThatThrownBy( + () -> + scalarSql("SELECT system.bucket(10, CAST('11 23:4:0' AS INTERVAL DAY TO SECOND))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'bucket' cannot process input: (int, interval day to second)"); + } + + @TestTemplate + public void testThatMagicFunctionsAreInvoked() { + // TinyInt + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6Y)")) + .asString() + .isNotNull() + .contains("static_invoke(org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); + + // SmallInt + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6S)")) + .asString() + .isNotNull() + .contains("static_invoke(org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); + + // Int + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6)")) + .asString() + .isNotNull() + .contains("static_invoke(org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); + + // Date + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, DATE '2022-08-08')")) + .asString() + .isNotNull() + .contains("static_invoke(org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); + + // Long + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6L)")) + .asString() + .isNotNull() + .contains("static_invoke(org.apache.iceberg.spark.functions.BucketFunction$BucketLong"); + + // Timestamp + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, TIMESTAMP '2022-08-08')")) + .asString() + .isNotNull() + .contains("static_invoke(org.apache.iceberg.spark.functions.BucketFunction$BucketLong"); + + // String + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 'abcdefg')")) + .asString() + .isNotNull() + .contains("static_invoke(org.apache.iceberg.spark.functions.BucketFunction$BucketString"); + + // Decimal + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, CAST('12.34' AS DECIMAL))")) + .asString() + .isNotNull() + .contains("static_invoke(org.apache.iceberg.spark.functions.BucketFunction$BucketDecimal"); + + // Binary + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(4, X'0102030405060708')")) + .asString() + .isNotNull() + .contains("static_invoke(org.apache.iceberg.spark.functions.BucketFunction$BucketBinary"); + } + + private String asBytesLiteral(String value) { + byte[] bytes = value.getBytes(StandardCharsets.UTF_8); + return "X'" + BaseEncoding.base16().encode(bytes) + "'"; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java new file mode 100644 index 000000000000..cfec6a33ab14 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java @@ -0,0 +1,107 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.sql.Date; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.spark.sql.AnalysisException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkDaysFunction extends TestBaseWithCatalog { + + @BeforeEach + public void useCatalog() { + sql("USE %s", catalogName); + } + + @TestTemplate + public void testDates() { + assertThat(scalarSql("SELECT system.days(date('2017-12-01'))")) + .as("Expected to produce 2017-12-01") + .isEqualTo(Date.valueOf("2017-12-01")); + assertThat(scalarSql("SELECT system.days(date('1970-01-01'))")) + .as("Expected to produce 1970-01-01") + .isEqualTo(Date.valueOf("1970-01-01")); + assertThat(scalarSql("SELECT system.days(date('1969-12-31'))")) + .as("Expected to produce 1969-12-31") + .isEqualTo(Date.valueOf("1969-12-31")); + assertThat(scalarSql("SELECT system.days(CAST(null AS DATE))")).isNull(); + } + + @TestTemplate + public void testTimestamps() { + assertThat(scalarSql("SELECT system.days(TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00')")) + .as("Expected to produce 2017-12-01") + .isEqualTo(Date.valueOf("2017-12-01")); + assertThat(scalarSql("SELECT system.days(TIMESTAMP '1970-01-01 00:00:01.000001 UTC+00:00')")) + .as("Expected to produce 1970-01-01") + .isEqualTo(Date.valueOf("1970-01-01")); + assertThat(scalarSql("SELECT system.days(TIMESTAMP '1969-12-31 23:59:58.999999 UTC+00:00')")) + .as("Expected to produce 1969-12-31") + .isEqualTo(Date.valueOf("1969-12-31")); + assertThat(scalarSql("SELECT system.days(CAST(null AS TIMESTAMP))")).isNull(); + } + + @TestTemplate + public void testTimestampNtz() { + assertThat(scalarSql("SELECT system.days(TIMESTAMP_NTZ '2017-12-01 10:12:55.038194 UTC')")) + .as("Expected to produce 2017-12-01") + .isEqualTo(Date.valueOf("2017-12-01")); + assertThat(scalarSql("SELECT system.days(TIMESTAMP_NTZ '1970-01-01 00:00:01.000001 UTC')")) + .as("Expected to produce 1970-01-01") + .isEqualTo(Date.valueOf("1970-01-01")); + assertThat(scalarSql("SELECT system.days(TIMESTAMP_NTZ '1969-12-31 23:59:58.999999 UTC')")) + .as("Expected to produce 1969-12-31") + .isEqualTo(Date.valueOf("1969-12-31")); + assertThat(scalarSql("SELECT system.days(CAST(null AS TIMESTAMP_NTZ))")).isNull(); + } + + @TestTemplate + public void testWrongNumberOfArguments() { + assertThatThrownBy(() -> scalarSql("SELECT system.days()")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith("Function 'days' cannot process input: (): Wrong number of inputs"); + + assertThatThrownBy( + () -> scalarSql("SELECT system.days(date('1969-12-31'), date('1969-12-31'))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'days' cannot process input: (date, date): Wrong number of inputs"); + } + + @TestTemplate + public void testInvalidInputTypes() { + assertThatThrownBy(() -> scalarSql("SELECT system.days(1)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'days' cannot process input: (int): Expected value to be date or timestamp"); + + assertThatThrownBy(() -> scalarSql("SELECT system.days(1L)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'days' cannot process input: (bigint): Expected value to be date or timestamp"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDefaultValues.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDefaultValues.java new file mode 100644 index 000000000000..ba856dc5383b --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDefaultValues.java @@ -0,0 +1,234 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Literal; +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.CatalogTestBase; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.AnalysisException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; + +/** + * Tests for Spark SQL Default values integration with Iceberg default values. + * + *

    Note: These tests use {@code validationCatalog.createTable()} to create tables with default + * values because the Iceberg Spark integration does not yet support default value clauses in Spark + * DDL. + * + *

    Partial column INSERT statements (e.g., {@code INSERT INTO table (col1) VALUES (val1)}) are + * not supported for DSV2 in Spark 4.0 + */ +public class TestSparkDefaultValues extends CatalogTestBase { + + @AfterEach + public void dropTestTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testWriteDefaultWithSparkDefaultKeyword() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional("bool_col") + .withId(2) + .ofType(Types.BooleanType.get()) + .withWriteDefault(Literal.of(true)) + .build(), + Types.NestedField.optional("int_col") + .withId(3) + .ofType(Types.IntegerType.get()) + .withWriteDefault(Literal.of(42)) + .build(), + Types.NestedField.optional("long_col") + .withId(4) + .ofType(Types.LongType.get()) + .withWriteDefault(Literal.of(100L)) + .build()); + + validationCatalog.createTable( + tableIdent, schema, PartitionSpec.unpartitioned(), ImmutableMap.of("format-version", "3")); + + sql("INSERT INTO %s VALUES (1, DEFAULT, DEFAULT, DEFAULT)", commitTarget()); + + assertEquals( + "Should have expected default values", + ImmutableList.of(row(1, true, 42, 100L)), + sql("SELECT * FROM %s", selectTarget())); + } + + @TestTemplate + public void testWriteDefaultWithDefaultKeywordAndReorderedSchema() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional("int_col") + .withId(2) + .ofType(Types.IntegerType.get()) + .withWriteDefault(Literal.of(123)) + .build(), + Types.NestedField.optional("string_col") + .withId(3) + .ofType(Types.StringType.get()) + .withWriteDefault(Literal.of("doom")) + .build()); + + validationCatalog.createTable( + tableIdent, schema, PartitionSpec.unpartitioned(), ImmutableMap.of("format-version", "3")); + + // Insert with columns in different order than table schema + sql("INSERT INTO %s (int_col, id, string_col) VALUES (DEFAULT, 1, DEFAULT)", commitTarget()); + + assertEquals( + "Should apply correct defaults regardless of column order", + ImmutableList.of(row(1, 123, "doom")), + sql("SELECT * FROM %s", selectTarget())); + } + + @TestTemplate + public void testBulkInsertWithDefaults() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withWriteDefault(Literal.of("default_data")) + .build()); + + validationCatalog.createTable( + tableIdent, schema, PartitionSpec.unpartitioned(), ImmutableMap.of("format-version", "3")); + + sql("INSERT INTO %s VALUES (1, DEFAULT), (2, DEFAULT), (3, DEFAULT)", commitTarget()); + + assertEquals( + "Should insert multiple rows with default values", + ImmutableList.of(row(1, "default_data"), row(2, "default_data"), row(3, "default_data")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testCreateTableWithDefaultsUnsupported() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + assertThatThrownBy( + () -> + sql( + "CREATE TABLE %s (id INT, data STRING DEFAULT 'default-value') USING iceberg", + tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("does not support column default value"); + } + + @TestTemplate + public void testAlterTableAddColumnWithDefaultUnsupported() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + Schema schema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + + validationCatalog.createTable( + tableIdent, schema, PartitionSpec.unpartitioned(), ImmutableMap.of("format-version", "3")); + + assertThatThrownBy( + () -> sql("ALTER TABLE %s ADD COLUMN data STRING DEFAULT 'default-value'", tableName)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("default values in Spark is currently unsupported"); + } + + @TestTemplate + public void testPartialInsertUnsupported() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional("data") + .withId(2) + .ofType(Types.StringType.get()) + .withWriteDefault(Literal.of("default-data")) + .build()); + + validationCatalog.createTable( + tableIdent, schema, PartitionSpec.unpartitioned(), ImmutableMap.of("format-version", "3")); + + assertThatThrownBy(() -> sql("INSERT INTO %s (id) VALUES (1)", commitTarget())) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot find data for the output column"); + } + + @TestTemplate + public void testSchemaEvolutionWithDefaultValueChanges() { + assertThat(validationCatalog.tableExists(tableIdent)) + .as("Table should not already exist") + .isFalse(); + + Schema initialSchema = new Schema(Types.NestedField.required(1, "id", Types.IntegerType.get())); + + validationCatalog.createTable( + tableIdent, + initialSchema, + PartitionSpec.unpartitioned(), + ImmutableMap.of("format-version", "3")); + + sql("INSERT INTO %s VALUES (1), (2)", commitTarget()); + + // Add a column with a default value + validationCatalog + .loadTable(tableIdent) + .updateSchema() + .addColumn("data", Types.StringType.get(), Literal.of("default_data")) + .commit(); + + // Refresh this when using SparkCatalog since otherwise the new column would not be caught. + sql("REFRESH TABLE %s", commitTarget()); + + sql("INSERT INTO %s VALUES (3, DEFAULT)", commitTarget()); + + assertEquals( + "Should have correct default values for existing and new rows", + ImmutableList.of(row(1, "default_data"), row(2, "default_data"), row(3, "default_data")), + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java new file mode 100644 index 000000000000..eeeb9d1a1e78 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java @@ -0,0 +1,93 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.spark.sql.AnalysisException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkHoursFunction extends TestBaseWithCatalog { + + @BeforeEach + public void useCatalog() { + sql("USE %s", catalogName); + } + + @TestTemplate + public void testTimestamps() { + assertThat(scalarSql("SELECT system.hours(TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00')")) + .as("Expected to produce 17501 * 24 + 10") + .isEqualTo(420034); + assertThat(scalarSql("SELECT system.hours(TIMESTAMP '1970-01-01 00:00:01.000001 UTC+00:00')")) + .as("Expected to produce 0 * 24 + 0 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.hours(TIMESTAMP '1969-12-31 23:59:58.999999 UTC+00:00')")) + .as("Expected to produce -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.hours(CAST(null AS TIMESTAMP))")).isNull(); + } + + @TestTemplate + public void testTimestampsNtz() { + assertThat(scalarSql("SELECT system.hours(TIMESTAMP_NTZ '2017-12-01 10:12:55.038194 UTC')")) + .as("Expected to produce 17501 * 24 + 10") + .isEqualTo(420034); + assertThat(scalarSql("SELECT system.hours(TIMESTAMP_NTZ '1970-01-01 00:00:01.000001 UTC')")) + .as("Expected to produce 0 * 24 + 0 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.hours(TIMESTAMP_NTZ '1969-12-31 23:59:58.999999 UTC')")) + .as("Expected to produce -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.hours(CAST(null AS TIMESTAMP_NTZ))")).isNull(); + } + + @TestTemplate + public void testWrongNumberOfArguments() { + assertThatThrownBy(() -> scalarSql("SELECT system.hours()")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'hours' cannot process input: (): Wrong number of inputs"); + + assertThatThrownBy( + () -> scalarSql("SELECT system.hours(date('1969-12-31'), date('1969-12-31'))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'hours' cannot process input: (date, date): Wrong number of inputs"); + } + + @TestTemplate + public void testInvalidInputTypes() { + assertThatThrownBy(() -> scalarSql("SELECT system.hours(1)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'hours' cannot process input: (int): Expected value to be timestamp"); + + assertThatThrownBy(() -> scalarSql("SELECT system.hours(1L)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'hours' cannot process input: (bigint): Expected value to be timestamp"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java new file mode 100644 index 000000000000..a61023196f1c --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java @@ -0,0 +1,125 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.iceberg.spark.functions.MonthsFunction; +import org.apache.spark.sql.AnalysisException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkMonthsFunction extends TestBaseWithCatalog { + + @BeforeEach + public void useCatalog() { + sql("USE %s", catalogName); + } + + @TestTemplate + public void testDates() { + assertThat(scalarSql("SELECT system.months(date('2017-12-01'))")) + .as("Expected to produce 47 * 12 + 11 = 575") + .isEqualTo(575); + assertThat(scalarSql("SELECT system.months(date('1970-01-01'))")) + .as("Expected to produce 0 * 12 + 0 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.months(date('1969-12-31'))")) + .as("Expected to produce -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.months(CAST(null AS DATE))")).isNull(); + } + + @TestTemplate + public void testTimestamps() { + assertThat(scalarSql("SELECT system.months(TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00')")) + .as("Expected to produce 47 * 12 + 11 = 575") + .isEqualTo(575); + assertThat(scalarSql("SELECT system.months(TIMESTAMP '1970-01-01 00:00:01.000001 UTC+00:00')")) + .as("Expected to produce 0 * 12 + 0 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.months(TIMESTAMP '1969-12-31 23:59:58.999999 UTC+00:00')")) + .as("Expected to produce -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.months(CAST(null AS TIMESTAMP))")).isNull(); + } + + @TestTemplate + public void testTimestampNtz() { + assertThat(scalarSql("SELECT system.months(TIMESTAMP_NTZ '2017-12-01 10:12:55.038194 UTC')")) + .as("Expected to produce 47 * 12 + 11 = 575") + .isEqualTo(575); + assertThat(scalarSql("SELECT system.months(TIMESTAMP_NTZ '1970-01-01 00:00:01.000001 UTC')")) + .as("Expected to produce 0 * 12 + 0 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.months(TIMESTAMP_NTZ '1969-12-31 23:59:58.999999 UTC')")) + .as("Expected to produce -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.months(CAST(null AS TIMESTAMP_NTZ))")).isNull(); + } + + @TestTemplate + public void testWrongNumberOfArguments() { + assertThatThrownBy(() -> scalarSql("SELECT system.months()")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'months' cannot process input: (): Wrong number of inputs"); + + assertThatThrownBy( + () -> scalarSql("SELECT system.months(date('1969-12-31'), date('1969-12-31'))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'months' cannot process input: (date, date): Wrong number of inputs"); + } + + @TestTemplate + public void testInvalidInputTypes() { + assertThatThrownBy(() -> scalarSql("SELECT system.months(1)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'months' cannot process input: (int): Expected value to be date or timestamp"); + + assertThatThrownBy(() -> scalarSql("SELECT system.months(1L)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'months' cannot process input: (bigint): Expected value to be date or timestamp"); + } + + @TestTemplate + public void testThatMagicFunctionsAreInvoked() { + String dateValue = "date('2017-12-01')"; + String dateTransformClass = MonthsFunction.DateToMonthsFunction.class.getName(); + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", dateValue)) + .asString() + .isNotNull() + .contains("static_invoke(" + dateTransformClass); + + String timestampValue = "TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00'"; + String timestampTransformClass = MonthsFunction.TimestampToMonthsFunction.class.getName(); + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", timestampValue)) + .asString() + .isNotNull() + .contains("static_invoke(" + timestampTransformClass); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java new file mode 100644 index 000000000000..d9e41213d966 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java @@ -0,0 +1,443 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.math.BigDecimal; +import java.nio.charset.StandardCharsets; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.spark.sql.AnalysisException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkTruncateFunction extends TestBaseWithCatalog { + + @BeforeEach + public void useCatalog() { + sql("USE %s", catalogName); + } + + @TestTemplate + public void testTruncateTinyInt() { + assertThat(scalarSql("SELECT system.truncate(10, 0Y)")).isEqualTo((byte) 0); + assertThat(scalarSql("SELECT system.truncate(10, 1Y)")).isEqualTo((byte) 0); + assertThat(scalarSql("SELECT system.truncate(10, 5Y)")).isEqualTo((byte) 0); + assertThat(scalarSql("SELECT system.truncate(10, 9Y)")).isEqualTo((byte) 0); + assertThat(scalarSql("SELECT system.truncate(10, 10Y)")).isEqualTo((byte) 10); + assertThat(scalarSql("SELECT system.truncate(10, 11Y)")).isEqualTo((byte) 10); + assertThat(scalarSql("SELECT system.truncate(10, -1Y)")).isEqualTo((byte) -10); + assertThat(scalarSql("SELECT system.truncate(10, -5Y)")).isEqualTo((byte) -10); + assertThat(scalarSql("SELECT system.truncate(10, -10Y)")).isEqualTo((byte) -10); + assertThat(scalarSql("SELECT system.truncate(10, -11Y)")).isEqualTo((byte) -20); + + // Check that different widths can be used + assertThat(scalarSql("SELECT system.truncate(2, -1Y)")).isEqualTo((byte) -2); + + assertThat(scalarSql("SELECT system.truncate(2, CAST(null AS tinyint))")) + .as("Null input should return null") + .isNull(); + } + + @TestTemplate + public void testTruncateSmallInt() { + assertThat(scalarSql("SELECT system.truncate(10, 0S)")).isEqualTo((short) 0); + assertThat(scalarSql("SELECT system.truncate(10, 1S)")).isEqualTo((short) 0); + assertThat(scalarSql("SELECT system.truncate(10, 5S)")).isEqualTo((short) 0); + assertThat(scalarSql("SELECT system.truncate(10, 9S)")).isEqualTo((short) 0); + assertThat(scalarSql("SELECT system.truncate(10, 10S)")).isEqualTo((short) 10); + assertThat(scalarSql("SELECT system.truncate(10, 11S)")).isEqualTo((short) 10); + assertThat(scalarSql("SELECT system.truncate(10, -1S)")).isEqualTo((short) -10); + assertThat(scalarSql("SELECT system.truncate(10, -5S)")).isEqualTo((short) -10); + assertThat(scalarSql("SELECT system.truncate(10, -10S)")).isEqualTo((short) -10); + assertThat(scalarSql("SELECT system.truncate(10, -11S)")).isEqualTo((short) -20); + + // Check that different widths can be used + assertThat(scalarSql("SELECT system.truncate(2, -1S)")).isEqualTo((short) -2); + + assertThat(scalarSql("SELECT system.truncate(2, CAST(null AS smallint))")) + .as("Null input should return null") + .isNull(); + } + + @TestTemplate + public void testTruncateInt() { + assertThat(scalarSql("SELECT system.truncate(10, 0)")).isEqualTo(0); + assertThat(scalarSql("SELECT system.truncate(10, 1)")).isEqualTo(0); + assertThat(scalarSql("SELECT system.truncate(10, 5)")).isEqualTo(0); + assertThat(scalarSql("SELECT system.truncate(10, 9)")).isEqualTo(0); + assertThat(scalarSql("SELECT system.truncate(10, 10)")).isEqualTo(10); + assertThat(scalarSql("SELECT system.truncate(10, 11)")).isEqualTo(10); + assertThat(scalarSql("SELECT system.truncate(10, -1)")).isEqualTo(-10); + assertThat(scalarSql("SELECT system.truncate(10, -5)")).isEqualTo(-10); + assertThat(scalarSql("SELECT system.truncate(10, -10)")).isEqualTo(-10); + assertThat(scalarSql("SELECT system.truncate(10, -11)")).isEqualTo(-20); + + // Check that different widths can be used + assertThat(scalarSql("SELECT system.truncate(2, -1)")).isEqualTo(-2); + assertThat(scalarSql("SELECT system.truncate(300, 1)")).isEqualTo(0); + + assertThat(scalarSql("SELECT system.truncate(2, CAST(null AS int))")) + .as("Null input should return null") + .isNull(); + } + + @TestTemplate + public void testTruncateBigInt() { + assertThat(scalarSql("SELECT system.truncate(10, 0L)")).isEqualTo(0L); + assertThat(scalarSql("SELECT system.truncate(10, 1L)")).isEqualTo(0L); + assertThat(scalarSql("SELECT system.truncate(10, 5L)")).isEqualTo(0L); + assertThat(scalarSql("SELECT system.truncate(10, 9L)")).isEqualTo(0L); + assertThat(scalarSql("SELECT system.truncate(10, 10L)")).isEqualTo(10L); + assertThat(scalarSql("SELECT system.truncate(10, 11L)")).isEqualTo(10L); + assertThat(scalarSql("SELECT system.truncate(10, -1L)")).isEqualTo(-10L); + assertThat(scalarSql("SELECT system.truncate(10, -5L)")).isEqualTo(-10L); + assertThat(scalarSql("SELECT system.truncate(10, -10L)")).isEqualTo(-10L); + assertThat(scalarSql("SELECT system.truncate(10, -11L)")).isEqualTo(-20L); + + // Check that different widths can be used + assertThat(scalarSql("SELECT system.truncate(2, -1L)")).isEqualTo(-2L); + + assertThat(scalarSql("SELECT system.truncate(2, CAST(null AS bigint))")) + .as("Null input should return null") + .isNull(); + } + + @TestTemplate + public void testTruncateDecimal() { + // decimal truncation works by applying the decimal scale to the width: ie 10 scale 2 = 0.10 + assertThat(scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 2)))", "12.34")) + .isEqualTo(new BigDecimal("12.30")); + + assertThat(scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 2)))", "12.30")) + .isEqualTo(new BigDecimal("12.30")); + + assertThat(scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 3)))", "12.299")) + .isEqualTo(new BigDecimal("12.290")); + + assertThat(scalarSql("SELECT system.truncate(3, CAST(%s as DECIMAL(5, 2)))", "0.05")) + .isEqualTo(new BigDecimal("0.03")); + + assertThat(scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 2)))", "0.05")) + .isEqualTo(new BigDecimal("0.00")); + + assertThat(scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(9, 2)))", "-0.05")) + .isEqualTo(new BigDecimal("-0.10")); + + assertThat(scalarSql("SELECT system.truncate(10, 12345.3482)")) + .as("Implicit decimal scale and precision should be allowed") + .isEqualTo(new BigDecimal("12345.3480")); + + BigDecimal truncatedDecimal = + (BigDecimal) scalarSql("SELECT system.truncate(10, CAST(%s as DECIMAL(6, 4)))", "-0.05"); + assertThat(truncatedDecimal.scale()) + .as("Truncating a decimal should return a decimal with the same scale") + .isEqualTo(4); + + assertThat(truncatedDecimal) + .as("Truncating a decimal should return a decimal with the correct scale") + .isEqualTo(BigDecimal.valueOf(-500, 4)); + + assertThat(scalarSql("SELECT system.truncate(2, CAST(null AS decimal))")) + .as("Null input should return null") + .isNull(); + } + + @SuppressWarnings("checkstyle:AvoidEscapedUnicodeCharacters") + @TestTemplate + public void testTruncateString() { + assertThat(scalarSql("SELECT system.truncate(5, 'abcdefg')")) + .as("Should system.truncate strings longer than length") + .isEqualTo("abcde"); + + assertThat(scalarSql("SELECT system.truncate(5, 'abc')")) + .as("Should not pad strings shorter than length") + .isEqualTo("abc"); + + assertThat(scalarSql("SELECT system.truncate(5, 'abcde')")) + .as("Should not alter strings equal to length") + .isEqualTo("abcde"); + + assertThat(scalarSql("SELECT system.truncate(2, 'イロハニホヘト')")) + .as("Strings with multibyte unicode characters should truncate along codepoint boundaries") + .isEqualTo("イロ"); + + assertThat(scalarSql("SELECT system.truncate(3, 'イロハニホヘト')")) + .as("Strings with multibyte unicode characters should truncate along codepoint boundaries") + .isEqualTo("イロハ"); + + assertThat(scalarSql("SELECT system.truncate(7, 'イロハニホヘト')")) + .as( + "Strings with multibyte unicode characters should not alter input with fewer codepoints than width") + .isEqualTo("イロハニホヘト"); + + String stringWithTwoCodePointsEachFourBytes = "\uD800\uDC00\uD800\uDC00"; + assertThat(scalarSql("SELECT system.truncate(1, '%s')", stringWithTwoCodePointsEachFourBytes)) + .as("String truncation on four byte codepoints should work as expected") + .isEqualTo("\uD800\uDC00"); + + assertThat(scalarSql("SELECT system.truncate(1, '测试')")) + .as("Should handle three-byte UTF-8 characters appropriately") + .isEqualTo("测"); + + assertThat(scalarSql("SELECT system.truncate(4, '测试raul试测')")) + .as("Should handle three-byte UTF-8 characters mixed with two byte utf-8 characters") + .isEqualTo("测试ra"); + + assertThat(scalarSql("SELECT system.truncate(10, '')")) + .as("Should not fail on the empty string") + .isEqualTo(""); + + assertThat(scalarSql("SELECT system.truncate(3, CAST(null AS string))")) + .as("Null input should return null as output") + .isNull(); + + assertThat(scalarSql("SELECT system.truncate(4, CAST('测试raul试测' AS varchar(8)))")) + .as("Varchar should work like string") + .isEqualTo("测试ra"); + + assertThat(scalarSql("SELECT system.truncate(4, CAST('测试raul试测' AS char(8)))")) + .as("Char should work like string") + .isEqualTo("测试ra"); + } + + @TestTemplate + public void testTruncateBinary() { + assertThat((byte[]) scalarSql("SELECT system.truncate(10, X'0102030405060708090a0b0c0d0e0f')")) + .isEqualTo(new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}); + + assertThat((byte[]) scalarSql("SELECT system.truncate(3, %s)", asBytesLiteral("abcdefg"))) + .as("Should return the same input when value is equal to truncation width") + .isEqualTo("abc".getBytes(StandardCharsets.UTF_8)); + + assertThat((byte[]) scalarSql("SELECT system.truncate(10, %s)", asBytesLiteral("abc\0\0"))) + .as("Should not truncate, pad, or trim the input when its length is less than the width") + .isEqualTo("abc\0\0".getBytes(StandardCharsets.UTF_8)); + + assertThat((byte[]) scalarSql("SELECT system.truncate(3, %s)", asBytesLiteral("abc"))) + .as("Should not pad the input when its length is equal to the width") + .isEqualTo("abc".getBytes(StandardCharsets.UTF_8)); + + assertThat((byte[]) scalarSql("SELECT system.truncate(6, %s)", asBytesLiteral("测试_"))) + .as("Should handle three-byte UTF-8 characters appropriately") + .isEqualTo("测试".getBytes(StandardCharsets.UTF_8)); + + assertThat(scalarSql("SELECT system.truncate(3, CAST(null AS binary))")) + .as("Null input should return null as output") + .isNull(); + } + + @TestTemplate + public void testTruncateUsingDataframeForWidthWithVaryingWidth() { + // This situation is atypical but allowed. Typically, width is static as data is partitioned on + // one width. + long rumRows = 10L; + long numNonZero = + spark + .range(rumRows) + .toDF("value") + .selectExpr("CAST(value + 1 AS INT) AS width", "value") + .selectExpr("system.truncate(width, value) as truncated_value") + .filter("truncated_value == 0") + .count(); + assertThat(numNonZero) + .as("A truncate function with variable widths should be usable on dataframe columns") + .isEqualTo(rumRows); + } + + @TestTemplate + public void testWidthAcceptsShortAndByte() { + assertThat(scalarSql("SELECT system.truncate(5S, 1L)")) + .as("Short types should be usable for the width field") + .isEqualTo(0L); + + assertThat(scalarSql("SELECT system.truncate(5Y, 1)")) + .as("Byte types should be allowed for the width field") + .isEqualTo(0); + } + + @TestTemplate + public void testWrongNumberOfArguments() { + assertThatThrownBy(() -> scalarSql("SELECT system.truncate()")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (): Wrong number of inputs (expected width and value)"); + + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (int): Wrong number of inputs (expected width and value)"); + + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1, 1L, 1)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (int, bigint, int): Wrong number of inputs (expected width and value)"); + } + + @TestTemplate + public void testInvalidTypesCannotBeUsedForWidth() { + assertThatThrownBy( + () -> scalarSql("SELECT system.truncate(CAST('12.34' as DECIMAL(9, 2)), 10)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (decimal(9,2), int): Expected truncation width to be tinyint, shortint or int"); + + assertThatThrownBy(() -> scalarSql("SELECT system.truncate('5', 10)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (string, int): Expected truncation width to be tinyint, shortint or int"); + + assertThatThrownBy( + () -> scalarSql("SELECT system.truncate(INTERVAL '100-00' YEAR TO MONTH, 10)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (interval year to month, int): Expected truncation width to be tinyint, shortint or int"); + + assertThatThrownBy( + () -> + scalarSql( + "SELECT system.truncate(CAST('11 23:4:0' AS INTERVAL DAY TO SECOND), 10)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (interval day to second, int): Expected truncation width to be tinyint, shortint or int"); + } + + @TestTemplate + public void testInvalidTypesForTruncationColumn() { + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, cast(12.3456 as float))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (int, float): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, cast(12.3456 as double))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (int, double): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, true)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (int, boolean): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, map(1, 1))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (int, map): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, array(1L))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (int, array): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + + assertThatThrownBy( + () -> scalarSql("SELECT system.truncate(10, INTERVAL '100-00' YEAR TO MONTH)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (int, interval year to month): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + + assertThatThrownBy( + () -> + scalarSql( + "SELECT system.truncate(10, CAST('11 23:4:0' AS INTERVAL DAY TO SECOND))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'truncate' cannot process input: (int, interval day to second): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + } + + @TestTemplate + public void testMagicFunctionsResolveForTinyIntAndSmallIntWidths() { + // Magic functions have staticinvoke in the explain output. Nonmagic calls use + // applyfunctionexpression instead. + String tinyIntWidthExplain = + (String) scalarSql("EXPLAIN EXTENDED SELECT system.truncate(1Y, 6)"); + assertThat(tinyIntWidthExplain) + .contains("cast(1 as int)") + .contains("static_invoke(org.apache.iceberg.spark.functions.TruncateFunction$TruncateInt"); + + String smallIntWidth = (String) scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5S, 6L)"); + assertThat(smallIntWidth) + .contains("cast(5 as int)") + .contains( + "static_invoke(org.apache.iceberg.spark.functions.TruncateFunction$TruncateBigInt"); + } + + @TestTemplate + public void testThatMagicFunctionsAreInvoked() { + // Magic functions have `staticinvoke` in the explain output. + // Non-magic calls have `applyfunctionexpression` instead. + + // TinyInt + assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6Y)")) + .asString() + .isNotNull() + .contains( + "static_invoke(org.apache.iceberg.spark.functions.TruncateFunction$TruncateTinyInt"); + + // SmallInt + assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6S)")) + .asString() + .isNotNull() + .contains( + "static_invoke(org.apache.iceberg.spark.functions.TruncateFunction$TruncateSmallInt"); + + // Int + assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6)")) + .asString() + .isNotNull() + .contains("static_invoke(org.apache.iceberg.spark.functions.TruncateFunction$TruncateInt"); + + // Long + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 6L)")) + .asString() + .isNotNull() + .contains( + "static_invoke(org.apache.iceberg.spark.functions.TruncateFunction$TruncateBigInt"); + + // String + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 'abcdefg')")) + .asString() + .isNotNull() + .contains( + "static_invoke(org.apache.iceberg.spark.functions.TruncateFunction$TruncateString"); + + // Decimal + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 12.34)")) + .asString() + .isNotNull() + .contains( + "static_invoke(org.apache.iceberg.spark.functions.TruncateFunction$TruncateDecimal"); + + // Binary + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(4, X'0102030405060708')")) + .asString() + .isNotNull() + .contains( + "static_invoke(org.apache.iceberg.spark.functions.TruncateFunction$TruncateBinary"); + } + + private String asBytesLiteral(String value) { + byte[] bytes = value.getBytes(StandardCharsets.UTF_8); + return "X'" + BaseEncoding.base16().encode(bytes) + "'"; + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkVariantRead.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkVariantRead.java new file mode 100644 index 000000000000..599bf591e9a4 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkVariantRead.java @@ -0,0 +1,316 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.List; +import org.apache.iceberg.spark.SparkCatalog; +import org.apache.iceberg.spark.TestBase; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.types.variant.Variant; +import org.apache.spark.unsafe.types.VariantVal; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestSparkVariantRead extends TestBase { + + private static final String CATALOG = "local"; + private static final String TABLE = CATALOG + ".default.var"; + + @BeforeAll + public static void setupCatalog() { + // Use a Hadoop catalog to avoid Hive schema conversion (Hive doesn't support VARIANT yet) + spark.conf().set("spark.sql.catalog." + CATALOG, SparkCatalog.class.getName()); + spark.conf().set("spark.sql.catalog." + CATALOG + ".type", "hadoop"); + spark.conf().set("spark.sql.catalog." + CATALOG + ".default-namespace", "default"); + spark.conf().set("spark.sql.catalog." + CATALOG + ".cache-enabled", "false"); + // point warehouse to a temp directory + String temp = System.getProperty("java.io.tmpdir") + "/iceberg_spark_variant_warehouse"; + spark.conf().set("spark.sql.catalog." + CATALOG + ".warehouse", temp); + } + + @BeforeEach + public void setupTable() { + sql("DROP TABLE IF EXISTS %s", TABLE); + sql( + "CREATE TABLE %s (id BIGINT, v1 VARIANT, v2 VARIANT) USING iceberg " + + "TBLPROPERTIES ('format-version'='3')", + TABLE); + + String v1r1 = "{\"a\":1}"; + String v2r1 = "{\"x\":10}"; + String v1r2 = "{\"b\":2}"; + String v2r2 = "{\"y\":20}"; + + sql("INSERT INTO %s SELECT 1, parse_json('%s'), parse_json('%s')", TABLE, v1r1, v2r1); + sql("INSERT INTO %s SELECT 2, parse_json('%s'), parse_json('%s')", TABLE, v1r2, v2r2); + } + + @AfterEach + public void cleanup() { + sql("DROP TABLE IF EXISTS %s", TABLE); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testVariantColumnProjection_singleVariant(boolean vectorized) { + assumeThat(vectorized).as("Variant vectorized Parquet read is not implemented yet").isFalse(); + setVectorization(vectorized); + Dataset df = spark.table(TABLE).select("id", "v1").orderBy("id"); + assertThat(df.schema().fieldNames()).containsExactly("id", "v1"); + assertThat(df.count()).isEqualTo(2); + + List directRows = df.collectAsList(); + Object v1row1 = directRows.get(0).get(1); + Object v1row2 = directRows.get(1).get(1); + assertThat(v1row1).isInstanceOf(VariantVal.class); + assertThat(v1row2).isInstanceOf(VariantVal.class); + VariantVal r1 = (VariantVal) v1row1; + VariantVal r2 = (VariantVal) v1row2; + Variant vv1 = new Variant(r1.getValue(), r1.getMetadata()); + Variant vv2 = new Variant(r2.getValue(), r2.getMetadata()); + + // row 1 has {"a":1} + Variant fieldA = vv1.getFieldByKey("a"); + assertThat(fieldA).isNotNull(); + assertThat(fieldA.getLong()).isEqualTo(1L); + + // row 2 has {"b":2} + Variant fieldB = vv2.getFieldByKey("b"); + assertThat(fieldB).isNotNull(); + assertThat(fieldB.getLong()).isEqualTo(2L); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testVariantColumnProjectionNoVariant(boolean vectorized) { + assumeThat(vectorized).as("Variant vectorized Parquet read is not implemented yet").isFalse(); + setVectorization(vectorized); + Dataset df = spark.table(TABLE).select("id"); + assertThat(df.schema().fieldNames()).containsExactly("id"); + assertThat(df.count()).isEqualTo(2); + assertThat(df.collectAsList()).extracting(r -> r.getLong(0)).containsExactlyInAnyOrder(1L, 2L); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testFilterOnVariantColumnOnWholeValue(boolean vectorized) { + assumeThat(vectorized).as("Variant vectorized Parquet read is not implemented yet").isFalse(); + setVectorization(vectorized); + sql("INSERT INTO %s SELECT 3, NULL, NULL", TABLE); + + Dataset nullDf = spark.table(TABLE).where("v1 IS NULL").select("id"); + assertThat(nullDf.collectAsList()).extracting(r -> r.getLong(0)).containsExactly(3L); + + Dataset notNullDf = spark.table(TABLE).where("v1 IS NOT NULL").select("id"); + assertThat(notNullDf.collectAsList()) + .extracting(r -> r.getLong(0)) + .containsExactlyInAnyOrder(1L, 2L); + + // verify variant contents for non-null rows + Dataset notNullVals = + spark + .table(TABLE) + .where("v1 IS NOT NULL") + .selectExpr("id", "to_json(v1) as v1_json") + .orderBy("id"); + List nn = notNullVals.collectAsList(); + assertThat(nn).hasSize(2); + assertThat(nn.get(0).getLong(0)).isEqualTo(1L); + assertThat(nn.get(0).getString(1)).isEqualTo("{\"a\":1}"); + assertThat(nn.get(1).getLong(0)).isEqualTo(2L); + assertThat(nn.get(1).getString(1)).isEqualTo("{\"b\":2}"); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testVariantNullValueProjection(boolean vectorized) { + assumeThat(vectorized).as("Variant vectorized Parquet read is not implemented yet").isFalse(); + setVectorization(vectorized); + + // insert a row with NULL variant values + sql("INSERT INTO %s SELECT 10, NULL, NULL", TABLE); + + // select id and variant; ensure the variant value is null + Dataset df = spark.table(TABLE).where("id = 10").select("id", "v1"); + List rows = df.collectAsList(); + assertThat(rows).hasSize(1); + Row row = rows.get(0); + assertThat(row.getLong(0)).isEqualTo(10L); + assertThat(row.isNullAt(1)).isTrue(); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testNestedStructVariant(boolean vectorized) { + assumeThat(vectorized).as("Variant vectorized Parquet read is not implemented yet").isFalse(); + + String structTable = CATALOG + ".default.var_struct"; + sql("DROP TABLE IF EXISTS %s", structTable); + sql( + "CREATE TABLE %s (id BIGINT, s STRUCT) USING iceberg " + + "TBLPROPERTIES ('format-version'='3')", + structTable); + setVectorization(structTable, vectorized); + + String j1 = "{\"a\":1}"; + String j2 = "{\"b\":2}"; + sql("INSERT INTO %s SELECT 1, named_struct('v', parse_json('%s'))", structTable, j1); + sql("INSERT INTO %s SELECT 2, named_struct('v', parse_json('%s'))", structTable, j2); + + Dataset df = spark.table(structTable).selectExpr("id", "s.v AS v").orderBy("id"); + java.util.List rows = df.collectAsList(); + assertThat(rows.get(0).getLong(0)).isEqualTo(1L); + Object sv1 = rows.get(0).get(1); + assertThat(sv1).isInstanceOf(VariantVal.class); + Variant sv1Var = new Variant(((VariantVal) sv1).getValue(), ((VariantVal) sv1).getMetadata()); + assertThat(sv1Var.getFieldByKey("a").getLong()).isEqualTo(1L); + + assertThat(rows.get(1).getLong(0)).isEqualTo(2L); + Object sv2 = rows.get(1).get(1); + assertThat(sv2).isInstanceOf(VariantVal.class); + Variant sv2Var = new Variant(((VariantVal) sv2).getValue(), ((VariantVal) sv2).getMetadata()); + assertThat(sv2Var.getFieldByKey("b").getLong()).isEqualTo(2L); + + sql("DROP TABLE IF EXISTS %s", structTable); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testNestedArrayVariant(boolean vectorized) { + assumeThat(vectorized).as("Variant vectorized Parquet read is not implemented yet").isFalse(); + + String arrayTable = CATALOG + ".default.var_array"; + sql("DROP TABLE IF EXISTS %s", arrayTable); + sql( + "CREATE TABLE %s (id BIGINT, arr ARRAY) USING iceberg " + + "TBLPROPERTIES ('format-version'='3')", + arrayTable); + setVectorization(arrayTable, vectorized); + + String a1 = "{\"a\":1}"; + String a2 = "{\"x\":10}"; + String b1 = "{\"b\":2}"; + String b2 = "{\"y\":20}"; + sql("INSERT INTO %s SELECT 1, array(parse_json('%s'), parse_json('%s'))", arrayTable, a1, a2); + sql("INSERT INTO %s SELECT 2, array(parse_json('%s'), parse_json('%s'))", arrayTable, b1, b2); + + Dataset df = + spark.table(arrayTable).selectExpr("id", "arr[0] as e0", "arr[1] as e1").orderBy("id"); + java.util.List rows = df.collectAsList(); + assertThat(rows.get(0).getLong(0)).isEqualTo(1L); + Variant e0r1 = + new Variant( + ((VariantVal) rows.get(0).get(1)).getValue(), + ((VariantVal) rows.get(0).get(1)).getMetadata()); + Variant e1r1 = + new Variant( + ((VariantVal) rows.get(0).get(2)).getValue(), + ((VariantVal) rows.get(0).get(2)).getMetadata()); + assertThat(e0r1.getFieldByKey("a").getLong()).isEqualTo(1L); + assertThat(e1r1.getFieldByKey("x").getLong()).isEqualTo(10L); + assertThat(rows.get(1).getLong(0)).isEqualTo(2L); + Variant e0r2 = + new Variant( + ((VariantVal) rows.get(1).get(1)).getValue(), + ((VariantVal) rows.get(1).get(1)).getMetadata()); + Variant e1r2 = + new Variant( + ((VariantVal) rows.get(1).get(2)).getValue(), + ((VariantVal) rows.get(1).get(2)).getMetadata()); + assertThat(e0r2.getFieldByKey("b").getLong()).isEqualTo(2L); + assertThat(e1r2.getFieldByKey("y").getLong()).isEqualTo(20L); + + sql("DROP TABLE IF EXISTS %s", arrayTable); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testNestedMapVariant(boolean vectorized) { + assumeThat(vectorized).as("Variant vectorized Parquet read is not implemented yet").isFalse(); + + String mapTable = CATALOG + ".default.var_map"; + sql("DROP TABLE IF EXISTS %s", mapTable); + sql( + "CREATE TABLE %s (id BIGINT, m MAP) USING iceberg " + + "TBLPROPERTIES ('format-version'='3')", + mapTable); + setVectorization(mapTable, vectorized); + + String k1a = "{\"a\":1}"; + String k2x = "{\"x\":10}"; + String k1b = "{\"b\":2}"; + String k2y = "{\"y\":20}"; + sql( + "INSERT INTO %s SELECT 1, map('k1', parse_json('%s'), 'k2', parse_json('%s'))", + mapTable, k1a, k2x); + sql( + "INSERT INTO %s SELECT 2, map('k1', parse_json('%s'), 'k2', parse_json('%s'))", + mapTable, k1b, k2y); + + Dataset df = + spark + .table(mapTable) + .selectExpr("id", "element_at(m, 'k1') as k1", "element_at(m, 'k2') as k2") + .orderBy("id"); + java.util.List rows = df.collectAsList(); + assertThat(rows.get(0).getLong(0)).isEqualTo(1L); + Variant k1r1 = + new Variant( + ((VariantVal) rows.get(0).get(1)).getValue(), + ((VariantVal) rows.get(0).get(1)).getMetadata()); + Variant k2r1 = + new Variant( + ((VariantVal) rows.get(0).get(2)).getValue(), + ((VariantVal) rows.get(0).get(2)).getMetadata()); + assertThat(k1r1.getFieldByKey("a").getLong()).isEqualTo(1L); + assertThat(k2r1.getFieldByKey("x").getLong()).isEqualTo(10L); + assertThat(rows.get(1).getLong(0)).isEqualTo(2L); + Variant k1r2 = + new Variant( + ((VariantVal) rows.get(1).get(1)).getValue(), + ((VariantVal) rows.get(1).get(1)).getMetadata()); + Variant k2r2 = + new Variant( + ((VariantVal) rows.get(1).get(2)).getValue(), + ((VariantVal) rows.get(1).get(2)).getMetadata()); + assertThat(k1r2.getFieldByKey("b").getLong()).isEqualTo(2L); + assertThat(k2r2.getFieldByKey("y").getLong()).isEqualTo(20L); + + sql("DROP TABLE IF EXISTS %s", mapTable); + } + + private void setVectorization(boolean on) { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('read.parquet.vectorization.enabled'='%s')", + TABLE, Boolean.toString(on)); + } + + private void setVectorization(String table, boolean on) { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('read.parquet.vectorization.enabled'='%s')", + table, Boolean.toString(on)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java new file mode 100644 index 000000000000..213963a65ff1 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java @@ -0,0 +1,125 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.iceberg.spark.functions.YearsFunction; +import org.apache.spark.sql.AnalysisException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSparkYearsFunction extends TestBaseWithCatalog { + + @BeforeEach + public void useCatalog() { + sql("USE %s", catalogName); + } + + @TestTemplate + public void testDates() { + assertThat(scalarSql("SELECT system.years(date('2017-12-01'))")) + .as("Expected to produce 2017 - 1970 = 47") + .isEqualTo(47); + assertThat(scalarSql("SELECT system.years(date('1970-01-01'))")) + .as("Expected to produce 1970 - 1970 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.years(date('1969-12-31'))")) + .as("Expected to produce 1969 - 1970 = -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.years(CAST(null AS DATE))")).isNull(); + } + + @TestTemplate + public void testTimestamps() { + assertThat(scalarSql("SELECT system.years(TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00')")) + .as("Expected to produce 2017 - 1970 = 47") + .isEqualTo(47); + assertThat(scalarSql("SELECT system.years(TIMESTAMP '1970-01-01 00:00:01.000001 UTC+00:00')")) + .as("Expected to produce 1970 - 1970 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.years(TIMESTAMP '1969-12-31 23:59:58.999999 UTC+00:00')")) + .as("Expected to produce 1969 - 1970 = -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.years(CAST(null AS TIMESTAMP))")).isNull(); + } + + @TestTemplate + public void testTimestampNtz() { + assertThat(scalarSql("SELECT system.years(TIMESTAMP_NTZ '2017-12-01 10:12:55.038194 UTC')")) + .as("Expected to produce 2017 - 1970 = 47") + .isEqualTo(47); + assertThat(scalarSql("SELECT system.years(TIMESTAMP_NTZ '1970-01-01 00:00:01.000001 UTC')")) + .as("Expected to produce 1970 - 1970 = 0") + .isEqualTo(0); + assertThat(scalarSql("SELECT system.years(TIMESTAMP_NTZ '1969-12-31 23:59:58.999999 UTC')")) + .as("Expected to produce 1969 - 1970 = -1") + .isEqualTo(-1); + assertThat(scalarSql("SELECT system.years(CAST(null AS TIMESTAMP_NTZ))")).isNull(); + } + + @TestTemplate + public void testWrongNumberOfArguments() { + assertThatThrownBy(() -> scalarSql("SELECT system.years()")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'years' cannot process input: (): Wrong number of inputs"); + + assertThatThrownBy( + () -> scalarSql("SELECT system.years(date('1969-12-31'), date('1969-12-31'))")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'years' cannot process input: (date, date): Wrong number of inputs"); + } + + @TestTemplate + public void testInvalidInputTypes() { + assertThatThrownBy(() -> scalarSql("SELECT system.years(1)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'years' cannot process input: (int): Expected value to be date or timestamp"); + + assertThatThrownBy(() -> scalarSql("SELECT system.years(1L)")) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + "Function 'years' cannot process input: (bigint): Expected value to be date or timestamp"); + } + + @TestTemplate + public void testThatMagicFunctionsAreInvoked() { + String dateValue = "date('2017-12-01')"; + String dateTransformClass = YearsFunction.DateToYearsFunction.class.getName(); + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", dateValue)) + .asString() + .isNotNull() + .contains("static_invoke(" + dateTransformClass); + + String timestampValue = "TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00'"; + String timestampTransformClass = YearsFunction.TimestampToYearsFunction.class.getName(); + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", timestampValue)) + .asString() + .isNotNull() + .contains("static_invoke(" + timestampTransformClass); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java new file mode 100644 index 000000000000..8788dff15806 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestStoragePartitionedJoins.java @@ -0,0 +1,881 @@ +/* + * 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.sql; + +import static org.apache.iceberg.PlanningMode.DISTRIBUTED; +import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.commons.lang3.StringUtils; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkSQLProperties; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestStoragePartitionedJoins extends TestBaseWithCatalog { + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, planningMode = {3}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + LOCAL + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + DISTRIBUTED + }, + }; + } + + private static final String OTHER_TABLE_NAME = "other_table"; + + // open file cost and split size are set as 16 MB to produce a split per file + private static final Map TABLE_PROPERTIES = + ImmutableMap.of( + TableProperties.SPLIT_SIZE, "16777216", TableProperties.SPLIT_OPEN_FILE_COST, "16777216"); + + // only v2 bucketing and preserve data grouping properties have to be enabled to trigger SPJ + // other properties are only to simplify testing and validation + private static final Map ENABLED_SPJ_SQL_CONF = + ImmutableMap.of( + SQLConf.V2_BUCKETING_ENABLED().key(), + "true", + SQLConf.V2_BUCKETING_PUSH_PART_VALUES_ENABLED().key(), + "true", + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(), + "false", + SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), + "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(), + "-1", + SparkSQLProperties.PRESERVE_DATA_GROUPING, + "true", + SQLConf.V2_BUCKETING_PARTIALLY_CLUSTERED_DISTRIBUTION_ENABLED().key(), + "false", + SQLConf.V2_BUCKETING_ALLOW_JOIN_KEYS_SUBSET_OF_PARTITION_KEYS().key(), + "true", + SQLConf.V2_BUCKETING_ALLOW_COMPATIBLE_TRANSFORMS().key(), + "true"); + + private static final Map DISABLED_SPJ_SQL_CONF = + ImmutableMap.of( + SQLConf.V2_BUCKETING_ENABLED().key(), + "false", + SQLConf.REQUIRE_ALL_CLUSTER_KEYS_FOR_CO_PARTITION().key(), + "false", + SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), + "false", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD().key(), + "-1", + SparkSQLProperties.PRESERVE_DATA_GROUPING, + "true"); + + @Parameter(index = 3) + private PlanningMode planningMode; + + @BeforeAll + public static void setupSparkConf() { + spark.conf().set("spark.sql.shuffle.partitions", "4"); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME)); + } + + // TODO: add tests for truncate transforms once SPARK-40295 is released + + @TestTemplate + public void testJoinsWithBucketingOnByteColumn() throws NoSuchTableException { + checkJoin("byte_col", "TINYINT", "bucket(4, byte_col)"); + } + + @TestTemplate + public void testJoinsWithBucketingOnShortColumn() throws NoSuchTableException { + checkJoin("short_col", "SMALLINT", "bucket(4, short_col)"); + } + + @TestTemplate + public void testJoinsWithBucketingOnIntColumn() throws NoSuchTableException { + checkJoin("int_col", "INT", "bucket(16, int_col)"); + } + + @TestTemplate + public void testJoinsWithBucketingOnLongColumn() throws NoSuchTableException { + checkJoin("long_col", "BIGINT", "bucket(16, long_col)"); + } + + @TestTemplate + public void testJoinsWithBucketingOnTimestampColumn() throws NoSuchTableException { + checkJoin("timestamp_col", "TIMESTAMP", "bucket(16, timestamp_col)"); + } + + @TestTemplate + public void testJoinsWithBucketingOnTimestampNtzColumn() throws NoSuchTableException { + checkJoin("timestamp_col", "TIMESTAMP_NTZ", "bucket(16, timestamp_col)"); + } + + @TestTemplate + public void testJoinsWithBucketingOnDateColumn() throws NoSuchTableException { + checkJoin("date_col", "DATE", "bucket(8, date_col)"); + } + + @TestTemplate + public void testJoinsWithBucketingOnDecimalColumn() throws NoSuchTableException { + checkJoin("decimal_col", "DECIMAL(20, 2)", "bucket(8, decimal_col)"); + } + + @TestTemplate + public void testJoinsWithBucketingOnBinaryColumn() throws NoSuchTableException { + checkJoin("binary_col", "BINARY", "bucket(8, binary_col)"); + } + + @TestTemplate + public void testJoinsWithYearsOnTimestampColumn() throws NoSuchTableException { + checkJoin("timestamp_col", "TIMESTAMP", "years(timestamp_col)"); + } + + @TestTemplate + public void testJoinsWithYearsOnTimestampNtzColumn() throws NoSuchTableException { + checkJoin("timestamp_col", "TIMESTAMP_NTZ", "years(timestamp_col)"); + } + + @TestTemplate + public void testJoinsWithYearsOnDateColumn() throws NoSuchTableException { + checkJoin("date_col", "DATE", "years(date_col)"); + } + + @TestTemplate + public void testJoinsWithMonthsOnTimestampColumn() throws NoSuchTableException { + checkJoin("timestamp_col", "TIMESTAMP", "months(timestamp_col)"); + } + + @TestTemplate + public void testJoinsWithMonthsOnTimestampNtzColumn() throws NoSuchTableException { + checkJoin("timestamp_col", "TIMESTAMP_NTZ", "months(timestamp_col)"); + } + + @TestTemplate + public void testJoinsWithMonthsOnDateColumn() throws NoSuchTableException { + checkJoin("date_col", "DATE", "months(date_col)"); + } + + @TestTemplate + public void testJoinsWithDaysOnTimestampColumn() throws NoSuchTableException { + checkJoin("timestamp_col", "TIMESTAMP", "days(timestamp_col)"); + } + + @TestTemplate + public void testJoinsWithDaysOnTimestampNtzColumn() throws NoSuchTableException { + checkJoin("timestamp_col", "TIMESTAMP_NTZ", "days(timestamp_col)"); + } + + @TestTemplate + public void testJoinsWithDaysOnDateColumn() throws NoSuchTableException { + checkJoin("date_col", "DATE", "days(date_col)"); + } + + @TestTemplate + public void testJoinsWithHoursOnTimestampColumn() throws NoSuchTableException { + checkJoin("timestamp_col", "TIMESTAMP", "hours(timestamp_col)"); + } + + @TestTemplate + public void testJoinsWithHoursOnTimestampNtzColumn() throws NoSuchTableException { + checkJoin("timestamp_col", "TIMESTAMP_NTZ", "hours(timestamp_col)"); + } + + @TestTemplate + public void testJoinsWithMultipleTransformTypes() throws NoSuchTableException { + String createTableStmt = + "CREATE TABLE %s (" + + " id BIGINT, int_col INT, date_col1 DATE, date_col2 DATE, date_col3 DATE," + + " timestamp_col TIMESTAMP, string_col STRING, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (" + + " years(date_col1), months(date_col2), days(date_col3), hours(timestamp_col), " + + " bucket(8, int_col), dep)" + + "TBLPROPERTIES (%s)"; + + sql(createTableStmt, tableName, tablePropsAsString(TABLE_PROPERTIES)); + sql(createTableStmt, tableName(OTHER_TABLE_NAME), tablePropsAsString(TABLE_PROPERTIES)); + + Table table = validationCatalog.loadTable(tableIdent); + + Dataset dataDF = randomDataDF(table.schema(), 16); + + // write to the first table 1 time to generate 1 file per partition + append(tableName, dataDF); + + // write to the second table 2 times to generate 2 files per partition + append(tableName(OTHER_TABLE_NAME), dataDF); + append(tableName(OTHER_TABLE_NAME), dataDF); + + // Spark SPJ support is limited at the moment and requires all source partitioning columns, + // which were projected in the query, to be part of the join condition + // suppose a table is partitioned by `p1`, `bucket(8, pk)` + // queries covering `p1` and `pk` columns must include equality predicates + // on both `p1` and `pk` to benefit from SPJ + // this is a temporary Spark limitation that will be removed in a future release + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT t1.id " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id AND t1.dep = t2.dep " + + "ORDER BY t1.id", + tableName, + tableName(OTHER_TABLE_NAME)); + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT t1.id, t1.int_col, t1.date_col1 " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id AND t1.int_col = t2.int_col AND t1.date_col1 = t2.date_col1 " + + "ORDER BY t1.id, t1.int_col, t1.date_col1", + tableName, + tableName(OTHER_TABLE_NAME)); + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT t1.id, t1.timestamp_col, t1.string_col " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id AND t1.timestamp_col = t2.timestamp_col AND t1.string_col = t2.string_col " + + "ORDER BY t1.id, t1.timestamp_col, t1.string_col", + tableName, + tableName(OTHER_TABLE_NAME)); + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT t1.id, t1.date_col1, t1.date_col2, t1.date_col3 " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id AND t1.date_col1 = t2.date_col1 AND t1.date_col2 = t2.date_col2 AND t1.date_col3 = t2.date_col3 " + + "ORDER BY t1.id, t1.date_col1, t1.date_col2, t1.date_col3", + tableName, + tableName(OTHER_TABLE_NAME)); + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT t1.id, t1.int_col, t1.timestamp_col, t1.dep " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id AND t1.int_col = t2.int_col AND t1.timestamp_col = t2.timestamp_col AND t1.dep = t2.dep " + + "ORDER BY t1.id, t1.int_col, t1.timestamp_col, t1.dep", + tableName, + tableName(OTHER_TABLE_NAME)); + } + + @TestTemplate + public void testJoinsWithCompatibleSpecEvolution() { + // create a table with an empty spec + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "TBLPROPERTIES (%s)", + tableName, tablePropsAsString(TABLE_PROPERTIES)); + + Table table = validationCatalog.loadTable(tableIdent); + + // evolve the spec in the first table by adding `dep` + table.updateSpec().addField("dep").commit(); + + // insert data into the first table partitioned by `dep` + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO %s VALUES (1L, 100, 'software')", tableName); + + // evolve the spec in the first table by adding `bucket(int_col, 8)` + table.updateSpec().addField(Expressions.bucket("int_col", 8)).commit(); + + // insert data into the first table partitioned by `dep`, `bucket(8, int_col)` + sql("REFRESH TABLE %s", tableName); + sql("INSERT INTO %s VALUES (2L, 200, 'hr')", tableName); + + // create another table partitioned by `other_dep` + sql( + "CREATE TABLE %s (other_id BIGINT, other_int_col INT, other_dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (other_dep)" + + "TBLPROPERTIES (%s)", + tableName(OTHER_TABLE_NAME), tablePropsAsString(TABLE_PROPERTIES)); + + // insert data into the second table partitioned by 'other_dep' + sql("INSERT INTO %s VALUES (1L, 100, 'software')", tableName(OTHER_TABLE_NAME)); + sql("INSERT INTO %s VALUES (2L, 200, 'hr')", tableName(OTHER_TABLE_NAME)); + + // SPJ would apply as the grouping keys are compatible + // the first table: `dep` (an intersection of all active partition fields across scanned specs) + // the second table: `other_dep` (the only partition field). + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT * " + + "FROM %s " + + "INNER JOIN %s " + + "ON id = other_id AND int_col = other_int_col AND dep = other_dep " + + "ORDER BY id, int_col, dep", + tableName, + tableName(OTHER_TABLE_NAME)); + } + + @TestTemplate + public void testJoinsWithIncompatibleSpecs() { + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep)" + + "TBLPROPERTIES (%s)", + tableName, tablePropsAsString(TABLE_PROPERTIES)); + + sql("INSERT INTO %s VALUES (1L, 100, 'software')", tableName); + sql("INSERT INTO %s VALUES (2L, 200, 'software')", tableName); + sql("INSERT INTO %s VALUES (3L, 300, 'software')", tableName); + + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (bucket(8, int_col))" + + "TBLPROPERTIES (%s)", + tableName(OTHER_TABLE_NAME), tablePropsAsString(TABLE_PROPERTIES)); + + sql("INSERT INTO %s VALUES (1L, 100, 'software')", tableName(OTHER_TABLE_NAME)); + sql("INSERT INTO %s VALUES (2L, 200, 'software')", tableName(OTHER_TABLE_NAME)); + sql("INSERT INTO %s VALUES (3L, 300, 'software')", tableName(OTHER_TABLE_NAME)); + + // queries can't benefit from SPJ as specs are not compatible + // the first table: `dep` + // the second table: `bucket(8, int_col)` + + assertPartitioningAwarePlan( + 3, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles with SPJ */ + "SELECT * " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id AND t1.int_col = t2.int_col AND t1.dep = t2.dep " + + "ORDER BY t1.id, t1.int_col, t1.dep, t2.id, t2.int_col, t2.dep", + tableName, + tableName(OTHER_TABLE_NAME)); + } + + @TestTemplate + public void testJoinsWithUnpartitionedTables() { + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "TBLPROPERTIES (" + + " 'read.split.target-size' = 16777216," + + " 'read.split.open-file-cost' = 16777216)", + tableName); + + sql("INSERT INTO %s VALUES (1L, 100, 'software')", tableName); + sql("INSERT INTO %s VALUES (2L, 200, 'software')", tableName); + sql("INSERT INTO %s VALUES (3L, 300, 'software')", tableName); + + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "TBLPROPERTIES (" + + " 'read.split.target-size' = 16777216," + + " 'read.split.open-file-cost' = 16777216)", + tableName(OTHER_TABLE_NAME)); + + sql("INSERT INTO %s VALUES (1L, 100, 'software')", tableName(OTHER_TABLE_NAME)); + sql("INSERT INTO %s VALUES (2L, 200, 'software')", tableName(OTHER_TABLE_NAME)); + sql("INSERT INTO %s VALUES (3L, 300, 'software')", tableName(OTHER_TABLE_NAME)); + + // queries covering unpartitioned tables can't benefit from SPJ but shouldn't fail + + assertPartitioningAwarePlan( + 3, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT * " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id AND t1.int_col = t2.int_col AND t1.dep = t2.dep " + + "ORDER BY t1.id, t1.int_col, t1.dep, t2.id, t2.int_col, t2.dep", + tableName, + tableName(OTHER_TABLE_NAME)); + } + + @TestTemplate + public void testJoinsWithEmptyTable() { + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep)" + + "TBLPROPERTIES (%s)", + tableName, tablePropsAsString(TABLE_PROPERTIES)); + + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep)" + + "TBLPROPERTIES (%s)", + tableName(OTHER_TABLE_NAME), tablePropsAsString(TABLE_PROPERTIES)); + + sql("INSERT INTO %s VALUES (1L, 100, 'software')", tableName(OTHER_TABLE_NAME)); + sql("INSERT INTO %s VALUES (2L, 200, 'software')", tableName(OTHER_TABLE_NAME)); + sql("INSERT INTO %s VALUES (3L, 300, 'software')", tableName(OTHER_TABLE_NAME)); + + assertPartitioningAwarePlan( + 3, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT * " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id AND t1.int_col = t2.int_col AND t1.dep = t2.dep " + + "ORDER BY t1.id, t1.int_col, t1.dep, t2.id, t2.int_col, t2.dep", + tableName, + tableName(OTHER_TABLE_NAME)); + } + + @TestTemplate + public void testJoinsWithOneSplitTables() { + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep)" + + "TBLPROPERTIES (%s)", + tableName, tablePropsAsString(TABLE_PROPERTIES)); + + sql("INSERT INTO %s VALUES (1L, 100, 'software')", tableName); + + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep)" + + "TBLPROPERTIES (%s)", + tableName(OTHER_TABLE_NAME), tablePropsAsString(TABLE_PROPERTIES)); + + sql("INSERT INTO %s VALUES (1L, 100, 'software')", tableName(OTHER_TABLE_NAME)); + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT * " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id AND t1.int_col = t2.int_col AND t1.dep = t2.dep " + + "ORDER BY t1.id, t1.int_col, t1.dep, t2.id, t2.int_col, t2.dep", + tableName, + tableName(OTHER_TABLE_NAME)); + } + + @TestTemplate + public void testJoinsWithMismatchingPartitionKeys() { + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep)" + + "TBLPROPERTIES (%s)", + tableName, tablePropsAsString(TABLE_PROPERTIES)); + + sql("INSERT INTO %s VALUES (1L, 100, 'software')", tableName); + sql("INSERT INTO %s VALUES (2L, 100, 'hr')", tableName); + + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep)" + + "TBLPROPERTIES (%s)", + tableName(OTHER_TABLE_NAME), tablePropsAsString(TABLE_PROPERTIES)); + + sql("INSERT INTO %s VALUES (1L, 100, 'software')", tableName(OTHER_TABLE_NAME)); + sql("INSERT INTO %s VALUES (3L, 300, 'hardware')", tableName(OTHER_TABLE_NAME)); + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT * " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id AND t1.dep = t2.dep " + + "ORDER BY t1.id, t1.int_col, t1.dep, t2.id, t2.int_col, t2.dep", + tableName, + tableName(OTHER_TABLE_NAME)); + } + + @TestTemplate + public void testJoinsCompatibleBucketNumbers() { + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (bucket(4, id))" + + "TBLPROPERTIES (%s)", + tableName, tablePropsAsString(TABLE_PROPERTIES)); + + sql( + "INSERT INTO %s VALUES " + + "(1L, 100, 'software')," + + "(2L, 101, 'hr')," + + "(3L, 102, 'operation')," + + "(4L, 103, 'sales')," + + "(5L, 104, 'marketing')," + + "(6L, 105, 'pr')", + tableName); + + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (bucket(6, id))" + + "TBLPROPERTIES (%s)", + tableName(OTHER_TABLE_NAME), tablePropsAsString(TABLE_PROPERTIES)); + + sql( + "INSERT INTO %s VALUES " + + "(1L, 100, 'software')," + + "(3L, 300, 'hardware')," + + "(4L, 103, 'sales')," + + "(5L, 104, 'marketing')," + + "(6L, 105, 'pr')", + tableName(OTHER_TABLE_NAME)); + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT * " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id " + + "ORDER BY t1.id, t1.int_col, t1.dep, t2.id, t2.int_col, t2.dep", + tableName, + tableName(OTHER_TABLE_NAME)); + } + + @TestTemplate + public void testJoinsWithBucketWithOneSideReducing() { + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (bucket(4, id))" + + "TBLPROPERTIES (%s)", + tableName, tablePropsAsString(TABLE_PROPERTIES)); + + sql( + "INSERT INTO %s VALUES " + + "(1L, 100, 'software')," + + "(2L, 101, 'hr')," + + "(3L, 102, 'operation')," + + "(4L, 103, 'sales')," + + "(5L, 104, 'marketing')," + + "(6L, 105, 'pr')", + tableName); + + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (bucket(8, id))" + + "TBLPROPERTIES (%s)", + tableName(OTHER_TABLE_NAME), tablePropsAsString(TABLE_PROPERTIES)); + + sql( + "INSERT INTO %s VALUES " + + "(1L, 100, 'software')," + + "(3L, 300, 'hardware')," + + "(4L, 103, 'sales')," + + "(5L, 104, 'marketing')," + + "(6L, 105, 'pr')", + tableName(OTHER_TABLE_NAME)); + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT * " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id " + + "ORDER BY t1.id, t1.int_col, t1.dep, t2.id, t2.int_col, t2.dep", + tableName, + tableName(OTHER_TABLE_NAME)); + } + + @TestTemplate + public void testJoinsIncompatibleBucketNumbers() { + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (bucket(3, id))" + + "TBLPROPERTIES (%s)", + tableName, tablePropsAsString(TABLE_PROPERTIES)); + + sql( + "INSERT INTO %s VALUES " + + "(1L, 100, 'software')," + + "(2L, 101, 'hr')," + + "(3L, 102, 'operation')," + + "(4L, 103, 'sales')," + + "(5L, 104, 'marketing')," + + "(6L, 105, 'pr')", + tableName); + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (bucket(5, id))" + + "TBLPROPERTIES (%s)", + tableName(OTHER_TABLE_NAME), tablePropsAsString(TABLE_PROPERTIES)); + + sql( + "INSERT INTO %s VALUES " + + "(1L, 100, 'software')," + + "(3L, 300, 'hardware')," + + "(4L, 103, 'sales')," + + "(5L, 104, 'marketing')," + + "(6L, 105, 'pr')", + tableName(OTHER_TABLE_NAME)); + + assertPartitioningAwarePlan( + 3, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT * " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id " + + "ORDER BY t1.id, t1.int_col, t1.dep, t2.id, t2.int_col, t2.dep", + tableName, + tableName(OTHER_TABLE_NAME)); + } + + @TestTemplate + public void testAggregates() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT, int_col INT, dep STRING)" + + "USING iceberg " + + "PARTITIONED BY (dep, bucket(8, int_col))" + + "TBLPROPERTIES (%s)", + tableName, tablePropsAsString(TABLE_PROPERTIES)); + + // write to the table 3 times to generate 3 files per partition + Table table = validationCatalog.loadTable(tableIdent); + Dataset dataDF = randomDataDF(table.schema(), 100); + append(tableName, dataDF); + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT COUNT (DISTINCT id) AS count FROM %s GROUP BY dep, int_col ORDER BY count", + tableName, + tableName(OTHER_TABLE_NAME)); + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT COUNT (DISTINCT id) AS count FROM %s GROUP BY dep ORDER BY count", + tableName, + tableName(OTHER_TABLE_NAME)); + } + + @TestTemplate + public void testJoinsHourToDays() throws NoSuchTableException { + String createTableStmt = + "CREATE TABLE %s (" + + "id BIGINT, int_col INT,dep STRING,timestamp_col TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (days(timestamp_col)) " + + "TBLPROPERTIES (%s)"; + + sql(createTableStmt, tableName, tablePropsAsString(TABLE_PROPERTIES)); + + sql( + "INSERT INTO %s VALUES " + + "(1L, 100, 'software', TIMESTAMP('2024-11-11 10:00:00'))," + + "(2L, 101, 'hr', TIMESTAMP('2024-11-10 09:00:00'))," + + "(3L, 102, 'operation', TIMESTAMP('2024-11-10 11:00:00'))," + + "(4L, 103, 'sales', TIMESTAMP('2024-11-10 10:00:00'))," + + "(5L, 104, 'marketing', TIMESTAMP('2024-11-11 10:00:00'))," + + "(6L, 105, 'pr', TIMESTAMP('2024-11-10 10:00:00'))", + tableName); + + String create2ndTableStmt = + "CREATE TABLE %s (" + + "id BIGINT, int_col INT, dep STRING, timestamp_col TIMESTAMP) " + + "USING iceberg " + + "PARTITIONED BY (hours(timestamp_col)) " + + "TBLPROPERTIES (%s)"; + + String otherTableName = tableName(OTHER_TABLE_NAME); + + sql(create2ndTableStmt, otherTableName, tablePropsAsString(TABLE_PROPERTIES)); + + sql( + "INSERT INTO %s VALUES " + + "(1L, 100, 'software', TIMESTAMP('2024-11-11 10:00:00'))," + + "(3L, 102, 'operation', TIMESTAMP('2024-11-10 11:00:00'))," + + "(5L, 104, 'marketing', TIMESTAMP('2024-11-11 10:00:00'))," + + "(5L, 104, 'marketing', TIMESTAMP('2024-11-11 10:00:00'))," + + "(6L, 105, 'pr', TIMESTAMP('2024-11-10 10:00:00'))", + otherTableName); + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT * " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id and t1.timestamp_col = t2.timestamp_col " + + "ORDER BY t1.id, t1.int_col, t1.dep, t2.id, t2.int_col, t2.dep", + tableName, + otherTableName); + } + + private void checkJoin(String sourceColumnName, String sourceColumnType, String transform) + throws NoSuchTableException { + + String createTableStmt = + "CREATE TABLE %s (id BIGINT, salary INT, %s %s)" + + "USING iceberg " + + "PARTITIONED BY (%s)" + + "TBLPROPERTIES (%s)"; + + sql( + createTableStmt, + tableName, + sourceColumnName, + sourceColumnType, + transform, + tablePropsAsString(TABLE_PROPERTIES)); + configurePlanningMode(tableName, planningMode); + + sql( + createTableStmt, + tableName(OTHER_TABLE_NAME), + sourceColumnName, + sourceColumnType, + transform, + tablePropsAsString(TABLE_PROPERTIES)); + configurePlanningMode(tableName(OTHER_TABLE_NAME), planningMode); + + Table table = validationCatalog.loadTable(tableIdent); + Dataset dataDF = randomDataDF(table.schema(), 200); + append(tableName, dataDF); + append(tableName(OTHER_TABLE_NAME), dataDF); + + assertPartitioningAwarePlan( + 1, /* expected num of shuffles with SPJ */ + 3, /* expected num of shuffles without SPJ */ + "SELECT t1.id, t1.salary, t1.%s " + + "FROM %s t1 " + + "INNER JOIN %s t2 " + + "ON t1.id = t2.id AND t1.%s = t2.%s " + + "ORDER BY t1.id, t1.%s, t1.salary", + sourceColumnName, + tableName, + tableName(OTHER_TABLE_NAME), + sourceColumnName, + sourceColumnName, + sourceColumnName); + } + + private void assertPartitioningAwarePlan( + int expectedNumShufflesWithSPJ, + int expectedNumShufflesWithoutSPJ, + String query, + Object... args) { + + AtomicReference> rowsWithSPJ = new AtomicReference<>(); + AtomicReference> rowsWithoutSPJ = new AtomicReference<>(); + + withSQLConf( + ENABLED_SPJ_SQL_CONF, + () -> { + String plan = executeAndKeepPlan(query, args).toString(); + int actualNumShuffles = StringUtils.countMatches(plan, "Exchange"); + assertThat(actualNumShuffles) + .as("Number of shuffles with enabled SPJ must match") + .isEqualTo(expectedNumShufflesWithSPJ); + + rowsWithSPJ.set(sql(query, args)); + }); + + withSQLConf( + DISABLED_SPJ_SQL_CONF, + () -> { + String plan = executeAndKeepPlan(query, args).toString(); + int actualNumShuffles = StringUtils.countMatches(plan, "Exchange"); + assertThat(actualNumShuffles) + .as("Number of shuffles with disabled SPJ must match") + .isEqualTo(expectedNumShufflesWithoutSPJ); + + rowsWithoutSPJ.set(sql(query, args)); + }); + + assertEquals("SPJ should not change query output", rowsWithoutSPJ.get(), rowsWithSPJ.get()); + } + + private Dataset randomDataDF(Schema schema, int numRows) { + Iterable rows = RandomData.generateSpark(schema, numRows, 0); + JavaRDD rowRDD = sparkContext.parallelize(Lists.newArrayList(rows)); + StructType rowSparkType = SparkSchemaUtil.convert(schema); + Preconditions.checkArgument( + spark instanceof org.apache.spark.sql.classic.SparkSession, + "Expected instance of org.apache.spark.sql.classic.SparkSession, but got: %s", + spark.getClass().getName()); + + return ((org.apache.spark.sql.classic.SparkSession) spark) + .internalCreateDataFrame(JavaRDD.toRDD(rowRDD), rowSparkType, false); + } + + private void append(String table, Dataset df) throws NoSuchTableException { + // fanout writes are enabled as write-time clustering is not supported without Spark extensions + df.coalesce(1).writeTo(table).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java new file mode 100644 index 000000000000..905516bff92c --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java @@ -0,0 +1,341 @@ +/* + * 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.sql; + +import static org.apache.iceberg.Files.localInput; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.ChecksumFileSystem; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.encryption.Ciphers; +import org.apache.iceberg.encryption.UnitestKMS; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.SeekableInputStream; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.types.Types; +import org.apache.parquet.crypto.ParquetCryptoRuntimeException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.mockito.internal.util.collections.Iterables; + +public class TestTableEncryption extends CatalogTestBase { + private static Map appendCatalogEncryptionProperties(Map props) { + Map newProps = Maps.newHashMap(); + newProps.putAll(props); + newProps.put(CatalogProperties.ENCRYPTION_KMS_IMPL, UnitestKMS.class.getCanonicalName()); + return newProps; + } + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + appendCatalogEncryptionProperties(SparkCatalogConfig.HIVE.properties()) + } + }; + } + + @BeforeEach + public void createTables() { + sql( + "CREATE TABLE %s (id bigint, data string, float float) USING iceberg " + + "TBLPROPERTIES ( " + + "'encryption.key-id'='%s', 'format-version'='3')", + tableName, UnitestKMS.MASTER_KEY_NAME1); + + sql("INSERT INTO %s VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', float('NaN'))", tableName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testSelect() { + List expected = + ImmutableList.of(row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", Float.NaN)); + + assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); + } + + private static List currentDataFiles(Table table) { + return Streams.stream(table.newScan().planFiles()) + .map(FileScanTask::file) + .collect(Collectors.toList()); + } + + @TestTemplate + public void testRefresh() { + validationCatalog.initialize(catalogName, catalogConfig); + Table table = validationCatalog.loadTable(tableIdent); + + assertThat(currentDataFiles(table)).isNotEmpty(); + + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0), (6, 'f', float('NaN'))", tableName); + + table.refresh(); + assertThat(currentDataFiles(table)).isNotEmpty(); + } + + @TestTemplate + public void testAppendTransaction() { + validationCatalog.initialize(catalogName, catalogConfig); + Table table = validationCatalog.loadTable(tableIdent); + + List dataFiles = currentDataFiles(table); + Transaction transaction = table.newTransaction(); + AppendFiles append = transaction.newAppend(); + + // add an arbitrary datafile + append.appendFile(dataFiles.get(0)); + append.commit(); + transaction.commitTransaction(); + + assertThat(currentDataFiles(table)).hasSize(dataFiles.size() + 1); + } + + @TestTemplate + public void testConcurrentAppendTransactions() { + validationCatalog.initialize(catalogName, catalogConfig); + Table table = validationCatalog.loadTable(tableIdent); + + List dataFiles = currentDataFiles(table); + Transaction transaction = table.newTransaction(); + AppendFiles append = transaction.newAppend(); + + // add an arbitrary datafile + append.appendFile(dataFiles.get(0)); + + // append to the table in the meantime. use a separate load to avoid shared operations + validationCatalog.loadTable(tableIdent).newFastAppend().appendFile(dataFiles.get(0)).commit(); + + append.commit(); + transaction.commitTransaction(); + + assertThat(currentDataFiles(table)).hasSize(dataFiles.size() + 2); + } + + // See CatalogTests#testConcurrentReplaceTransactions + @TestTemplate + public void testConcurrentReplaceTransactions() { + validationCatalog.initialize(catalogName, catalogConfig); + + Table table = validationCatalog.loadTable(tableIdent); + DataFile file = currentDataFiles(table).get(0); + Schema schema = table.schema(); + + // Write data for a replace transaction that will be committed later + Transaction secondReplace = + validationCatalog + .buildTable(tableIdent, schema) + .withProperty("encryption.key-id", UnitestKMS.MASTER_KEY_NAME1) + .replaceTransaction(); + secondReplace.newFastAppend().appendFile(file).commit(); + + // Commit another replace transaction first + Transaction firstReplace = + validationCatalog + .buildTable(tableIdent, schema) + .withProperty("encryption.key-id", UnitestKMS.MASTER_KEY_NAME1) + .replaceTransaction(); + firstReplace.newFastAppend().appendFile(file).commit(); + firstReplace.commitTransaction(); + + secondReplace.commitTransaction(); + + Table afterSecondReplace = validationCatalog.loadTable(tableIdent); + assertThat(currentDataFiles(afterSecondReplace)).hasSize(1); + } + + @TestTemplate + public void testInsertAndDelete() { + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0), (6, 'f', float('NaN'))", tableName); + + List expected = + ImmutableList.of( + row(1L, "a", 1.0F), + row(2L, "b", 2.0F), + row(3L, "c", Float.NaN), + row(4L, "d", 4.0F), + row(5L, "e", 5.0F), + row(6L, "f", Float.NaN)); + + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT * FROM %s ORDER BY id", tableName)); + + sql("DELETE FROM %s WHERE id < 4", tableName); + + expected = ImmutableList.of(row(4L, "d", 4.0F), row(5L, "e", 5.0F), row(6L, "f", Float.NaN)); + + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testMetadataTamperproofing() throws IOException { + ChecksumFileSystem fs = ((ChecksumFileSystem) FileSystem.newInstance(new Configuration())); + catalog.initialize(catalogName, catalogConfig); + + Table table = catalog.loadTable(tableIdent); + TableMetadata currentMetadata = ((HasTableOperations) table).operations().current(); + Path metadataFile = new Path(currentMetadata.metadataFileLocation()); + Path previousMetadataFile = new Path(Iterables.firstOf(currentMetadata.previousFiles()).file()); + + // manual FS tampering: replacing the current metadata file with a previous one + Path crcPath = fs.getChecksumFile(metadataFile); + fs.delete(crcPath, false); + fs.delete(metadataFile, false); + fs.rename(previousMetadataFile, metadataFile); + + assertThatThrownBy(() -> catalog.loadTable(tableIdent)) + .hasMessageContaining( + String.format( + "The current metadata file %s might have been modified. Hash of metadata loaded from storage differs from HMS-stored metadata hash.", + metadataFile)); + } + + @TestTemplate + public void testKeyDelete() { + assertThatThrownBy( + () -> sql("ALTER TABLE %s UNSET TBLPROPERTIES (`encryption.key-id`)", tableName)) + .hasMessageContaining("Cannot remove key in encrypted table"); + } + + @TestTemplate + public void testKeyAlter() { + assertThatThrownBy( + () -> sql("ALTER TABLE %s SET TBLPROPERTIES ('encryption.key-id'='abcd')", tableName)) + .hasMessageContaining("Cannot modify key in encrypted table"); + } + + @TestTemplate + public void testDirectDataFileRead() { + List dataFileTable = + sql("SELECT file_path FROM %s.%s", tableName, MetadataTableType.ALL_DATA_FILES); + List dataFiles = + Streams.concat(dataFileTable.stream()) + .map(row -> (String) row[0]) + .collect(Collectors.toList()); + + if (dataFiles.isEmpty()) { + throw new RuntimeException("No data files found for table " + tableName); + } + + Schema schema = new Schema(optional(0, "id", Types.IntegerType.get())); + for (String filePath : dataFiles) { + assertThatThrownBy( + () -> + Parquet.read(localInput(filePath)) + .project(schema) + .callInit() + .build() + .iterator() + .next()) + .isInstanceOf(ParquetCryptoRuntimeException.class) + .hasMessageContaining("Trying to read file with encrypted footer. No keys available"); + } + } + + @TestTemplate + public void testManifestEncryption() throws IOException { + List manifestFileTable = + sql("SELECT path FROM %s.%s", tableName, MetadataTableType.MANIFESTS); + + List manifestFiles = + Streams.concat(manifestFileTable.stream()) + .map(row -> (String) row[0]) + .collect(Collectors.toList()); + + if (manifestFiles.isEmpty()) { + throw new RuntimeException("No manifest files found for table " + tableName); + } + + String metadataFolderPath = null; + + // Check encryption of manifest files + for (String manifestFilePath : manifestFiles) { + checkMetadataFileEncryption(localInput(manifestFilePath)); + + if (metadataFolderPath == null) { + metadataFolderPath = new File(manifestFilePath).getParent().replaceFirst("file:", ""); + } + } + + if (metadataFolderPath == null) { + throw new RuntimeException("No metadata folder found for table " + tableName); + } + + // Find manifest list and metadata files; check their encryption + File[] listOfMetadataFiles = new File(metadataFolderPath).listFiles(); + boolean foundManifestListFile = false; + + for (File metadataFile : listOfMetadataFiles) { + if (metadataFile.getName().startsWith("snap-")) { + foundManifestListFile = true; + checkMetadataFileEncryption(localInput(metadataFile)); + } + } + + if (!foundManifestListFile) { + throw new RuntimeException("No manifest list files found for table " + tableName); + } + } + + private void checkMetadataFileEncryption(InputFile file) throws IOException { + SeekableInputStream stream = file.newStream(); + byte[] magic = new byte[4]; + stream.read(magic); + stream.close(); + assertThat(magic).isEqualTo(Ciphers.GCM_STREAM_MAGIC_STRING.getBytes(StandardCharsets.UTF_8)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java new file mode 100644 index 000000000000..f721ff3decf0 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java @@ -0,0 +1,243 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.sql.Timestamp; +import java.time.LocalDateTime; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +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.CatalogTestBase; +import org.apache.iceberg.spark.SparkSessionCatalog; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.joda.time.DateTime; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestTimestampWithoutZone extends CatalogTestBase { + + private static final String NEW_TABLE_NAME = "created_table"; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.required(3, "tsz", Types.TimestampType.withZone())); + + private final List values = + ImmutableList.of( + row(1L, toLocalDateTime("2021-01-01T00:00:00.0"), toTimestamp("2021-02-01T00:00:00.0")), + row(2L, toLocalDateTime("2021-01-01T00:00:00.0"), toTimestamp("2021-02-01T00:00:00.0")), + row(3L, toLocalDateTime("2021-01-01T00:00:00.0"), toTimestamp("2021-02-01T00:00:00.0"))); + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + "spark_catalog", + SparkSessionCatalog.class.getName(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "parquet-enabled", "true", + "cache-enabled", "false") + } + }; + } + + @BeforeEach + public void createTables() { + validationCatalog.createTable(tableIdent, SCHEMA); + } + + @AfterEach + public void removeTables() { + validationCatalog.dropTable(tableIdent, true); + sql("DROP TABLE IF EXISTS %s", NEW_TABLE_NAME); + } + + /* + Spark does not really care about the timezone, it will just convert it + + spark-sql (default)> CREATE TABLE t1 (tz TIMESTAMP, ntz TIMESTAMP_NTZ); + Time taken: 1.925 seconds + + spark-sql (default)> INSERT INTO t1 VALUES(timestamp '2020-01-01T00:00:00+02:00', timestamp_ntz '2020-01-01T00:00:00'); + Time taken: 1.355 seconds + spark-sql (default)> INSERT INTO t1 VALUES(timestamp_ntz '2020-01-01T00:00:00+02:00', timestamp_ntz '2020-01-01T00:00:00'); + Time taken: 0.129 seconds + spark-sql (default)> INSERT INTO t1 VALUES(timestamp_ntz '2020-01-01T00:00:00+02:00', timestamp '2020-01-01T00:00:00'); + Time taken: 0.125 seconds + spark-sql (default)> INSERT INTO t1 VALUES(timestamp '2020-01-01T00:00:00+02:00', timestamp '2020-01-01T00:00:00'); + Time taken: 0.122 seconds + + spark-sql (default)> select * from t1; + 2020-01-01 00:00:00 2020-01-01 00:00:00 + 2020-01-01 00:00:00 2020-01-01 00:00:00 + 2019-12-31 23:00:00 2020-01-01 00:00:00 + 2019-12-31 23:00:00 2020-01-01 00:00:00 + Time taken: 0.32 seconds, Fetched 4 row(s) + + spark-sql (default)> SELECT count(1) FROM t1 JOIN t1 as t2 ON t1.tz = t2.ntz; + 8 + */ + + @TestTemplate + public void testAppendTimestampWithoutZone() { + // Both NTZ + sql( + "INSERT INTO %s VALUES %s", + tableName, + rowToSqlValues( + ImmutableList.of( + row( + 1L, + toLocalDateTime("2021-01-01T00:00:00.0"), + toLocalDateTime("2021-02-01T00:00:00.0"))))); + } + + @TestTemplate + public void testAppendTimestampWithZone() { + // Both TZ + sql( + "INSERT INTO %s VALUES %s", + tableName, + rowToSqlValues( + ImmutableList.of( + row( + 1L, + toTimestamp("2021-01-01T00:00:00.0"), + toTimestamp("2021-02-01T00:00:00.0"))))); + } + + @TestTemplate + public void testCreateAsSelectWithTimestampWithoutZone() { + sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); + + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", NEW_TABLE_NAME, tableName); + + assertThat(scalarSql("SELECT count(*) FROM %s", NEW_TABLE_NAME)) + .as("Should have " + values.size() + " row") + .isEqualTo((long) values.size()); + + assertEquals( + "Row data should match expected", + sql("SELECT * FROM %s ORDER BY id", tableName), + sql("SELECT * FROM %s ORDER BY id", NEW_TABLE_NAME)); + } + + @TestTemplate + public void testCreateNewTableShouldHaveTimestampWithZoneIcebergType() { + sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); + + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", NEW_TABLE_NAME, tableName); + + assertThat(scalarSql("SELECT count(*) FROM %s", NEW_TABLE_NAME)) + .as("Should have " + values.size() + " row") + .isEqualTo((long) values.size()); + + assertEquals( + "Data from created table should match data from base table", + sql("SELECT * FROM %s ORDER BY id", tableName), + sql("SELECT * FROM %s ORDER BY id", NEW_TABLE_NAME)); + + Table createdTable = validationCatalog.loadTable(TableIdentifier.of("default", NEW_TABLE_NAME)); + assertFieldsType(createdTable.schema(), Types.TimestampType.withoutZone(), "ts"); + assertFieldsType(createdTable.schema(), Types.TimestampType.withZone(), "tsz"); + } + + @TestTemplate + public void testCreateNewTableShouldHaveTimestampWithoutZoneIcebergType() { + spark + .sessionState() + .catalogManager() + .currentCatalog() + .initialize(catalog.name(), new CaseInsensitiveStringMap(catalogConfig)); + sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); + + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", NEW_TABLE_NAME, tableName); + + assertThat(scalarSql("SELECT count(*) FROM %s", NEW_TABLE_NAME)) + .as("Should have " + values.size() + " row") + .isEqualTo((long) values.size()); + + assertEquals( + "Row data should match expected", + sql("SELECT * FROM %s ORDER BY id", tableName), + sql("SELECT * FROM %s ORDER BY id", NEW_TABLE_NAME)); + Table createdTable = validationCatalog.loadTable(TableIdentifier.of("default", NEW_TABLE_NAME)); + assertFieldsType(createdTable.schema(), Types.TimestampType.withoutZone(), "ts"); + assertFieldsType(createdTable.schema(), Types.TimestampType.withZone(), "tsz"); + } + + private Timestamp toTimestamp(String value) { + return new Timestamp(DateTime.parse(value).getMillis()); + } + + private LocalDateTime toLocalDateTime(String value) { + return LocalDateTime.parse(value); + } + + private String rowToSqlValues(List rows) { + List rowValues = + rows.stream() + .map( + row -> { + List columns = + Arrays.stream(row) + .map( + value -> { + if (value instanceof Long) { + return value.toString(); + } else if (value instanceof Timestamp) { + return String.format("timestamp '%s'", value); + } else if (value instanceof LocalDateTime) { + return String.format("timestamp_ntz '%s'", value); + } + throw new RuntimeException("Type is not supported"); + }) + .collect(Collectors.toList()); + return "(" + Joiner.on(",").join(columns) + ")"; + }) + .collect(Collectors.toList()); + return Joiner.on(",").join(rowValues); + } + + private void assertFieldsType(Schema actual, Type.PrimitiveType expected, String... fields) { + actual + .select(fields) + .asStruct() + .fields() + .forEach(field -> assertThat(field.type()).isEqualTo(expected)); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java new file mode 100644 index 000000000000..7d9dfe95efc0 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWrites.java @@ -0,0 +1,21 @@ +/* + * 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.sql; + +public class TestUnpartitionedWrites extends UnpartitionedWritesTestBase {} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java new file mode 100644 index 000000000000..4991a9fd553e --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java @@ -0,0 +1,61 @@ +/* + * 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.sql; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestUnpartitionedWritesToBranch extends UnpartitionedWritesTestBase { + + private static final String BRANCH = "test"; + + @Override + @BeforeEach + public void createTables() { + super.createTables(); + Table table = validationCatalog.loadTable(tableIdent); + table.manageSnapshots().createBranch(BRANCH, table.currentSnapshot().snapshotId()).commit(); + sql("REFRESH TABLE " + tableName); + } + + @Override + protected String commitTarget() { + return String.format("%s.branch_%s", tableName, BRANCH); + } + + @Override + protected String selectTarget() { + return String.format("%s VERSION AS OF '%s'", tableName, BRANCH); + } + + @TestTemplate + public void testInsertIntoNonExistingBranchFails() { + assertThatThrownBy( + () -> sql("INSERT INTO %s.branch_not_exist VALUES (4, 'd'), (5, 'e')", tableName)) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot use branch (does not exist): not_exist"); + } +} diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java new file mode 100644 index 000000000000..756f9c755ca0 --- /dev/null +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java @@ -0,0 +1,191 @@ +/* + * 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.sql; + +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.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.functions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class UnpartitionedWritesTestBase extends CatalogTestBase { + + @BeforeEach + public void createTables() { + sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); + sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + @TestTemplate + public void testInsertAppend() { + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); + + sql("INSERT INTO %s VALUES (4, 'd'), (5, 'e')", commitTarget()); + + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 5 rows") + .isEqualTo(5L); + + List expected = + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d"), row(5L, "e")); + + assertEquals( + "Row data should match expected", + expected, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testInsertOverwrite() { + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); + + sql("INSERT OVERWRITE %s VALUES (4, 'd'), (5, 'e')", commitTarget()); + + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 2 rows after overwrite") + .isEqualTo(2L); + + List expected = ImmutableList.of(row(4L, "d"), row(5L, "e")); + + assertEquals( + "Row data should match expected", + expected, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testInsertAppendAtSnapshot() { + assumeThat(tableName.equals(commitTarget())).isTrue(); + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + String prefix = "snapshot_id_"; + + assertThatThrownBy( + () -> + 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"); + } + + @TestTemplate + public void testInsertOverwriteAtSnapshot() { + assumeThat(tableName.equals(commitTarget())).isTrue(); + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + String prefix = "snapshot_id_"; + + assertThatThrownBy( + () -> + sql( + "INSERT OVERWRITE %s.%s VALUES (4, 'd'), (5, 'e')", + tableName, prefix + snapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot write to table at a specific snapshot"); + } + + @TestTemplate + public void testDataFrameV2Append() throws NoSuchTableException { + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); + + List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); + Dataset ds = spark.createDataFrame(data, SimpleRecord.class); + + ds.writeTo(commitTarget()).append(); + + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 5 rows after insert") + .isEqualTo(5L); + + List expected = + ImmutableList.of(row(1L, "a"), row(2L, "b"), row(3L, "c"), row(4L, "d"), row(5L, "e")); + + assertEquals( + "Row data should match expected", + expected, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testDataFrameV2DynamicOverwrite() throws NoSuchTableException { + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); + + List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); + Dataset ds = spark.createDataFrame(data, SimpleRecord.class); + + ds.writeTo(commitTarget()).overwritePartitions(); + + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 2 rows after overwrite") + .isEqualTo(2L); + + List expected = ImmutableList.of(row(4L, "d"), row(5L, "e")); + + assertEquals( + "Row data should match expected", + expected, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testDataFrameV2Overwrite() throws NoSuchTableException { + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 3 rows") + .isEqualTo(3L); + + List data = ImmutableList.of(new SimpleRecord(4, "d"), new SimpleRecord(5, "e")); + Dataset ds = spark.createDataFrame(data, SimpleRecord.class); + + ds.writeTo(commitTarget()).overwrite(functions.col("id").$less$eq(3)); + + assertThat(scalarSql("SELECT count(*) FROM %s", selectTarget())) + .as("Should have 2 rows after overwrite") + .isEqualTo(2L); + + List expected = ImmutableList.of(row(4L, "d"), row(5L, "e")); + + assertEquals( + "Row data should match expected", + expected, + sql("SELECT * FROM %s ORDER BY id", selectTarget())); + } +} diff --git a/spark/v4.0/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet b/spark/v4.0/spark/src/test/resources/decimal_dict_and_plain_encoding.parquet new file mode 100644 index 0000000000000000000000000000000000000000..48b3bd1bf24f13150e2e283bf61f5d4776c3754a GIT binary patch literal 3685 zcmd6qd2~}%9>-tuZt`xEv<-!nq@;^o>6Ruby``mT`%+rcjV5K6tp#eKGlfu=0%a)( zRAB~!tm0M?1=*&`fFKCP3dq433wj2IL8gf4IG_U#=n<4N_xo_pnWOySpY#66=lz!Z zyZi6=;+3l%CSq!JnjZA2Fa<=D5GDw%7D2G71ZwNQ)fOrk1XJxnr%|BNK_4>;#P!2{NXlWm^%kBUmVLpGMubKMf)%R3jTR1T`)!6{tyUK4+Zqi z`vLjKaX?(+XIP>WDG;jFUjilNK|p9r8xRqZ1^6bG0IoYn0C8~vP~AHiFfUF6k}lr^ zq}Y)_@>g08l75v1RMd3$4Gd_&hRe_y>RF9Q4XqQQ3(d`kuDF94)im4o4s=Cl%m|^2 z8VzKg90J6jqd;n9F;KZk0*u#o0Md8=0BqCZfb@^tK{Hwxg z=d*tE8Z5C}S&fP$oMm1$4cb)7Eu?BR*1PKXG(HSs+|O>QXvVjnVoaW|zd_Itzl^5l z3U*6PLr3?9X~+&0Oc7UR0Q%uc@X$TSI@KgZWUYj6)n0V#cv%exx*!s8RR@5SlD`AS zq?dt=70rNnu@6wKJ^=V8MgWm}@;Hb-zzVgrXhtTwcB%gaT~!NK~Eod$f34M0Dk9B_1HaFFO5tX)l`;?vQC{l+5b zbd5|<5$AjsBNBZv-X$Mv*U;1()16)(;H`@Y5ci653S(EZ8V#kMMg)*lFT#|6X(`a_ zQYFGwS=cR+->rCfB%GZKUHmo1g_42Ka~&BVGeN^Q>Nte#`^>H(kDX?%{IFTaV}7NZ zSdW@-_!0!jS!73m3>k;tcw`Kr?5gw^!u4tBoR40 zFdfiqveVBO#?N|0T7148%{^c5L-VwX+*=~q{~A=$88@H`apRbbO9DA$K2e9j)H}E=-}htX)O2&#-op`l4RJ z+BS{5!^yrgyM zvL~0XSlRZ})2mkh?)Pif2G%{Ze#6F1o3}i>^||L?c=4t7m$$vL{ngiY?0kLK?mc_= z{h{NHH~0VXz`?f;9e(@B(RYp=?>zBt*Lx>Vo#uVeErO5%2J_!d&} z$wr``7Po?yEHMCms+9649yLM=V@kS}I^rSO2PzdS@mdZf{U#;(W-eO|DeJx)FvN@p zd}A_!YVtf#O4b57v>UM9_W|WIm3k>H8xAS|m{Lo79V>-2bcB-YQU@w|DaE)ON=M;! zK!0ik5a<5@aNbi2W%ckAkcxK{a**)jJQ>lxn?J$9*WN?^q4`1KfT30?ZY!1IHl%9| z+}Rh*t|HzG%*zw;swiA1$3dk&%5MkI$l9gcPAOg~B~^cOKsof^lL61FUC5w~%7=le zk8%t9u0*mdzo0??2FGFe#Xe99Y4C#tNF_%`0zS_LAY|}hU~qCBpl`bgBzTSjnX{E5 zkmss~)ccZ`gLqu5hgXDoW#}M#Zz>;wP{#Wtp@)&;gFj#$Y8p}SF)p@lffL_9?n#%O zy?EjO*oHXK{oVws4qJhW&y@OayLb}82TkDFho-M+KyY#I9e50A|0_&!!&o<=hG83E zN^H)DDXjS_AcYSDjMvWtIV~2zbLbLKNxuN3x>FeFGUV6NYmtcI8-w~tf+-6KB(shu zbE4nGIz-|y@?t<-!1% zU3xE6lk6Nn*OB5K%&w*e|2PN{yO~kLk5LeWgdof)^ZY6_r3rZhQY4j@T4fUu>T z5ieyUi_wy_n=GIq;SH=^BECUPkZ98D5h!+vPppJ3Mz3g3lL(O zixXD%LLyYQ0#?YY=z=OlkxE#hhPsl{xIn{_dN7Wm>%uS&8j8W&e_)*&>Ymh!rn&`) zqd(9IZ|{M6c=zjzO~X;*D`Vb!1X0r>m|w>?cMbfc@K9{TZRVxa)jSibq=|O4aXayW zqf23eo@C8nenNv=wmJ)W*%`(HYEo`zc9E2sSr0z|+qT0nC=X+ox|DB0DSio}RPJLn zl=kmhgvgGA>!B<9kVnHEwM)cnXMrGU3r^wzXu9_)Zl}3}@$Eti+E|B%SZ6aoKS*u- zbAiPD1^ZTV1Y2NmTu;1cx0*sMS%TJRF{`IdF{#7bwc+O*!n^H>J57o7S5&!2CG20; z!g*zaC6E6r5lm`<|NI0amjuB?ENNMSSvza;)Vju6IE0vDOlpiLLC0My(#GMJd-B0q ztQPD4#i+4L?b<}jm#U8-SgdA)TwPr?D)-?m%zIQBcrxo{{`m02pK@7lmfiJoi7eO4 zMT$vwyJfi&#v)lR1-blRa;X@q5%}S6ob_n$K^*@0;Qa6@_9&iiMX$U*)FYJJ|M){; zh|*q`Ic2%*VgGn_3Fa7oZPUy-b+gka`~B&)O-(bJW9HU1&6+l2dW<96X1CcbRx96u MJt{%)@;{jW0iIk85dZ)H literal 0 HcmV?d00001 From 317968426afb0a254ee3b8cd930578c774ac4fdb Mon Sep 17 00:00:00 2001 From: manuzhang Date: Mon, 1 Dec 2025 15:03:07 +0800 Subject: [PATCH 129/201] Spark: Initial support for 4.1.0 --- .github/workflows/spark-ci.yml | 6 ++++- .gitignore | 2 ++ dev/stage-binaries.sh | 2 +- gradle.properties | 4 +-- gradle/libs.versions.toml | 1 + jmh.gradle | 5 ++++ settings.gradle | 12 +++++++++ spark/build.gradle | 4 +++ spark/v4.1/build.gradle | 24 ++++++++++++----- .../iceberg/DeleteFileIndexBenchmark.java | 4 +-- .../spark/MergeCardinalityCheckBenchmark.java | 4 +-- .../iceberg/spark/PlanningBenchmark.java | 4 +-- .../spark/TaskGroupPlanningBenchmark.java | 4 +-- .../sql/catalyst/analysis/ResolveViews.scala | 6 ++--- .../sql/catalyst/analysis/ViewUtil.scala | 2 +- .../IcebergSparkSqlExtensionsParser.scala | 2 +- .../iceberg/spark/extensions/TestMerge.java | 23 +--------------- .../spark/extensions/TestMetadataTables.java | 2 +- .../TestRewriteDataFilesProcedure.java | 2 +- .../TestRewriteManifestsProcedure.java | 2 +- ...stRewritePositionDeleteFilesProcedure.java | 2 +- .../iceberg/spark/extensions/TestUpdate.java | 2 +- .../action/DeleteOrphanFilesBenchmark.java | 4 +-- .../SparkParquetReadersFlatDataBenchmark.java | 4 +-- ...parkParquetReadersNestedDataBenchmark.java | 4 +-- .../SparkParquetWritersFlatDataBenchmark.java | 4 +-- ...parkParquetWritersNestedDataBenchmark.java | 4 +-- .../source/avro/AvroWritersBenchmark.java | 4 +-- ...cebergSourceFlatAvroDataReadBenchmark.java | 4 +-- ...bergSourceNestedAvroDataReadBenchmark.java | 4 +-- ...IcebergSourceFlatORCDataReadBenchmark.java | 4 +-- ...SourceNestedListORCDataWriteBenchmark.java | 4 +-- ...ebergSourceNestedORCDataReadBenchmark.java | 4 +-- ...gSourceFlatParquetDataFilterBenchmark.java | 4 +-- ...ergSourceFlatParquetDataReadBenchmark.java | 4 +-- ...rgSourceFlatParquetDataWriteBenchmark.java | 4 +-- ...ceNestedListParquetDataWriteBenchmark.java | 4 +-- ...ourceNestedParquetDataFilterBenchmark.java | 4 +-- ...gSourceNestedParquetDataReadBenchmark.java | 4 +-- ...SourceNestedParquetDataWriteBenchmark.java | 4 +-- ...IcebergSourceParquetEqDeleteBenchmark.java | 4 +-- ...SourceParquetMultiDeleteFileBenchmark.java | 4 +-- ...cebergSourceParquetPosDeleteBenchmark.java | 4 +-- ...ceParquetWithUnrelatedDeleteBenchmark.java | 4 +-- .../parquet/ParquetWritersBenchmark.java | 4 +-- ...ionaryEncodedFlatParquetDataBenchmark.java | 4 +-- ...ectorizedReadFlatParquetDataBenchmark.java | 4 +-- ...VectorizedReadParquetDecimalBenchmark.java | 4 +-- .../org/apache/iceberg/spark/BaseCatalog.java | 11 ++++++++ .../apache/iceberg/spark/SparkTableUtil.java | 3 ++- .../spark/data/SparkParquetReaders.java | 12 +++++++++ .../spark/procedures/SparkProcedures.java | 5 ++++ .../spark/source/StructInternalRow.java | 12 +++++++++ .../spark/data/TestSparkParquetReader.java | 1 + .../source/TestForwardCompatibility.java | 10 +++---- .../spark/source/TestStructuredStreaming.java | 16 ++++++------ .../spark/sql/TestSparkDefaultValues.java | 26 ------------------- 57 files changed, 175 insertions(+), 144 deletions(-) diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index be4083714d9c..ecc973e32a8d 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -72,17 +72,21 @@ jobs: strategy: matrix: jvm: [11, 17, 21] - spark: ['3.4', '3.5', '4.0'] + spark: ['3.4', '3.5', '4.0', '4.1'] scala: ['2.12', '2.13'] exclude: # Spark 3.5 is the first version not failing on Java 21 (https://issues.apache.org/jira/browse/SPARK-42369) # Full Java 21 support is coming in Spark 4 (https://issues.apache.org/jira/browse/SPARK-43831) - jvm: 11 spark: '4.0' + - jvm: 11 + spark: '4.1' - jvm: 21 spark: '3.4' - spark: '4.0' scala: '2.12' + - spark: '4.1' + scala: '2.12' env: SPARK_LOCAL_IP: localhost steps: diff --git a/.gitignore b/.gitignore index f931c10e9407..bcac4d1610fc 100644 --- a/.gitignore +++ b/.gitignore @@ -38,6 +38,8 @@ spark/v3.5/spark/benchmark/* spark/v3.5/spark-extensions/benchmark/* spark/v4.0/spark/benchmark/* spark/v4.0/spark-extensions/benchmark/* +spark/v4.1/spark/benchmark/* +spark/v4.1/spark-extensions/benchmark/* */benchmark/* __pycache__/ diff --git a/dev/stage-binaries.sh b/dev/stage-binaries.sh index ec3080575b86..50f984eb404e 100755 --- a/dev/stage-binaries.sh +++ b/dev/stage-binaries.sh @@ -20,7 +20,7 @@ SCALA_VERSION=2.12 FLINK_VERSIONS=1.20,2.0,2.1 -SPARK_VERSIONS=3.4,3.5,4.0 +SPARK_VERSIONS=3.4,3.5,4.0,4.1 KAFKA_VERSIONS=3 ./gradlew -Prelease -DscalaVersion=$SCALA_VERSION -DflinkVersions=$FLINK_VERSIONS -DsparkVersions=$SPARK_VERSIONS -DkafkaVersions=$KAFKA_VERSIONS publishApachePublicationToMavenRepository --no-parallel --no-configuration-cache diff --git a/gradle.properties b/gradle.properties index 0f70b49eb722..c0f283303ae0 100644 --- a/gradle.properties +++ b/gradle.properties @@ -18,8 +18,8 @@ jmhJsonOutputPath=build/reports/jmh/results.json jmhIncludeRegex=.* systemProp.defaultFlinkVersions=2.1 systemProp.knownFlinkVersions=1.20,2.0,2.1 -systemProp.defaultSparkVersions=4.0 -systemProp.knownSparkVersions=3.4,3.5,4.0 +systemProp.defaultSparkVersions=4.1 +systemProp.knownSparkVersions=3.4,3.5,4.0,4.1 systemProp.defaultKafkaVersions=3 systemProp.knownKafkaVersions=3 systemProp.defaultScalaVersion=2.12 diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 5e2d5435eb14..4fbba96317ce 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -87,6 +87,7 @@ snowflake-jdbc = "3.28.0" spark34 = "3.4.4" spark35 = "3.5.7" spark40 = "4.0.1" +spark41 = "4.1.0" sqlite-jdbc = "3.51.1.0" testcontainers = "2.0.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above diff --git a/jmh.gradle b/jmh.gradle index 57efb3821d8f..d2c4709bf349 100644 --- a/jmh.gradle +++ b/jmh.gradle @@ -53,6 +53,11 @@ if (sparkVersions.contains("4.0")) { jmhProjects.add(project(":iceberg-spark:iceberg-spark-extensions-4.0_2.13")) } +if (sparkVersions.contains("4.1")) { + jmhProjects.add(project(":iceberg-spark:iceberg-spark-4.1_2.13")) + jmhProjects.add(project(":iceberg-spark:iceberg-spark-extensions-4.1_2.13")) +} + configure(jmhProjects) { apply plugin: 'me.champeau.jmh' apply plugin: 'io.morethan.jmhreport' diff --git a/settings.gradle b/settings.gradle index de342dda1476..70f9343a252b 100644 --- a/settings.gradle +++ b/settings.gradle @@ -175,6 +175,18 @@ if (sparkVersions.contains("4.0")) { project(":iceberg-spark:spark-runtime-4.0_2.13").name = "iceberg-spark-runtime-4.0_2.13" } +if (sparkVersions.contains("4.1")) { + include ":iceberg-spark:spark-4.1_2.13" + include ":iceberg-spark:spark-extensions-4.1_2.13" + include ":iceberg-spark:spark-runtime-4.1_2.13" + project(":iceberg-spark:spark-4.1_2.13").projectDir = file('spark/v4.1/spark') + project(":iceberg-spark:spark-4.1_2.13").name = "iceberg-spark-4.1_2.13" + project(":iceberg-spark:spark-extensions-4.1_2.13").projectDir = file('spark/v4.1/spark-extensions') + project(":iceberg-spark:spark-extensions-4.1_2.13").name = "iceberg-spark-extensions-4.1_2.13" + project(":iceberg-spark:spark-runtime-4.1_2.13").projectDir = file('spark/v4.1/spark-runtime') + project(":iceberg-spark:spark-runtime-4.1_2.13").name = "iceberg-spark-runtime-4.1_2.13" +} + if (kafkaVersions.contains("3")) { include 'kafka-connect' project(':kafka-connect').name = 'iceberg-kafka-connect' diff --git a/spark/build.gradle b/spark/build.gradle index 75d3f899e5d6..4d4a84fd390a 100644 --- a/spark/build.gradle +++ b/spark/build.gradle @@ -31,3 +31,7 @@ if (sparkVersions.contains("3.5")) { if (sparkVersions.contains("4.0")) { apply from: file("$projectDir/v4.0/build.gradle") } + +if (sparkVersions.contains("4.1")) { + apply from: file("$projectDir/v4.1/build.gradle") +} diff --git a/spark/v4.1/build.gradle b/spark/v4.1/build.gradle index 8ebed9bd439b..14a07ac543c3 100644 --- a/spark/v4.1/build.gradle +++ b/spark/v4.1/build.gradle @@ -17,13 +17,13 @@ * under the License. */ -String sparkMajorVersion = '4.0' +String sparkMajorVersion = '4.1' String scalaVersion = '2.13' JavaVersion javaVersion = JavaVersion.current() Boolean javaVersionSupported = javaVersion == JavaVersion.VERSION_17 || javaVersion == JavaVersion.VERSION_21 if (!javaVersionSupported) { - logger.warn("Skip Spark 4.0 build which requires JDK 17 or 21 but was executed with JDK " + javaVersion) + logger.warn("Skip Spark 4.1 build which requires JDK 17 or 21 but was executed with JDK " + javaVersion) } def sparkProjects = [ @@ -51,6 +51,14 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { apply plugin: 'scala' apply plugin: 'com.github.alisiikh.scalastyle' + // Set target to JDK17 for Spark 4.1 to fix following error + // "spark/v4.1/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala:52:12: Class java.lang.Record not found" + tasks.withType(ScalaCompile.class) { + sourceCompatibility = "17" + targetCompatibility = "17" + scalaCompileOptions.additionalParameters.add("-release:17") + } + sourceSets { main { scala.srcDirs = ['src/main/scala', 'src/main/java'] @@ -74,7 +82,7 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { compileOnly libs.errorprone.annotations compileOnly libs.avro.avro - compileOnly("org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark40.get()}") { + compileOnly("org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark41.get()}") { exclude group: 'org.apache.avro', module: 'avro' exclude group: 'org.apache.arrow' exclude group: 'org.apache.parquet' @@ -84,7 +92,8 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { exclude group: 'org.roaringbitmap' } - compileOnly "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_2.13:${libs.versions.comet.get()}" + // TODO: datafusion-comet Spark 4.1 support + compileOnly "org.apache.datafusion:comet-spark-spark4.0_2.13:${libs.versions.comet.get()}" implementation libs.parquet.column implementation libs.parquet.hadoop @@ -165,7 +174,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer compileOnly project(':iceberg-core') compileOnly project(':iceberg-common') compileOnly project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") - compileOnly("org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark40.get()}") { + compileOnly("org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark41.get()}") { exclude group: 'org.apache.avro', module: 'avro' exclude group: 'org.apache.arrow' exclude group: 'org.apache.parquet' @@ -194,7 +203,8 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer testImplementation libs.avro.avro testImplementation libs.parquet.hadoop testImplementation libs.awaitility - testImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_2.13:${libs.versions.comet.get()}" + // TODO: datafusion-comet Spark 4.1 support + testImplementation "org.apache.datafusion:comet-spark-spark4.0_2.13:${libs.versions.comet.get()}" testImplementation(testFixtures(project(':iceberg-parquet'))) // Required because we remove antlr plugin dependencies from the compile configuration, see note above @@ -267,7 +277,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio } integrationImplementation "org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}" - integrationImplementation "org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark40.get()}" + integrationImplementation "org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark41.get()}" integrationImplementation libs.junit.jupiter integrationImplementation libs.junit.platform.launcher integrationImplementation libs.slf4j.simple diff --git a/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java b/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java index 9375ca3a4f46..a8b226ea1e37 100644 --- a/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java +++ b/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/DeleteFileIndexBenchmark.java @@ -54,8 +54,8 @@ /** * A benchmark that evaluates the delete file index build and lookup performance. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-extensions-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-extensions-4.1_2.13:jmh * -PjmhIncludeRegex=DeleteFileIndexBenchmark * -PjmhOutputPath=benchmark/iceberg-delete-file-index-benchmark.txt * diff --git a/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/MergeCardinalityCheckBenchmark.java b/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/MergeCardinalityCheckBenchmark.java index 963daa2c364c..eeea81634596 100644 --- a/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/MergeCardinalityCheckBenchmark.java +++ b/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/MergeCardinalityCheckBenchmark.java @@ -53,8 +53,8 @@ /** * A benchmark that evaluates the performance of the cardinality check in MERGE operations. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-extensions-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-extensions-4.1_2.13:jmh * -PjmhIncludeRegex=MergeCardinalityCheckBenchmark * -PjmhOutputPath=benchmark/iceberg-merge-cardinality-check-benchmark.txt * diff --git a/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java b/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java index 34d9d70e6ccb..0eff3a847e41 100644 --- a/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java +++ b/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/PlanningBenchmark.java @@ -73,8 +73,8 @@ /** * A benchmark that evaluates the job planning performance. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-extensions-4.0_2.12:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-extensions-4.1_2.13:jmh * -PjmhIncludeRegex=PlanningBenchmark * -PjmhOutputPath=benchmark/iceberg-planning-benchmark.txt * diff --git a/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java b/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java index 7c2def237874..45c95bf99741 100644 --- a/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java +++ b/spark/v4.1/spark-extensions/src/jmh/java/org/apache/iceberg/spark/TaskGroupPlanningBenchmark.java @@ -63,8 +63,8 @@ /** * A benchmark that evaluates the task group planning performance. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-extensions-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-extensions-4.1_2.13:jmh * -PjmhIncludeRegex=TaskGroupPlanningBenchmark * -PjmhOutputPath=benchmark/iceberg-task-group-planning-benchmark.txt * diff --git a/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala index ff7d20241bed..76db30a5b619 100644 --- a/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala +++ b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveViews.scala @@ -137,11 +137,9 @@ case class ResolveViews(spark: SparkSession) extends Rule[LogicalPlan] with Look private def rewriteIdentifiers( plan: LogicalPlan, catalogAndNamespace: Seq[String]): LogicalPlan = { - // Substitute CTEs and Unresolved Ordinals within the view, then rewrite unresolved functions and relations + // Rewrite unresolved functions and relations qualifyTableIdentifiers( - qualifyFunctionIdentifiers( - SubstituteUnresolvedOrdinals.apply(CTESubstitution.apply(plan)), - catalogAndNamespace), + qualifyFunctionIdentifiers(CTESubstitution.apply(plan), catalogAndNamespace), catalogAndNamespace) } diff --git a/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala index a7188837c51e..c27cb140347e 100644 --- a/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala +++ b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/ViewUtil.scala @@ -44,7 +44,7 @@ object ViewUtil { case viewCatalog: ViewCatalog => viewCatalog case _ => - throw QueryCompilationErrors.missingCatalogAbilityError(plugin, "views") + throw QueryCompilationErrors.missingCatalogViewsAbilityError(plugin) } } } diff --git a/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala index 25e056ee2d96..ac127f754a91 100644 --- a/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala +++ b/spark/v4.1/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/IcebergSparkSqlExtensionsParser.scala @@ -319,7 +319,7 @@ class IcebergParseException( val builder = new StringBuilder builder ++= "\n" ++= message start match { - case Origin(Some(l), Some(p), Some(_), Some(_), Some(_), Some(_), Some(_), _, _) => + case Origin(Some(l), Some(p), Some(_), Some(_), Some(_), Some(_), Some(_), _, _, _) => builder ++= s"(line $l, pos $p)\n" command.foreach { cmd => val (above, below) = cmd.split("\n").splitAt(l) diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java index 9b7ed8f9be95..dd1a5b74aaf4 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java @@ -2567,16 +2567,6 @@ public void testMergeWithInvalidAssignmentsAnsi() { .isInstanceOf(SparkRuntimeException.class) .hasMessageContaining( "[NOT_NULL_ASSERT_VIOLATION] NULL value appeared in non-nullable field"); - assertThatThrownBy( - () -> - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.s = s.c2", - commitTarget())) - .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Cannot find data for the output column `s`.`n2`"); assertThatThrownBy( () -> sql( @@ -2634,17 +2624,6 @@ public void testMergeWithInvalidAssignmentsStrict() { commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot safely cast `s`.`n1` \"VOID\" to \"INT\""); - - assertThatThrownBy( - () -> - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.s = s.c2", - commitTarget())) - .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Cannot find data for the output column `s`.`n2`"); assertThatThrownBy( () -> sql( @@ -2864,7 +2843,7 @@ public void testMergeWithNonIcebergTargetTableNotSupported() { + "WHEN MATCHED THEN " + " UPDATE SET *")) .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("MERGE INTO TABLE is not supported temporarily."); + .hasMessageContaining("Table `unknown` does not support MERGE INTO TABLE"); } /** diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java index ac528d1c470e..96e002979fcf 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java @@ -31,7 +31,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.avro.generic.GenericData.Record; -import org.apache.commons.collections.ListUtils; +import org.apache.commons.collections4.ListUtils; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index 3aabd635bb69..e23a06a76d33 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -978,7 +978,7 @@ public void testRewriteDataFilesSummary() { .containsKey(CatalogProperties.APP_ID) .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") .hasEntrySatisfying( - EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("4.0")); + EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("4.1")); } @TestTemplate diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index b8dca4b2cd18..98bcd401a51b 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -322,7 +322,7 @@ public void testInvalidRewriteManifestsCases() { () -> sql("CALL %s.system.rewrite_manifests(table => 't', tAbLe => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage( - "[UNRECOGNIZED_PARAMETER_NAME] Cannot invoke routine `rewrite_manifests` because the routine call included a named argument reference for the argument named `tAbLe`, but this routine does not include any signature containing an argument with this name. Did you mean one of the following? [`table` `spec_id` `use_caching`]. SQLSTATE: 4274K"); + "[DUPLICATE_ROUTINE_PARAMETER_ASSIGNMENT.DOUBLE_NAMED_ARGUMENT_REFERENCE] Call to routine `rewrite_manifests` is invalid because it includes multiple argument assignments to the same parameter name `tAbLe`. More than one named argument referred to the same parameter. Please assign a value only once. SQLSTATE: 4274K"); assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('')", catalogName)) .isInstanceOf(IllegalArgumentException.class) diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java index 4a6d2ff1979e..cd824c191080 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java @@ -241,7 +241,7 @@ public void testRewriteSummary() throws Exception { .containsKey(CatalogProperties.APP_ID) .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") .hasEntrySatisfying( - EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("4.0")); + EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("4.1")); } private Map snapshotSummary() { diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java index 77011fd28cf0..2afbc697e178 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestUpdate.java @@ -1447,7 +1447,7 @@ public void testUpdateOnNonIcebergTableNotSupported() { assertThatThrownBy(() -> sql("UPDATE %s SET c1 = -1 WHERE c2 = 1", "testtable")) .isInstanceOf(UnsupportedOperationException.class) - .hasMessage("UPDATE TABLE is not supported temporarily."); + .hasMessageContaining("Table `unknown` does not support UPDATE TABLE."); } @TestTemplate diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java index e1d9ac18dac1..231bb7c619f4 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java @@ -59,8 +59,8 @@ /** * A benchmark that evaluates the performance of remove orphan files action in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=DeleteOrphanFilesBenchmark * -PjmhOutputPath=benchmark/delete-orphan-files-benchmark-results.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java index 3dbee5dfd0f5..7f5d70171564 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersFlatDataBenchmark.java @@ -58,8 +58,8 @@ * A benchmark that evaluates the performance of reading Parquet data with a flat schema using * Iceberg and Spark Parquet readers. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=SparkParquetReadersFlatDataBenchmark * -PjmhOutputPath=benchmark/spark-parquet-readers-flat-data-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java index 8487988d9e5b..e16f18b281d2 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetReadersNestedDataBenchmark.java @@ -58,8 +58,8 @@ * A benchmark that evaluates the performance of reading nested Parquet data using Iceberg and Spark * Parquet readers. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=SparkParquetReadersNestedDataBenchmark * -PjmhOutputPath=benchmark/spark-parquet-readers-nested-data-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java index 47f0b72088f5..00c361449a0f 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersFlatDataBenchmark.java @@ -51,8 +51,8 @@ * A benchmark that evaluates the performance of writing Parquet data with a flat schema using * Iceberg and Spark Parquet writers. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=SparkParquetWritersFlatDataBenchmark * -PjmhOutputPath=benchmark/spark-parquet-writers-flat-data-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java index 4df890d86164..24d7fa405148 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/data/parquet/SparkParquetWritersNestedDataBenchmark.java @@ -51,8 +51,8 @@ * A benchmark that evaluates the performance of writing nested Parquet data using Iceberg and Spark * Parquet writers. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=SparkParquetWritersNestedDataBenchmark * -PjmhOutputPath=benchmark/spark-parquet-writers-nested-data-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/AvroWritersBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/AvroWritersBenchmark.java index 4dcd58c0c4d0..5c46a740dabe 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/AvroWritersBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/AvroWritersBenchmark.java @@ -24,8 +24,8 @@ /** * A benchmark that evaluates the performance of various Iceberg writers for Avro data. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=AvroWritersBenchmark * -PjmhOutputPath=benchmark/avro-writers-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java index f0297f644a52..23f72f0dff9f 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceFlatAvroDataReadBenchmark.java @@ -40,8 +40,8 @@ * A benchmark that evaluates the performance of reading Avro data with a flat schema using Iceberg * and the built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=IcebergSourceFlatAvroDataReadBenchmark * -PjmhOutputPath=benchmark/iceberg-source-flat-avro-data-read-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java index 00d06566fbaa..fe641747dfd7 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/avro/IcebergSourceNestedAvroDataReadBenchmark.java @@ -40,8 +40,8 @@ * A benchmark that evaluates the performance of reading Avro data with a flat schema using Iceberg * and the built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=IcebergSourceNestedAvroDataReadBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-avro-data-read-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java index 593fbc955703..7b473770bf3e 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceFlatORCDataReadBenchmark.java @@ -40,8 +40,8 @@ * A benchmark that evaluates the performance of reading ORC data with a flat schema using Iceberg * and the built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=IcebergSourceFlatORCDataReadBenchmark * -PjmhOutputPath=benchmark/iceberg-source-flat-orc-data-read-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java index 0442ed02eb49..3053b3c93c07 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedListORCDataWriteBenchmark.java @@ -39,8 +39,8 @@ * A benchmark that evaluates the performance of writing nested Parquet data using Iceberg and the * built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=IcebergSourceNestedListORCDataWriteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-list-orc-data-write-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java index a64a23774eec..6faeb21b5a16 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/orc/IcebergSourceNestedORCDataReadBenchmark.java @@ -41,8 +41,8 @@ * A benchmark that evaluates the performance of reading ORC data with a flat schema using Iceberg * and the built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=IcebergSourceNestedORCDataReadBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-orc-data-read-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java index 5b7b22f5ead7..73e479053b70 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataFilterBenchmark.java @@ -43,8 +43,8 @@ * *

    The performance is compared to the built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=IcebergSourceFlatParquetDataFilterBenchmark * -PjmhOutputPath=benchmark/iceberg-source-flat-parquet-data-filter-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java index ec1514fe4297..f95de3cf7f6f 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataReadBenchmark.java @@ -39,8 +39,8 @@ * A benchmark that evaluates the performance of reading Parquet data with a flat schema using * Iceberg and the built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=IcebergSourceFlatParquetDataReadBenchmark * -PjmhOutputPath=benchmark/iceberg-source-flat-parquet-data-read-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java index 787ae389ca6b..dccd0c76647c 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceFlatParquetDataWriteBenchmark.java @@ -37,8 +37,8 @@ * A benchmark that evaluates the performance of writing Parquet data with a flat schema using * Iceberg and the built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=IcebergSourceFlatParquetDataWriteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-flat-parquet-data-write-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java index 0d17bd3e5653..383604c186ef 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedListParquetDataWriteBenchmark.java @@ -40,8 +40,8 @@ * A benchmark that evaluates the performance of writing nested Parquet data using Iceberg and the * built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=IcebergSourceNestedListParquetDataWriteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-list-parquet-data-write-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java index a5ddd060422f..c24e336c1283 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataFilterBenchmark.java @@ -43,8 +43,8 @@ * *

    The performance is compared to the built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=IcebergSourceNestedParquetDataFilterBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-parquet-data-filter-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java index 24e2d99902b4..1c365d49f274 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataReadBenchmark.java @@ -39,8 +39,8 @@ * A benchmark that evaluates the performance of reading nested Parquet data using Iceberg and the * built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=IcebergSourceNestedParquetDataReadBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-parquet-data-read-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java index eef14854c4d6..e851e6a930e4 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceNestedParquetDataWriteBenchmark.java @@ -38,8 +38,8 @@ * A benchmark that evaluates the performance of writing nested Parquet data using Iceberg and the * built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh * -PjmhIncludeRegex=IcebergSourceNestedParquetDataWriteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-nested-parquet-data-write-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetEqDeleteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetEqDeleteBenchmark.java index 3b54b448b8b5..d4555f8c6b76 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetEqDeleteBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetEqDeleteBenchmark.java @@ -27,8 +27,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with equality delete in * the Spark data source for Iceberg. * - *

    This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh + *

    This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh * -PjmhIncludeRegex=IcebergSourceParquetEqDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-eq-delete-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java index 7891890dff4a..9110ff071fbe 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetMultiDeleteFileBenchmark.java @@ -28,8 +28,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

    This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh \ + *

    This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh \ * -PjmhIncludeRegex=IcebergSourceParquetMultiDeleteFileBenchmark \ * -PjmhOutputPath=benchmark/iceberg-source-parquet-multi-delete-file-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java index 3c6dfa6bd94f..daf7c295c31e 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetPosDeleteBenchmark.java @@ -28,8 +28,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

    This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh + *

    This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh * -PjmhIncludeRegex=IcebergSourceParquetPosDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-pos-delete-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java index 01096ac79649..ab83b688f55c 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/IcebergSourceParquetWithUnrelatedDeleteBenchmark.java @@ -28,8 +28,8 @@ * A benchmark that evaluates the non-vectorized read and vectorized read with pos-delete in the * Spark data source for Iceberg. * - *

    This class uses a dataset with a flat schema. To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0:jmh + *

    This class uses a dataset with a flat schema. To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1:jmh * -PjmhIncludeRegex=IcebergSourceParquetWithUnrelatedDeleteBenchmark * -PjmhOutputPath=benchmark/iceberg-source-parquet-with-unrelated-delete-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/ParquetWritersBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/ParquetWritersBenchmark.java index 8bcd871a07da..2b8e522148ed 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/ParquetWritersBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/ParquetWritersBenchmark.java @@ -24,8 +24,8 @@ /** * A benchmark that evaluates the performance of various Iceberg writers for Parquet data. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh \ + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh \ * -PjmhIncludeRegex=ParquetWritersBenchmark \ * -PjmhOutputPath=benchmark/parquet-writers-benchmark-result.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java index 73d4f6211803..8c88a4952f2c 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java @@ -39,8 +39,8 @@ * Benchmark to compare performance of reading Parquet dictionary encoded data with a flat schema * using vectorized Iceberg read path and the built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh \ + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh \ * -PjmhIncludeRegex=VectorizedReadDictionaryEncodedFlatParquetDataBenchmark \ * -PjmhOutputPath=benchmark/results.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java index 6cf327c1cf81..429171212a77 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadFlatParquetDataBenchmark.java @@ -50,8 +50,8 @@ * Benchmark to compare performance of reading Parquet data with a flat schema using vectorized * Iceberg read path and the built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh \ + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh \ * -PjmhIncludeRegex=VectorizedReadFlatParquetDataBenchmark \ * -PjmhOutputPath=benchmark/results.txt * diff --git a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadParquetDecimalBenchmark.java b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadParquetDecimalBenchmark.java index ccf28e3fdc77..3aafe8a72ac9 100644 --- a/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadParquetDecimalBenchmark.java +++ b/spark/v4.1/spark/src/jmh/java/org/apache/iceberg/spark/source/parquet/vectorized/VectorizedReadParquetDecimalBenchmark.java @@ -48,8 +48,8 @@ * Benchmark to compare performance of reading Parquet decimal data using vectorized Iceberg read * path and the built-in file source in Spark. * - *

    To run this benchmark for spark-4.0: - * ./gradlew -DsparkVersions=4.0 :iceberg-spark:iceberg-spark-4.0_2.13:jmh \ + *

    To run this benchmark for spark-4.1: + * ./gradlew -DsparkVersions=4.1 :iceberg-spark:iceberg-spark-4.1_2.13:jmh \ * -PjmhIncludeRegex=VectorizedReadParquetDecimalBenchmark \ * -PjmhOutputPath=benchmark/results.txt * diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java index c6784f1041be..a0a5b0518d21 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/BaseCatalog.java @@ -60,6 +60,17 @@ public UnboundProcedure loadProcedure(Identifier ident) { throw new RuntimeException("Procedure " + ident + " not found"); } + @Override + public Identifier[] listProcedures(String[] namespace) { + if (isSystemNamespace(namespace)) { + return SparkProcedures.names().stream() + .map(name -> Identifier.of(namespace, name)) + .toArray(Identifier[]::new); + } else { + return new Identifier[0]; + } + } + @Override public boolean isFunctionNamespace(String[] namespace) { // Allow for empty namespace, as Spark's storage partitioned joins look up 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 98d3e41535cf..0b4578630a06 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 @@ -1021,7 +1021,8 @@ public static Dataset loadMetadataTable( private static DataSourceV2Relation createRelation( SparkTable sparkTable, Map extraOptions) { CaseInsensitiveStringMap options = new CaseInsensitiveStringMap(extraOptions); - return DataSourceV2Relation.create(sparkTable, Option.empty(), Option.empty(), options); + return DataSourceV2Relation.create( + sparkTable, Option.empty(), Option.empty(), options, Option.empty()); } /** diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java index 28a9a31c6a6e..a19ed8060737 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java @@ -66,6 +66,8 @@ import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.types.CalendarInterval; +import org.apache.spark.unsafe.types.GeographyVal; +import org.apache.spark.unsafe.types.GeometryVal; import org.apache.spark.unsafe.types.UTF8String; import org.apache.spark.unsafe.types.VariantVal; @@ -778,5 +780,15 @@ public MapData getMap(int ordinal) { public VariantVal getVariant(int ordinal) { return (VariantVal) values[ordinal]; } + + @Override + public GeographyVal getGeography(int ordinal) { + return (GeographyVal) values[ordinal]; + } + + @Override + public GeometryVal getGeometry(int ordinal) { + return (GeometryVal) values[ordinal]; + } } } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java index 6b42a04421dc..bad31a12c19a 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java @@ -20,6 +20,7 @@ import java.util.Locale; import java.util.Map; +import java.util.Set; import java.util.function.Supplier; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.spark.sql.connector.catalog.TableCatalog; @@ -37,6 +38,10 @@ public static ProcedureBuilder newBuilder(String name) { return builderSupplier != null ? builderSupplier.get() : null; } + public static Set names() { + return BUILDERS.keySet(); + } + private static Map> initProcedureBuilders() { ImmutableMap.Builder> mapBuilder = ImmutableMap.builder(); mapBuilder.put(RollbackToSnapshotProcedure.NAME, RollbackToSnapshotProcedure::builder); diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java index 2d3c917e58f9..074f04d03468 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/StructInternalRow.java @@ -62,6 +62,8 @@ import org.apache.spark.sql.types.TimestampType; import org.apache.spark.sql.types.VariantType; import org.apache.spark.unsafe.types.CalendarInterval; +import org.apache.spark.unsafe.types.GeographyVal; +import org.apache.spark.unsafe.types.GeometryVal; import org.apache.spark.unsafe.types.UTF8String; import org.apache.spark.unsafe.types.VariantVal; @@ -247,6 +249,16 @@ private VariantVal getVariantInternal(int ordinal) { return toVariantVal(value); } + @Override + public GeographyVal getGeography(int ordinal) { + return isNullAt(ordinal) ? null : GeographyVal.fromBytes(getBinaryInternal(ordinal)); + } + + @Override + public GeometryVal getGeometry(int ordinal) { + return isNullAt(ordinal) ? null : GeometryVal.fromBytes(getBinaryInternal(ordinal)); + } + @Override @SuppressWarnings("checkstyle:CyclomaticComplexity") public Object get(int ordinal, DataType dataType) { diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java index 1d1ccca1a2fb..f42c37f5e41d 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java @@ -171,6 +171,7 @@ public void testInt96TimestampProducedBySparkIsReadCorrectly() throws IOExceptio .set("spark.sql.parquet.writeLegacyFormat", "false") .set("spark.sql.parquet.outputTimestampType", "INT96") .set("spark.sql.parquet.fieldId.write.enabled", "true") + .set("spark.sql.parquet.variant.annotateLogicalType.enabled", "true") .build()) { for (InternalRow row : rows) { writer.write(row); 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 c4e0d26c1c31..290e73c3bd1e 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 @@ -53,9 +53,8 @@ import org.apache.spark.sql.Encoder; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.execution.streaming.MemoryStream; +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream; import org.apache.spark.sql.streaming.StreamingQuery; import org.apache.spark.sql.streaming.StreamingQueryException; import org.junit.jupiter.api.AfterAll; @@ -147,7 +146,7 @@ public void testSparkStreamingWriteFailsUnknownTransform() throws IOException, T HadoopTables tables = new HadoopTables(CONF); tables.create(SCHEMA, UNKNOWN_SPEC, location.toString()); - MemoryStream inputStream = newMemoryStream(1, spark.sqlContext(), Encoders.INT()); + MemoryStream inputStream = newMemoryStream(1, spark, Encoders.INT()); StreamingQuery query = inputStream .toDF() @@ -219,8 +218,9 @@ public void testSparkCanReadUnknownTransform() throws IOException { } } - private MemoryStream newMemoryStream(int id, SQLContext sqlContext, Encoder encoder) { - return new MemoryStream<>(id, sqlContext, Option.empty(), encoder); + private MemoryStream newMemoryStream( + int id, SparkSession sparkSession, Encoder encoder) { + return new MemoryStream<>(id, sparkSession, Option.empty(), encoder); } private void send(List records, MemoryStream stream) { diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java index 54048bbf218a..635229f6a005 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java @@ -39,9 +39,8 @@ import org.apache.spark.sql.Encoder; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.execution.streaming.MemoryStream; +import org.apache.spark.sql.execution.streaming.runtime.MemoryStream; import org.apache.spark.sql.streaming.DataStreamWriter; import org.apache.spark.sql.streaming.StreamingQuery; import org.apache.spark.sql.streaming.StreamingQueryException; @@ -96,7 +95,7 @@ public void testStreamingWriteAppendMode() throws Exception { new SimpleRecord(3, "3"), new SimpleRecord(4, "4")); - MemoryStream inputStream = newMemoryStream(1, spark.sqlContext(), Encoders.INT()); + MemoryStream inputStream = newMemoryStream(1, spark, Encoders.INT()); DataStreamWriter streamWriter = inputStream .toDF() @@ -155,7 +154,7 @@ public void testStreamingWriteCompleteMode() throws Exception { Lists.newArrayList( new SimpleRecord(2, "1"), new SimpleRecord(3, "2"), new SimpleRecord(1, "3")); - MemoryStream inputStream = newMemoryStream(1, spark.sqlContext(), Encoders.INT()); + MemoryStream inputStream = newMemoryStream(1, spark, Encoders.INT()); DataStreamWriter streamWriter = inputStream .toDF() @@ -216,7 +215,7 @@ public void testStreamingWriteCompleteModeWithProjection() throws Exception { Lists.newArrayList( new SimpleRecord(1, null), new SimpleRecord(2, null), new SimpleRecord(3, null)); - MemoryStream inputStream = newMemoryStream(1, spark.sqlContext(), Encoders.INT()); + MemoryStream inputStream = newMemoryStream(1, spark, Encoders.INT()); DataStreamWriter streamWriter = inputStream .toDF() @@ -273,7 +272,7 @@ public void testStreamingWriteUpdateMode() throws Exception { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); tables.create(SCHEMA, spec, location.toString()); - MemoryStream inputStream = newMemoryStream(1, spark.sqlContext(), Encoders.INT()); + MemoryStream inputStream = newMemoryStream(1, spark, Encoders.INT()); DataStreamWriter streamWriter = inputStream .toDF() @@ -299,8 +298,9 @@ public void testStreamingWriteUpdateMode() throws Exception { } } - private MemoryStream newMemoryStream(int id, SQLContext sqlContext, Encoder encoder) { - return new MemoryStream<>(id, sqlContext, Option.empty(), encoder); + private MemoryStream newMemoryStream( + int id, SparkSession sparkSession, Encoder encoder) { + return new MemoryStream<>(id, sparkSession, Option.empty(), encoder); } private void send(List records, MemoryStream stream) { diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDefaultValues.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDefaultValues.java index ba856dc5383b..838e735c75f0 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDefaultValues.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDefaultValues.java @@ -38,9 +38,6 @@ *

    Note: These tests use {@code validationCatalog.createTable()} to create tables with default * values because the Iceberg Spark integration does not yet support default value clauses in Spark * DDL. - * - *

    Partial column INSERT statements (e.g., {@code INSERT INTO table (col1) VALUES (val1)}) are - * not supported for DSV2 in Spark 4.0 */ public class TestSparkDefaultValues extends CatalogTestBase { @@ -175,29 +172,6 @@ public void testAlterTableAddColumnWithDefaultUnsupported() { .hasMessageContaining("default values in Spark is currently unsupported"); } - @TestTemplate - public void testPartialInsertUnsupported() { - assertThat(validationCatalog.tableExists(tableIdent)) - .as("Table should not already exist") - .isFalse(); - - Schema schema = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.optional("data") - .withId(2) - .ofType(Types.StringType.get()) - .withWriteDefault(Literal.of("default-data")) - .build()); - - validationCatalog.createTable( - tableIdent, schema, PartitionSpec.unpartitioned(), ImmutableMap.of("format-version", "3")); - - assertThatThrownBy(() -> sql("INSERT INTO %s (id) VALUES (1)", commitTarget())) - .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Cannot find data for the output column"); - } - @TestTemplate public void testSchemaEvolutionWithDefaultValueChanges() { assertThat(validationCatalog.tableExists(tableIdent)) From ed26fd7acae713d4197f4823e86b0da4e0cec8e5 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 22 Dec 2025 21:16:35 -0800 Subject: [PATCH 130/201] DOAP: add release 1.10.1 (#14917) * DOAP: add release 1.10.1 * change name to 1.10.1 --- doap.rdf | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/doap.rdf b/doap.rdf index 0ed7548a89b1..38ea90fe9ed5 100644 --- a/doap.rdf +++ b/doap.rdf @@ -42,9 +42,9 @@ - 1.10.0 - 2025-09-11 - 1.10.0 + 1.10.1 + 2025-12-22 + 1.10.1 From 0069c5e09617d99a3ddce33a4a093fe288243eeb Mon Sep 17 00:00:00 2001 From: Prashant Singh <35593236+singhpk234@users.noreply.github.com> Date: Tue, 23 Dec 2025 10:15:34 -0800 Subject: [PATCH 131/201] INFRA: Skip running CI for doap.rdf file (#14919) Co-authored-by: Prashant Kumar Singh --- .github/workflows/delta-conversion-ci.yml | 1 + .github/workflows/flink-ci.yml | 1 + .github/workflows/hive-ci.yml | 1 + .github/workflows/java-ci.yml | 1 + .github/workflows/kafka-connect-ci.yml | 1 + .github/workflows/spark-ci.yml | 1 + 6 files changed, 6 insertions(+) diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index fd3ac0a14cb2..c7c71085b3f6 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -61,6 +61,7 @@ on: - 'CONTRIBUTING.md' - '**/LICENSE' - '**/NOTICE' + - 'doap.rdf' concurrency: group: ${{ github.workflow }}-${{ github.ref }} diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 4f7a1e438b21..7e8786c672fe 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -61,6 +61,7 @@ on: - 'CONTRIBUTING.md' - '**/LICENSE' - '**/NOTICE' + - 'doap.rdf' concurrency: group: ${{ github.workflow }}-${{ github.ref }} diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index 448d13b23381..5827d3d2fd46 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -62,6 +62,7 @@ on: - 'CONTRIBUTING.md' - '**/LICENSE' - '**/NOTICE' + - 'doap.rdf' concurrency: group: ${{ github.workflow }}-${{ github.ref }} diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index dedb8fc42765..8ff7bb1d4cc8 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -57,6 +57,7 @@ on: - 'CONTRIBUTING.md' - '**/LICENSE' - '**/NOTICE' + - 'doap.rdf' concurrency: group: ${{ github.workflow }}-${{ github.ref }} diff --git a/.github/workflows/kafka-connect-ci.yml b/.github/workflows/kafka-connect-ci.yml index 1effed7c2e56..463a7a85f866 100644 --- a/.github/workflows/kafka-connect-ci.yml +++ b/.github/workflows/kafka-connect-ci.yml @@ -61,6 +61,7 @@ on: - 'CONTRIBUTING.md' - '**/LICENSE' - '**/NOTICE' + - 'doap.rdf' concurrency: group: ${{ github.workflow }}-${{ github.ref }} diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index ecc973e32a8d..4daf93048f55 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -61,6 +61,7 @@ on: - 'CONTRIBUTING.md' - '**/LICENSE' - '**/NOTICE' + - 'doap.rdf' concurrency: group: ${{ github.workflow }}-${{ github.ref }} From 026ec35b7969539b535af03944fae53d059cd233 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Tue, 23 Dec 2025 20:32:01 -0700 Subject: [PATCH 132/201] Core: Small cleanup in MergingSnapshotProducer cleanUncommittedAppends (#14923) --- .../apache/iceberg/BaseRewriteManifests.java | 16 +---- .../java/org/apache/iceberg/FastAppend.java | 17 +---- .../iceberg/MergingSnapshotProducer.java | 65 ++++--------------- .../org/apache/iceberg/SnapshotProducer.java | 16 +++++ 4 files changed, 34 insertions(+), 80 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index 28de63f7f6b8..e98027ec4a0a 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -229,11 +229,10 @@ private void keepActiveManifests(List currentManifests) { } private void reset() { - cleanUncommitted(newManifests, ImmutableSet.of()); + deleteUncommitted(newManifests, ImmutableSet.of(), true /* clear new manifests */); entryCount.set(0); keptManifests.clear(); rewrittenManifests.clear(); - newManifests.clear(); writers.clear(); } @@ -345,19 +344,10 @@ private WriterWrapper getWriter(Object key, int partitionSpecId) { @Override protected void cleanUncommitted(Set committed) { - cleanUncommitted(newManifests, committed); + deleteUncommitted(newManifests, committed, false); // clean up only rewrittenAddedManifests as they are always owned by the table // don't clean up addedManifests as they are added to the manifest list and are not compacted - cleanUncommitted(rewrittenAddedManifests, committed); - } - - private void cleanUncommitted( - Iterable manifests, Set committedManifests) { - for (ManifestFile manifest : manifests) { - if (!committedManifests.contains(manifest)) { - deleteFile(manifest.path()); - } - } + deleteUncommitted(rewrittenAddedManifests, committed, false); } long getManifestTargetSizeBytes() { diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 3d58e192a061..11459e0ecbe1 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -181,25 +181,12 @@ public Object updateEvent() { @Override protected void cleanUncommitted(Set committed) { if (newManifests != null) { - boolean hasDeletes = false; - for (ManifestFile manifest : newManifests) { - if (!committed.contains(manifest)) { - deleteFile(manifest.path()); - hasDeletes = true; - } - } - if (hasDeletes) { - this.newManifests.clear(); - } + deleteUncommitted(newManifests, committed, true /* clear manifests */); } // clean up only rewrittenAppendManifests as they are always owned by the table // don't clean up appendManifests as they are added to the manifest list and are not compacted - for (ManifestFile manifest : rewrittenAppendManifests) { - if (!committed.contains(manifest)) { - deleteFile(manifest.path()); - } - } + deleteUncommitted(rewrittenAppendManifests, committed, false); } /** diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 9ed2f4f4c0fb..51d17fbdd0f2 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -289,12 +289,6 @@ protected void validateNewDeleteFile(DeleteFile file) { ContentFileUtil.dvDesc(file)); break; case 3: - Preconditions.checkArgument( - file.content() == FileContent.EQUALITY_DELETES || ContentFileUtil.isDV(file), - "Must use DVs for position deletes in V%s: %s", - formatVersion(), - file.location()); - break; case 4: Preconditions.checkArgument( file.content() == FileContent.EQUALITY_DELETES || ContentFileUtil.isDV(file), @@ -991,62 +985,29 @@ public Object updateEvent() { return new CreateSnapshotEvent(tableName, operation(), snapshotId, sequenceNumber, summary); } - @SuppressWarnings("checkstyle:CyclomaticComplexity") - private void cleanUncommittedAppends(Set committed) { - if (!cachedNewDataManifests.isEmpty()) { - boolean hasDeletes = false; - for (ManifestFile manifest : cachedNewDataManifests) { - if (!committed.contains(manifest)) { - deleteFile(manifest.path()); - hasDeletes = true; - } - } - - if (hasDeletes) { - this.cachedNewDataManifests.clear(); - } - } - - boolean hasDeleteDeletes = false; - for (ManifestFile cachedNewDeleteManifest : cachedNewDeleteManifests) { - if (!committed.contains(cachedNewDeleteManifest)) { - deleteFile(cachedNewDeleteManifest.path()); - hasDeleteDeletes = true; - } - } - - if (hasDeleteDeletes) { - this.cachedNewDeleteManifests.clear(); - } + @Override + protected void cleanUncommitted(Set committed) { + mergeManager.cleanUncommitted(committed); + filterManager.cleanUncommitted(committed); + deleteMergeManager.cleanUncommitted(committed); + deleteFilterManager.cleanUncommitted(committed); + cleanUncommittedAppends(committed); + } + private void cleanUncommittedAppends(Set committed) { + deleteUncommitted(cachedNewDataManifests, committed, true /* clear manifests */); + deleteUncommitted(cachedNewDeleteManifests, committed, true /* clear manifests */); // rewritten manifests are always owned by the table - for (ManifestFile manifest : rewrittenAppendManifests) { - if (!committed.contains(manifest)) { - deleteFile(manifest.path()); - } - } + deleteUncommitted(rewrittenAppendManifests, committed, false); // manifests that are not rewritten are only owned by the table if the commit succeeded if (!committed.isEmpty()) { // the commit succeeded if at least one manifest was committed // the table now owns appendManifests; clean up any that are not used - for (ManifestFile manifest : appendManifests) { - if (!committed.contains(manifest)) { - deleteFile(manifest.path()); - } - } + deleteUncommitted(appendManifests, committed, false); } } - @Override - protected void cleanUncommitted(Set committed) { - mergeManager.cleanUncommitted(committed); - filterManager.cleanUncommitted(committed); - deleteMergeManager.cleanUncommitted(committed); - deleteFilterManager.cleanUncommitted(committed); - cleanUncommittedAppends(committed); - } - private Iterable prepareNewDataManifests() { Iterable newManifests; if (!newDataFilesBySpec.isEmpty()) { diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 876e0190d28c..a8f28855ab9b 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -654,6 +654,22 @@ protected List writeDataManifests( return writeManifests(files, group -> writeDataFileGroup(group, dataSeq, spec)); } + // Deletes uncommitted manifests; clears list if clearManifests and any deleted. + protected void deleteUncommitted( + Collection manifests, Set committed, boolean clearManifests) { + boolean anyDeleted = false; + for (ManifestFile manifest : manifests) { + if (!committed.contains(manifest)) { + deleteFile(manifest.path()); + anyDeleted = true; + } + } + + if (clearManifests && anyDeleted) { + manifests.clear(); + } + } + private List writeDataFileGroup( Collection files, Long dataSeq, PartitionSpec spec) { RollingManifestWriter writer = newRollingManifestWriter(spec); From 0651b8913d27c3b1c9aca4a9609bec521905fb36 Mon Sep 17 00:00:00 2001 From: nhuantho <91556274+nhuantho@users.noreply.github.com> Date: Thu, 25 Dec 2025 01:02:23 +0700 Subject: [PATCH 133/201] [doc] Add highlight note for Hadoop S3A FileSystem (#14913) --------- Co-authored-by: nhuan.bc --- docs/docs/aws.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/docs/aws.md b/docs/docs/aws.md index c306d5493a0a..9c979b9da89f 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -416,9 +416,11 @@ There is no redundant consistency wait and check which might negatively impact p ### Hadoop S3A FileSystem +!!! important + **S3FileIO is recommended** for S3 use cases rather than the `S3A FileSystem` (`HadoopFileIO`). + Before `S3FileIO` was introduced, many Iceberg users choose to use `HadoopFileIO` to write data to S3 through the [S3A FileSystem](https://github.com/apache/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java). -As introduced in the previous sections, `S3FileIO` adopts the latest AWS clients and S3 features for optimized security and performance - and is thus recommended for S3 use cases rather than the S3A FileSystem. +As introduced in the previous sections, `S3FileIO` adopts the latest AWS clients and S3 features for optimized security and performance. `S3FileIO` writes data with `s3://` URI scheme, but it is also compatible with schemes written by the S3A FileSystem. This means for any table manifests containing `s3a://` or `s3n://` file paths, `S3FileIO` is still able to read them. From 53044611e25313e11e732ddf98fd2a6eae1aa923 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 27 Dec 2025 21:01:11 -0800 Subject: [PATCH 134/201] Build: Bump datamodel-code-generator from 0.46.0 to 0.49.0 (#14938) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.46.0 to 0.49.0. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Changelog](https://github.com/koxudaxi/datamodel-code-generator/blob/main/CHANGELOG.md) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.46.0...0.49.0) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-version: 0.49.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index 51d78c1ed46f..41e84cfb0b39 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,5 +16,5 @@ # under the License. openapi-spec-validator==0.7.2 -datamodel-code-generator==0.46.0 +datamodel-code-generator==0.49.0 yamllint==1.37.1 From b26009c5cc3a47be4fda062ac1cc0efa5f78d72f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 27 Dec 2025 21:01:19 -0800 Subject: [PATCH 135/201] Build: Bump pymarkdownlnt from 0.9.33 to 0.9.34 (#14937) Bumps [pymarkdownlnt](https://github.com/jackdewinter/pymarkdown) from 0.9.33 to 0.9.34. - [Release notes](https://github.com/jackdewinter/pymarkdown/releases) - [Changelog](https://github.com/jackdewinter/pymarkdown/blob/main/changelog.md) - [Commits](https://github.com/jackdewinter/pymarkdown/compare/v0.9.33...v0.9.34) --- updated-dependencies: - dependency-name: pymarkdownlnt dependency-version: 0.9.34 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index ac9f68f38f58..816bdc87f8a2 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -21,4 +21,4 @@ mkdocs-material==9.6.23 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.2 -pymarkdownlnt==0.9.33 +pymarkdownlnt==0.9.34 From 63c923e6bc78e72bb78adb0ba60ab3c6bf3884c9 Mon Sep 17 00:00:00 2001 From: Prashant Singh <35593236+singhpk234@users.noreply.github.com> Date: Sat, 27 Dec 2025 23:24:00 -0800 Subject: [PATCH 136/201] fix test regex (#14939) Co-authored-by: Prashant Singh --- .../test/java/org/apache/iceberg/aliyun/oss/TestOSSURI.java | 4 ++-- gradle/libs.versions.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSURI.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSURI.java index e388ee932589..cc44564ae6b7 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSURI.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSURI.java @@ -50,10 +50,10 @@ public void testEncodedString() { @Test public void invalidBucket() { - assertThatThrownBy(() -> new OSSURI("https://test_bucket/path/to/file")) + assertThatThrownBy(() -> new OSSURI("https://test#bucket/path/to/file")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( - OSS_RESOURCE_MANAGER.getFormattedString("BucketNameInvalid", "test_bucket")); + OSS_RESOURCE_MANAGER.getFormattedString("BucketNameInvalid", "test#bucket")); } @Test diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 4fbba96317ce..800526946f09 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -21,7 +21,7 @@ [versions] activation = "1.1.1" -aliyun-sdk-oss = "3.10.2" +aliyun-sdk-oss = "3.18.4" analyticsaccelerator = "1.3.1" antlr = "4.9.3" antlr413 = "4.13.1" # For Spark 4.0 support From 4db39095867843d4031cf5a2f04b878e930de13b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 27 Dec 2025 23:25:45 -0800 Subject: [PATCH 137/201] Build: Bump org.openapitools:openapi-generator-gradle-plugin (#14934) Bumps [org.openapitools:openapi-generator-gradle-plugin](https://github.com/OpenAPITools/openapi-generator) from 7.17.0 to 7.18.0. - [Release notes](https://github.com/OpenAPITools/openapi-generator/releases) - [Changelog](https://github.com/OpenAPITools/openapi-generator/blob/master/docs/release-summary.md) - [Commits](https://github.com/OpenAPITools/openapi-generator/compare/v7.17.0...v7.18.0) --- updated-dependencies: - dependency-name: org.openapitools:openapi-generator-gradle-plugin dependency-version: 7.18.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index bd8062193421..258b111ce516 100644 --- a/build.gradle +++ b/build.gradle @@ -36,7 +36,7 @@ buildscript { classpath 'org.revapi:gradle-revapi:1.8.0' classpath 'com.gorylenko.gradle-git-properties:gradle-git-properties:2.5.4' classpath 'com.palantir.gradle.gitversion:gradle-git-version:3.4.0' - classpath 'org.openapitools:openapi-generator-gradle-plugin:7.17.0' + classpath 'org.openapitools:openapi-generator-gradle-plugin:7.18.0' } } From 4632f3133efbf6b8232ed9526de7142fd50e1aa6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 27 Dec 2025 23:26:04 -0800 Subject: [PATCH 138/201] Build: Bump software.amazon.awssdk:bom from 2.40.13 to 2.40.16 (#14936) Bumps software.amazon.awssdk:bom from 2.40.13 to 2.40.16. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-version: 2.40.16 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 800526946f09..3639329293b7 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -31,7 +31,7 @@ arrow = "15.0.2" avro = "1.12.1" assertj-core = "3.27.6" awaitility = "4.3.0" -awssdk-bom = "2.40.13" +awssdk-bom = "2.40.16" azuresdk-bom = "1.3.3" awssdk-s3accessgrants = "2.3.0" bson-ver = "4.11.5" From 9c3bed6a651f20fc06b1a4c329e40910bd6eb36a Mon Sep 17 00:00:00 2001 From: Varun Lakhyani <130844282+varun-lakhyani@users.noreply.github.com> Date: Wed, 31 Dec 2025 01:46:19 +0530 Subject: [PATCH 139/201] Docs: Fix MERGE INTO example in Getting Started (#14943) * Docs: Fix MERGE INTO example in Getting Started * Docs: Fix Getting Started Spark SQL example to be runnable * Apply same documentation fix to Spark 3.4, 4.0, and 4.1 versions * Retry CI as earlier gave 429 --- docs/docs/spark-getting-started.md | 8 ++++++-- .../java/org/apache/iceberg/spark/TestRoundTrip.java | 2 -- .../java/org/apache/iceberg/spark/TestRoundTrip.java | 2 -- .../java/org/apache/iceberg/spark/TestRoundTrip.java | 2 -- .../java/org/apache/iceberg/spark/TestRoundTrip.java | 2 -- 5 files changed, 6 insertions(+), 10 deletions(-) diff --git a/docs/docs/spark-getting-started.md b/docs/docs/spark-getting-started.md index 6813c76937a1..821bfd022c20 100644 --- a/docs/docs/spark-getting-started.md +++ b/docs/docs/spark-getting-started.md @@ -61,6 +61,8 @@ To create your first Iceberg table in Spark, use the `spark-sql` shell or `spark ```sql -- local is the path-based catalog defined above CREATE TABLE local.db.table (id bigint, data string) USING iceberg; +CREATE TABLE source (id bigint, data string) USING parquet; +CREATE TABLE updates (id bigint, data string) USING parquet; ``` Iceberg catalogs support the full range of SQL DDL commands, including: @@ -76,14 +78,16 @@ Once your table is created, insert data using [`INSERT INTO`](spark-writes.md#in ```sql INSERT INTO local.db.table VALUES (1, 'a'), (2, 'b'), (3, 'c'); +INSERT INTO source VALUES (10, 'd'), (11, 'ee'); +INSERT INTO updates VALUES (1, 'x'), (2, 'x'), (4, 'z'); INSERT INTO local.db.table SELECT id, data FROM source WHERE length(data) = 1; ``` Iceberg also adds row-level SQL updates to Spark, [`MERGE INTO`](spark-writes.md#merge-into) and [`DELETE FROM`](spark-writes.md#delete-from): ```sql -MERGE INTO local.db.target t USING (SELECT * FROM updates) u ON t.id = u.id -WHEN MATCHED THEN UPDATE SET t.count = t.count + u.count +MERGE INTO local.db.table t USING (SELECT * FROM updates) u ON t.id = u.id +WHEN MATCHED THEN UPDATE SET t.data = u.data WHEN NOT MATCHED THEN INSERT *; ``` diff --git a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java index 29f725615a21..709f626b08c1 100644 --- a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java +++ b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java @@ -38,8 +38,6 @@ public void dropTable() { } // Run through our Doc's Getting Started Example - // TODO Update doc example so that it can actually be run, modifications were required for this - // test suite to run @TestTemplate public void testGettingStarted() throws IOException { // Creating a table diff --git a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java index 29f725615a21..709f626b08c1 100644 --- a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java +++ b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java @@ -38,8 +38,6 @@ public void dropTable() { } // Run through our Doc's Getting Started Example - // TODO Update doc example so that it can actually be run, modifications were required for this - // test suite to run @TestTemplate public void testGettingStarted() throws IOException { // Creating a table diff --git a/spark/v4.0/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java b/spark/v4.0/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java index 29f725615a21..709f626b08c1 100644 --- a/spark/v4.0/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java +++ b/spark/v4.0/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java @@ -38,8 +38,6 @@ public void dropTable() { } // Run through our Doc's Getting Started Example - // TODO Update doc example so that it can actually be run, modifications were required for this - // test suite to run @TestTemplate public void testGettingStarted() throws IOException { // Creating a table diff --git a/spark/v4.1/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java b/spark/v4.1/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java index 29f725615a21..709f626b08c1 100644 --- a/spark/v4.1/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java +++ b/spark/v4.1/spark-runtime/src/integration/java/org/apache/iceberg/spark/TestRoundTrip.java @@ -38,8 +38,6 @@ public void dropTable() { } // Run through our Doc's Getting Started Example - // TODO Update doc example so that it can actually be run, modifications were required for this - // test suite to run @TestTemplate public void testGettingStarted() throws IOException { // Creating a table From 00046005889c66ffc860bae012d7fc560e8f040a Mon Sep 17 00:00:00 2001 From: Dan LaRocque Date: Thu, 1 Jan 2026 20:11:51 -0800 Subject: [PATCH 140/201] Spec: fix impl note about snapshot ID generation (#14720) --- format/spec.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/format/spec.md b/format/spec.md index 57e8c7047e82..43348d856c9e 100644 --- a/format/spec.md +++ b/format/spec.md @@ -1827,7 +1827,7 @@ Snapshot summary can include metrics fields to track numeric stats of the snapsh Writers should produce positive values for snapshot ids in a manner that minimizes the probability of id collisions and should verify the id does not conflict with existing snapshots. Producing snapshot ids based on timestamps alone is not recommended as it increases the potential for collisions. -The reference Java implementation uses a type 4 uuid and XORs the 4 most significant bytes with the 4 least significant bytes then ANDs with the maximum long value to arrive at a pseudo-random snapshot id with a low probability of collision. +The reference Java implementation uses a type 4 uuid and XORs the 8 most significant bytes with the 8 least significant bytes then ANDs with the maximum long value to arrive at a pseudo-random snapshot id with a low probability of collision. Java writes `-1` for "no current snapshot" with V1 and V2 tables and considers this equivalent to omitted or `null`. This has never been formalized in the spec, but for compatibility, other implementations can accept `-1` as `null`. Java will no longer write `-1` and will use `null` for "no current snapshot" for all tables with a version greater than or equal to V3. From e131329a07fb5d726bc33177d9f0cd23aaebd2fb Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Sat, 3 Jan 2026 08:24:26 -0800 Subject: [PATCH 141/201] Spark: Add ordering to TestSelect to remove flakiness (#14956) --- .../apache/iceberg/spark/sql/TestSelect.java | 42 +++++++++++------- .../apache/iceberg/spark/sql/TestSelect.java | 43 ++++++++++++------- 2 files changed, 54 insertions(+), 31 deletions(-) diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 8b146d925bca..d3fe57704533 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -110,7 +110,10 @@ public void testSelect() { List expected = ImmutableList.of(row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", Float.NaN)); - assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT * FROM %s ORDER BY id", tableName)); } @TestTemplate @@ -121,7 +124,10 @@ public void testSelectWithSpecifiedTargetSplitSize() { Table table = validationCatalog.loadTable(tableIdent); table.updateProperties().set("read.split.target-size", "1024").commit(); spark.sql("REFRESH TABLE " + tableName); - assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT * FROM %s ORDER BY id", tableName)); // Query failed when `SPLIT_SIZE` < 0 table.updateProperties().set(SPLIT_SIZE, "-1").commit(); @@ -161,16 +167,21 @@ public void selectWithLimit() { // verify that LIMIT is properly applied in case SupportsPushDownLimit.isPartiallyPushed() is // ever overridden in SparkScanBuilder - assertThat(sql("SELECT * FROM %s LIMIT 1", tableName)).containsExactly(first); - assertThat(sql("SELECT * FROM %s LIMIT 2", tableName)).containsExactly(first, second); - assertThat(sql("SELECT * FROM %s LIMIT 3", tableName)).containsExactly(first, second, third); + assertThat(sql("SELECT * FROM %s ORDER BY id LIMIT 1", tableName)).containsExactly(first); + assertThat(sql("SELECT * FROM %s ORDER BY id LIMIT 2", tableName)) + .containsExactly(first, second); + assertThat(sql("SELECT * FROM %s ORDER BY id LIMIT 3", tableName)) + .containsExactly(first, second, third); } @TestTemplate public void testProjection() { List expected = ImmutableList.of(row(1L), row(2L), row(3L)); - assertEquals("Should return all expected rows", expected, sql("SELECT id FROM %s", tableName)); + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT id FROM %s ORDER BY id", tableName)); assertThat(scanEventCount).as("Should create only one scan").isEqualTo(1); assertThat(lastScanEvent.filter()) @@ -402,7 +413,7 @@ public void readAndWriteWithBranchAfterSchemaChange() { List expected = Arrays.asList(row(1L, "a", 1.0f), row(2L, "b", 2.0f), row(3L, "c", Float.NaN)); - assertThat(sql("SELECT * FROM %s", tableName)).containsExactlyElementsOf(expected); + assertThat(sql("SELECT * FROM %s", tableName)).containsExactlyInAnyOrderElementsOf(expected); // change schema on the table and add more data sql("ALTER TABLE %s DROP COLUMN float", tableName); @@ -678,24 +689,25 @@ public void simpleTypesInFilter() { + "(2, false, 2, 2L, 2.2, 2.4, '2.6', to_date('2022-02-02'), to_timestamp('2022-02-02T00:00:00')), " + "(3, true, 3, 3L, 3.3, 3.6, '3.9', to_date('2023-03-03'), to_timestamp('2023-03-03T00:00:00'))", tableName); - assertThat(sql("SELECT id FROM %s where id > 1", tableName)).containsExactly(row(2L), row(3L)); + assertThat(sql("SELECT id FROM %s where id > 1", tableName)) + .containsExactlyInAnyOrder(row(2L), row(3L)); assertThat(sql("SELECT id, boolean FROM %s where boolean = true", tableName)) - .containsExactly(row(1L, true), row(3L, true)); + .containsExactlyInAnyOrder(row(1L, true), row(3L, true)); assertThat(sql("SELECT long FROM %s where long > 1", tableName)) - .containsExactly(row(2L), row(3L)); + .containsExactlyInAnyOrder(row(2L), row(3L)); assertThat(sql("SELECT float FROM %s where float > 1.1f", tableName)) - .containsExactly(row(2.2f), row(3.3f)); + .containsExactlyInAnyOrder(row(2.2f), row(3.3f)); assertThat(sql("SELECT double FROM %s where double > 1.3", tableName)) - .containsExactly(row(2.4d), row(3.6d)); + .containsExactlyInAnyOrder(row(2.4d), row(3.6d)); assertThat(sql("SELECT string FROM %s where string > '1.5'", tableName)) - .containsExactly(row("2.6"), row("3.9")); + .containsExactlyInAnyOrder(row("2.6"), row("3.9")); java.sql.Date dateOne = java.sql.Date.valueOf("2022-02-02"); java.sql.Date dateTwo = java.sql.Date.valueOf("2023-03-03"); assertThat(sql("SELECT date FROM %s where date > to_date('2021-01-01')", tableName)) - .containsExactly(row(dateOne), row(dateTwo)); + .containsExactlyInAnyOrder(row(dateOne), row(dateTwo)); assertThat( sql("SELECT timestamp FROM %s where timestamp > to_timestamp('2021-01-01')", tableName)) - .containsExactly( + .containsExactlyInAnyOrder( row(new Timestamp(dateOne.getTime())), row(new Timestamp(dateTwo.getTime()))); sql("DROP TABLE IF EXISTS %s", tableName); 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 8b146d925bca..cd8ca630a12b 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 @@ -110,7 +110,10 @@ public void testSelect() { List expected = ImmutableList.of(row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", Float.NaN)); - assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT * FROM %s ORDER BY id", tableName)); } @TestTemplate @@ -121,7 +124,10 @@ public void testSelectWithSpecifiedTargetSplitSize() { Table table = validationCatalog.loadTable(tableIdent); table.updateProperties().set("read.split.target-size", "1024").commit(); spark.sql("REFRESH TABLE " + tableName); - assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT * FROM %s ORDER BY id", tableName)); // Query failed when `SPLIT_SIZE` < 0 table.updateProperties().set(SPLIT_SIZE, "-1").commit(); @@ -161,16 +167,20 @@ public void selectWithLimit() { // verify that LIMIT is properly applied in case SupportsPushDownLimit.isPartiallyPushed() is // ever overridden in SparkScanBuilder - assertThat(sql("SELECT * FROM %s LIMIT 1", tableName)).containsExactly(first); - assertThat(sql("SELECT * FROM %s LIMIT 2", tableName)).containsExactly(first, second); - assertThat(sql("SELECT * FROM %s LIMIT 3", tableName)).containsExactly(first, second, third); + assertThat(sql("SELECT * FROM %s ORDER BY id LIMIT 1", tableName)).containsExactly(first); + assertThat(sql("SELECT * FROM %s ORDER BY id LIMIT 2", tableName)) + .containsExactly(first, second); + assertThat(sql("SELECT * FROM %s ORDER BY id LIMIT 3", tableName)) + .containsExactly(first, second, third); } @TestTemplate public void testProjection() { List expected = ImmutableList.of(row(1L), row(2L), row(3L)); - - assertEquals("Should return all expected rows", expected, sql("SELECT id FROM %s", tableName)); + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT id FROM %s ORDER BY id", tableName)); assertThat(scanEventCount).as("Should create only one scan").isEqualTo(1); assertThat(lastScanEvent.filter()) @@ -402,7 +412,7 @@ public void readAndWriteWithBranchAfterSchemaChange() { List expected = Arrays.asList(row(1L, "a", 1.0f), row(2L, "b", 2.0f), row(3L, "c", Float.NaN)); - assertThat(sql("SELECT * FROM %s", tableName)).containsExactlyElementsOf(expected); + assertThat(sql("SELECT * FROM %s", tableName)).containsExactlyInAnyOrderElementsOf(expected); // change schema on the table and add more data sql("ALTER TABLE %s DROP COLUMN float", tableName); @@ -678,24 +688,25 @@ public void simpleTypesInFilter() { + "(2, false, 2, 2L, 2.2, 2.4, '2.6', to_date('2022-02-02'), to_timestamp('2022-02-02T00:00:00')), " + "(3, true, 3, 3L, 3.3, 3.6, '3.9', to_date('2023-03-03'), to_timestamp('2023-03-03T00:00:00'))", tableName); - assertThat(sql("SELECT id FROM %s where id > 1", tableName)).containsExactly(row(2L), row(3L)); + assertThat(sql("SELECT id FROM %s where id > 1", tableName)) + .containsExactlyInAnyOrder(row(2L), row(3L)); assertThat(sql("SELECT id, boolean FROM %s where boolean = true", tableName)) - .containsExactly(row(1L, true), row(3L, true)); + .containsExactlyInAnyOrder(row(1L, true), row(3L, true)); assertThat(sql("SELECT long FROM %s where long > 1", tableName)) - .containsExactly(row(2L), row(3L)); + .containsExactlyInAnyOrder(row(2L), row(3L)); assertThat(sql("SELECT float FROM %s where float > 1.1f", tableName)) - .containsExactly(row(2.2f), row(3.3f)); + .containsExactlyInAnyOrder(row(2.2f), row(3.3f)); assertThat(sql("SELECT double FROM %s where double > 1.3", tableName)) - .containsExactly(row(2.4d), row(3.6d)); + .containsExactlyInAnyOrder(row(2.4d), row(3.6d)); assertThat(sql("SELECT string FROM %s where string > '1.5'", tableName)) - .containsExactly(row("2.6"), row("3.9")); + .containsExactlyInAnyOrder(row("2.6"), row("3.9")); java.sql.Date dateOne = java.sql.Date.valueOf("2022-02-02"); java.sql.Date dateTwo = java.sql.Date.valueOf("2023-03-03"); assertThat(sql("SELECT date FROM %s where date > to_date('2021-01-01')", tableName)) - .containsExactly(row(dateOne), row(dateTwo)); + .containsExactlyInAnyOrder(row(dateOne), row(dateTwo)); assertThat( sql("SELECT timestamp FROM %s where timestamp > to_timestamp('2021-01-01')", tableName)) - .containsExactly( + .containsExactlyInAnyOrder( row(new Timestamp(dateOne.getTime())), row(new Timestamp(dateTwo.getTime()))); sql("DROP TABLE IF EXISTS %s", tableName); From 01b59d37f562295de6064f3898f6241639e1817e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 3 Jan 2026 23:48:37 -0800 Subject: [PATCH 142/201] Build: Bump software.amazon.awssdk:bom from 2.40.16 to 2.41.1 (#14961) Bumps software.amazon.awssdk:bom from 2.40.16 to 2.41.1. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-version: 2.41.1 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 3639329293b7..bbfb9d3616ba 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -31,7 +31,7 @@ arrow = "15.0.2" avro = "1.12.1" assertj-core = "3.27.6" awaitility = "4.3.0" -awssdk-bom = "2.40.16" +awssdk-bom = "2.41.1" azuresdk-bom = "1.3.3" awssdk-s3accessgrants = "2.3.0" bson-ver = "4.11.5" From 64b7b66222b1f59eaf0c4ce09350665d590271ae Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Sun, 4 Jan 2026 16:50:08 +0900 Subject: [PATCH 143/201] Build: Bump datamodel-code-generator from 0.49.0 to 0.52.1 (#14962) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.49.0 to 0.52.1. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Changelog](https://github.com/koxudaxi/datamodel-code-generator/blob/main/CHANGELOG.md) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.49.0...0.52.1) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-version: 0.52.1 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- open-api/rest-catalog-open-api.py | 199 +++++++++++------------------- 2 files changed, 74 insertions(+), 127 deletions(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index 41e84cfb0b39..c12adbaaedf2 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,5 +16,5 @@ # under the License. openapi-spec-validator==0.7.2 -datamodel-code-generator==0.49.0 +datamodel-code-generator==0.52.1 yamllint==1.37.1 diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index c39416388e7e..1079d277d3c7 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -136,11 +136,11 @@ class ExpressionType(BaseModel): class TrueExpression(BaseModel): - type: str = Field('true', const=True) + type: Literal['true'] = Field('true', const=True) class FalseExpression(BaseModel): - type: str = Field('false', const=True) + type: Literal['false'] = Field('false', const=True) class Reference(BaseModel): @@ -297,17 +297,19 @@ class AssignUUIDUpdate(BaseUpdate): Assigning a UUID to a table/view should only be done when creating the table/view. It is not safe to re-assign the UUID if a table/view already has a UUID assigned """ - action: str = Field('assign-uuid', const=True) + action: Literal['assign-uuid'] = Field('assign-uuid', const=True) uuid: str class UpgradeFormatVersionUpdate(BaseUpdate): - action: str = Field('upgrade-format-version', const=True) + action: Literal['upgrade-format-version'] = Field( + 'upgrade-format-version', const=True + ) format_version: int = Field(..., alias='format-version') class SetCurrentSchemaUpdate(BaseUpdate): - action: str = Field('set-current-schema', const=True) + action: Literal['set-current-schema'] = Field('set-current-schema', const=True) schema_id: int = Field( ..., alias='schema-id', @@ -316,12 +318,12 @@ class SetCurrentSchemaUpdate(BaseUpdate): class AddPartitionSpecUpdate(BaseUpdate): - action: str = Field('add-spec', const=True) + action: Literal['add-spec'] = Field('add-spec', const=True) spec: PartitionSpec class SetDefaultSpecUpdate(BaseUpdate): - action: str = Field('set-default-spec', const=True) + action: Literal['set-default-spec'] = Field('set-default-spec', const=True) spec_id: int = Field( ..., alias='spec-id', @@ -330,12 +332,14 @@ class SetDefaultSpecUpdate(BaseUpdate): class AddSortOrderUpdate(BaseUpdate): - action: str = Field('add-sort-order', const=True) + action: Literal['add-sort-order'] = Field('add-sort-order', const=True) sort_order: SortOrder = Field(..., alias='sort-order') class SetDefaultSortOrderUpdate(BaseUpdate): - action: str = Field('set-default-sort-order', const=True) + action: Literal['set-default-sort-order'] = Field( + 'set-default-sort-order', const=True + ) sort_order_id: int = Field( ..., alias='sort-order-id', @@ -344,52 +348,49 @@ class SetDefaultSortOrderUpdate(BaseUpdate): class AddSnapshotUpdate(BaseUpdate): - action: str = Field('add-snapshot', const=True) + action: Literal['add-snapshot'] = Field('add-snapshot', const=True) snapshot: Snapshot -class SetSnapshotRefUpdate(BaseModel): - action: str = Field(..., const=True) +class SetSnapshotRefUpdate(BaseUpdate, SnapshotReference): + action: Literal['set-snapshot-ref'] = Field('set-snapshot-ref', const=True) ref_name: str = Field(..., alias='ref-name') - type: Literal['tag', 'branch'] - snapshot_id: int = Field(..., alias='snapshot-id') - max_ref_age_ms: int | None = Field(None, alias='max-ref-age-ms') - max_snapshot_age_ms: int | None = Field(None, alias='max-snapshot-age-ms') - min_snapshots_to_keep: int | None = Field(None, alias='min-snapshots-to-keep') class RemoveSnapshotsUpdate(BaseUpdate): - action: str = Field('remove-snapshots', const=True) + action: Literal['remove-snapshots'] = Field('remove-snapshots', const=True) snapshot_ids: list[int] = Field(..., alias='snapshot-ids') class RemoveSnapshotRefUpdate(BaseUpdate): - action: str = Field('remove-snapshot-ref', const=True) + action: Literal['remove-snapshot-ref'] = Field('remove-snapshot-ref', const=True) ref_name: str = Field(..., alias='ref-name') class SetLocationUpdate(BaseUpdate): - action: str = Field('set-location', const=True) + action: Literal['set-location'] = Field('set-location', const=True) location: str class SetPropertiesUpdate(BaseUpdate): - action: str = Field('set-properties', const=True) + action: Literal['set-properties'] = Field('set-properties', const=True) updates: dict[str, str] class RemovePropertiesUpdate(BaseUpdate): - action: str = Field('remove-properties', const=True) + action: Literal['remove-properties'] = Field('remove-properties', const=True) removals: list[str] class AddViewVersionUpdate(BaseUpdate): - action: str = Field('add-view-version', const=True) + action: Literal['add-view-version'] = Field('add-view-version', const=True) view_version: ViewVersion = Field(..., alias='view-version') class SetCurrentViewVersionUpdate(BaseUpdate): - action: str = Field('set-current-view-version', const=True) + action: Literal['set-current-view-version'] = Field( + 'set-current-view-version', const=True + ) view_version_id: int = Field( ..., alias='view-version-id', @@ -398,32 +399,38 @@ class SetCurrentViewVersionUpdate(BaseUpdate): class RemoveStatisticsUpdate(BaseUpdate): - action: str = Field('remove-statistics', const=True) + action: Literal['remove-statistics'] = Field('remove-statistics', const=True) snapshot_id: int = Field(..., alias='snapshot-id') class RemovePartitionStatisticsUpdate(BaseUpdate): - action: str = Field('remove-partition-statistics', const=True) + action: Literal['remove-partition-statistics'] = Field( + 'remove-partition-statistics', const=True + ) snapshot_id: int = Field(..., alias='snapshot-id') class RemovePartitionSpecsUpdate(BaseUpdate): - action: str = Field('remove-partition-specs', const=True) + action: Literal['remove-partition-specs'] = Field( + 'remove-partition-specs', const=True + ) spec_ids: list[int] = Field(..., alias='spec-ids') class RemoveSchemasUpdate(BaseUpdate): - action: str = Field('remove-schemas', const=True) + action: Literal['remove-schemas'] = Field('remove-schemas', const=True) schema_ids: list[int] = Field(..., alias='schema-ids') class AddEncryptionKeyUpdate(BaseUpdate): - action: str = Field('add-encryption-key', const=True) + action: Literal['add-encryption-key'] = Field('add-encryption-key', const=True) encryption_key: EncryptedKey = Field(..., alias='encryption-key') class RemoveEncryptionKeyUpdate(BaseUpdate): - action: str = Field('remove-encryption-key', const=True) + action: Literal['remove-encryption-key'] = Field( + 'remove-encryption-key', const=True + ) key_id: str = Field(..., alias='key-id') @@ -436,7 +443,7 @@ class AssertCreate(TableRequirement): The table must not already exist; used for create transactions """ - type: str = Field(..., const=True) + type: Literal['assert-create'] = Field(..., const=True) class AssertTableUUID(TableRequirement): @@ -444,7 +451,7 @@ class AssertTableUUID(TableRequirement): The table UUID must match the requirement's `uuid` """ - type: str = Field(..., const=True) + type: Literal['assert-table-uuid'] = Field(..., const=True) uuid: str @@ -456,7 +463,9 @@ class AssertRefSnapshotId(TableRequirement): """ - type: str = Field('assert-ref-snapshot-id', const=True) + type: Literal['assert-ref-snapshot-id'] = Field( + 'assert-ref-snapshot-id', const=True + ) ref: str snapshot_id: int = Field(..., alias='snapshot-id') @@ -466,7 +475,9 @@ class AssertLastAssignedFieldId(TableRequirement): The table's last assigned column id must match the requirement's `last-assigned-field-id` """ - type: str = Field('assert-last-assigned-field-id', const=True) + type: Literal['assert-last-assigned-field-id'] = Field( + 'assert-last-assigned-field-id', const=True + ) last_assigned_field_id: int = Field(..., alias='last-assigned-field-id') @@ -475,7 +486,9 @@ class AssertCurrentSchemaId(TableRequirement): The table's current schema id must match the requirement's `current-schema-id` """ - type: str = Field('assert-current-schema-id', const=True) + type: Literal['assert-current-schema-id'] = Field( + 'assert-current-schema-id', const=True + ) current_schema_id: int = Field(..., alias='current-schema-id') @@ -484,7 +497,9 @@ class AssertLastAssignedPartitionId(TableRequirement): The table's last assigned partition id must match the requirement's `last-assigned-partition-id` """ - type: str = Field('assert-last-assigned-partition-id', const=True) + type: Literal['assert-last-assigned-partition-id'] = Field( + 'assert-last-assigned-partition-id', const=True + ) last_assigned_partition_id: int = Field(..., alias='last-assigned-partition-id') @@ -493,7 +508,9 @@ class AssertDefaultSpecId(TableRequirement): The table's default spec id must match the requirement's `default-spec-id` """ - type: str = Field('assert-default-spec-id', const=True) + type: Literal['assert-default-spec-id'] = Field( + 'assert-default-spec-id', const=True + ) default_spec_id: int = Field(..., alias='default-spec-id') @@ -502,7 +519,9 @@ class AssertDefaultSortOrderId(TableRequirement): The table's default sort order id must match the requirement's `default-sort-order-id` """ - type: str = Field('assert-default-sort-order-id', const=True) + type: Literal['assert-default-sort-order-id'] = Field( + 'assert-default-sort-order-id', const=True + ) default_sort_order_id: int = Field(..., alias='default-sort-order-id') @@ -511,7 +530,7 @@ class AssertViewUUID(BaseModel): The view UUID must match the requirement's `uuid` """ - type: str = Field('assert-view-uuid', const=True) + type: Literal['assert-view-uuid'] = Field('assert-view-uuid', const=True) uuid: str @@ -619,6 +638,7 @@ class OAuthTokenExchangeRequest(BaseModel): class OAuthTokenRequest(BaseModel): __root__: OAuthClientCredentialsRequest | OAuthTokenExchangeRequest = Field( ..., + deprecated=True, description='The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this spec, see description of the endpoint.', ) @@ -960,27 +980,6 @@ class PlanTask(BaseModel): ) -class ResidualFilter1(BaseModel): - """ - An optional filter to be applied to rows in this file scan task. - If the residual is not present, the client must produce the residual or use the original filter. - """ - - -class ResidualFilter2(TrueExpression, ResidualFilter1): - """ - An optional filter to be applied to rows in this file scan task. - If the residual is not present, the client must produce the residual or use the original filter. - """ - - -class ResidualFilter3(FalseExpression, ResidualFilter1): - """ - An optional filter to be applied to rows in this file scan task. - If the residual is not present, the client must produce the residual or use the original filter. - """ - - class CreateNamespaceRequest(BaseModel): namespace: Namespace properties: dict[str, str] | None = Field( @@ -996,13 +995,15 @@ class RenameTableRequest(BaseModel): class TransformTerm(BaseModel): - type: str = Field('transform', const=True) + type: Literal['transform'] = Field('transform', const=True) transform: Transform term: Reference class SetPartitionStatisticsUpdate(BaseUpdate): - action: str = Field('set-partition-statistics', const=True) + action: Literal['set-partition-statistics'] = Field( + 'set-partition-statistics', const=True + ) partition_statistics: PartitionStatisticsFile = Field( ..., alias='partition-statistics' ) @@ -1042,7 +1043,7 @@ class ValueMap(BaseModel): class DataFile(ContentFile): - content: str = Field(..., const=True) + content: Literal['data'] = Field(..., const=True) first_row_id: int | None = Field( None, alias='first-row-id', @@ -1093,10 +1094,11 @@ class Term(BaseModel): class SetStatisticsUpdate(BaseUpdate): - action: str = Field('set-statistics', const=True) + action: Literal['set-statistics'] = Field('set-statistics', const=True) snapshot_id: int | None = Field( None, alias='snapshot-id', + deprecated=True, description='This optional field is **DEPRECATED for REMOVAL** since it contains redundant information. Clients should use the `statistics.snapshot-id` field instead.', ) statistics: StatisticsFile @@ -1121,27 +1123,6 @@ class SetExpression(BaseModel): values: list[PrimitiveTypeValue] -class ResidualFilter6(SetExpression, ResidualFilter1): - """ - An optional filter to be applied to rows in this file scan task. - If the residual is not present, the client must produce the residual or use the original filter. - """ - - -class ResidualFilter7(LiteralExpression, ResidualFilter1): - """ - An optional filter to be applied to rows in this file scan task. - If the residual is not present, the client must produce the residual or use the original filter. - """ - - -class ResidualFilter8(UnaryExpression, ResidualFilter1): - """ - An optional filter to be applied to rows in this file scan task. - If the residual is not present, the client must produce the residual or use the original filter. - """ - - class StructField(BaseModel): id: int name: str @@ -1153,19 +1134,19 @@ class StructField(BaseModel): class StructType(BaseModel): - type: str = Field('struct', const=True) + type: Literal['struct'] = Field('struct', const=True) fields: list[StructField] class ListType(BaseModel): - type: str = Field('list', const=True) + type: Literal['list'] = Field('list', const=True) element_id: int = Field(..., alias='element-id') element: Type element_required: bool = Field(..., alias='element-required') class MapType(BaseModel): - type: str = Field('map', const=True) + type: Literal['map'] = Field('map', const=True) key_id: int = Field(..., alias='key-id') key: Type value_id: int = Field(..., alias='value-id') @@ -1196,7 +1177,7 @@ class AndOrExpression(BaseModel): class NotExpression(BaseModel): - type: str = Field('not', const=True) + type: Literal['not'] = Field('not', const=True) child: Expression @@ -1244,11 +1225,12 @@ class ViewMetadata(BaseModel): class AddSchemaUpdate(BaseUpdate): - action: str = Field('add-schema', const=True) + action: Literal['add-schema'] = Field('add-schema', const=True) schema_: Schema = Field(..., alias='schema') last_column_id: int | None = Field( None, alias='last-column-id', + deprecated=True, description="This optional field is **DEPRECATED for REMOVAL** since it more safe to handle this internally, and shouldn't be exposed to the clients.\nThe highest assigned column ID for the table. This is used to ensure columns are always assigned an unused ID when evolving schemas. When omitted, it will be computed on the server side.", ) @@ -1524,26 +1506,6 @@ class PlanTableScanRequest(BaseModel): ) -class ResidualFilter(BaseModel): - """ - An optional filter to be applied to rows in this file scan task. - If the residual is not present, the client must produce the residual or use the original filter. - """ - - __root__: ( - ResidualFilter2 - | ResidualFilter3 - | ResidualFilter4 - | ResidualFilter5 - | ResidualFilter6 - | ResidualFilter7 - | ResidualFilter8 - ) = Field( - ..., - description='An optional filter to be applied to rows in this file scan task.\nIf the residual is not present, the client must produce the residual or use the original filter.', - ) - - class FileScanTask(BaseModel): data_file: DataFile = Field(..., alias='data-file') delete_file_references: list[int] | None = Field( @@ -1551,7 +1513,7 @@ class FileScanTask(BaseModel): alias='delete-file-references', description='A list of indices in the delete files array (0-based)', ) - residual_filter: ResidualFilter | None = Field( + residual_filter: Expression | None = Field( None, alias='residual-filter', description='An optional filter to be applied to rows in this file scan task.\nIf the residual is not present, the client must produce the residual or use the original filter.', @@ -1563,20 +1525,6 @@ class Schema(StructType): identifier_field_ids: list[int] | None = Field(None, alias='identifier-field-ids') -class ResidualFilter4(AndOrExpression, ResidualFilter1): - """ - An optional filter to be applied to rows in this file scan task. - If the residual is not present, the client must produce the residual or use the original filter. - """ - - -class ResidualFilter5(NotExpression, ResidualFilter1): - """ - An optional filter to be applied to rows in this file scan task. - If the residual is not present, the client must produce the residual or use the original filter. - """ - - class CompletedPlanningResult(ScanTasks): """ Completed server-side planning result @@ -1620,7 +1568,6 @@ class CompletedPlanningWithIDResult(CompletedPlanningResult): CreateTableRequest.update_forward_refs() CreateViewRequest.update_forward_refs() ReportMetricsRequest.update_forward_refs() -ResidualFilter.update_forward_refs() CompletedPlanningResult.update_forward_refs() FetchScanTasksResult.update_forward_refs() CompletedPlanningWithIDResult.update_forward_refs() From 3048d772aed6572bce28abbddae661d441058f74 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Mon, 5 Jan 2026 14:22:39 +0100 Subject: [PATCH 144/201] Flink: Dynamic Sink: Fix serialization issues with schemas larger than 2^16 bytes (#14880) --- .../DynamicRecordInternalSerializer.java | 103 ++++++++++++--- .../source/split/IcebergSourceSplit.java | 1 + .../split => util}/SerializerHelper.java | 15 ++- ...namicRecordInternalSerializerTestBase.java | 8 +- .../TestDynamicRecordInternalSerializer.java | 119 ++++++++++++++++++ ...icRecordInternalSerializerWriteSchema.java | 4 +- ...RecordInternalSerializerWriteSchemaId.java | 4 +- ...nternalSerializerWriteSchemaIdLongUTF.java | 28 +++++ ...dInternalSerializerWriteSchemaLongUTF.java | 28 +++++ 9 files changed, 284 insertions(+), 26 deletions(-) rename flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/{source/split => util}/SerializerHelper.java (92%) create mode 100644 flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializer.java create mode 100644 flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF.java create mode 100644 flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaLongUTF.java diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java index 22b7742913e7..0d758ace1b19 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.Set; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; @@ -34,6 +35,9 @@ import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.flink.util.SerializerHelper; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @Internal @@ -43,18 +47,27 @@ class DynamicRecordInternalSerializer extends TypeSerializer duplicate() { return new DynamicRecordInternalSerializer( new TableSerializerCache(serializerCache.catalogLoader(), serializerCache.maximumSize()), - writeSchemaAndSpec); + writeSchemaAndSpec, + writeLongUTF); } @Override @@ -68,7 +81,12 @@ public void serialize(DynamicRecordInternal toSerialize, DataOutputView dataOutp dataOutputView.writeUTF(toSerialize.tableName()); dataOutputView.writeUTF(toSerialize.branch()); if (writeSchemaAndSpec) { - dataOutputView.writeUTF(SchemaParser.toJson(toSerialize.schema())); + if (writeLongUTF) { + SerializerHelper.writeLongUTF(dataOutputView, SchemaParser.toJson(toSerialize.schema())); + } else { + dataOutputView.writeUTF(SchemaParser.toJson(toSerialize.schema())); + } + dataOutputView.writeUTF(PartitionSpecParser.toJson(toSerialize.spec())); } else { dataOutputView.writeInt(toSerialize.schema().schemaId()); @@ -108,7 +126,12 @@ public DynamicRecordInternal deserialize(DataInputView dataInputView) throws IOE final PartitionSpec spec; final RowDataSerializer rowDataSerializer; if (writeSchemaAndSpec) { - schema = SchemaParser.fromJson(dataInputView.readUTF()); + if (writeLongUTF) { + schema = SchemaParser.fromJson(SerializerHelper.readLongUTF(dataInputView)); + } else { + schema = SchemaParser.fromJson(dataInputView.readUTF()); + } + spec = PartitionSpecParser.fromJson(schema, dataInputView.readUTF()); rowDataSerializer = serializerCache.serializer(tableName, schema, spec); } else { @@ -152,7 +175,12 @@ public DynamicRecordInternal deserialize(DynamicRecordInternal reuse, DataInputV final PartitionSpec spec; final RowDataSerializer rowDataSerializer; if (writeSchemaAndSpec) { - schema = SchemaParser.fromJson(dataInputView.readUTF()); + if (writeLongUTF) { + schema = SchemaParser.fromJson(SerializerHelper.readLongUTF(dataInputView)); + } else { + schema = SchemaParser.fromJson(dataInputView.readUTF()); + } + spec = PartitionSpecParser.fromJson(schema, dataInputView.readUTF()); reuse.setSchema(schema); reuse.setSpec(spec); @@ -245,25 +273,32 @@ public int getLength() { @Override public TypeSerializerSnapshot snapshotConfiguration() { - return new DynamicRecordInternalTypeSerializerSnapshot(writeSchemaAndSpec); + return new DynamicRecordInternalTypeSerializerSnapshot(writeSchemaAndSpec, serializerCache); } public static class DynamicRecordInternalTypeSerializerSnapshot implements TypeSerializerSnapshot { + private static final int MOST_RECENT_VERSION = 1; + private boolean writeSchemaAndSpec; + private int version; + private TableSerializerCache serializerCache; - // Zero args constructor is required to instantiate this class on restore + // Zero args constructor is required to instantiate this class on restore via readSnapshot(..) @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) public DynamicRecordInternalTypeSerializerSnapshot() {} - DynamicRecordInternalTypeSerializerSnapshot(boolean writeSchemaAndSpec) { + DynamicRecordInternalTypeSerializerSnapshot( + boolean writeSchemaAndSpec, TableSerializerCache serializerCache) { this.writeSchemaAndSpec = writeSchemaAndSpec; + this.serializerCache = serializerCache; + this.version = MOST_RECENT_VERSION; } @Override public int getCurrentVersion() { - return 0; + return version; } @Override @@ -274,22 +309,62 @@ public void writeSnapshot(DataOutputView out) throws IOException { @Override public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { + this.version = readVersion; this.writeSchemaAndSpec = in.readBoolean(); } @Override public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializerSnapshot oldSerializerSnapshot) { - return TypeSerializerSchemaCompatibility.compatibleAsIs(); + if (oldSerializerSnapshot.getCurrentVersion() == getCurrentVersion()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + // Old TypeSerializerSnapshots do not contain the serializer cache, but the newest one does. + // This will also ensure that we always use the up-to-date cache alongside with its catalog + // configuration. + Preconditions.checkNotNull(serializerCache, "serializerCache should not be null"); + try { + DynMethods.builder("initializeSerializerCache") + .hiddenImpl( + DynamicRecordInternalTypeSerializerSnapshot.class, TableSerializerCache.class) + .build() + .invoke(oldSerializerSnapshot, serializerCache); + } catch (Exception e) { + throw new RuntimeException( + "Failed to initialize serializerCache for reading data with old serializer", e); + } + + // This will first read data with the old serializer, then switch to the most recent one. + return TypeSerializerSchemaCompatibility.compatibleAfterMigration(); } @Override public TypeSerializer restoreSerializer() { - // Note: We pass in a null serializer cache which would create issues if we tried to use this - // restored serializer, but since we are using {@code - // TypeSerializerSchemaCompatibility.compatibleAsIs()} above, this serializer will never be - // used. A new one will be created via {@code DynamicRecordInternalType}. - return new DynamicRecordInternalSerializer(null, writeSchemaAndSpec); + if (getCurrentVersion() < MOST_RECENT_VERSION) { + // If this serializer is not the most recent one, we need to read old data with the correct + // parameters. + return new DynamicRecordInternalSerializer(serializerCache, writeSchemaAndSpec, false); + } + + // In all other cases, we just use the newest serializer. + return new DynamicRecordInternalSerializer(serializerCache, writeSchemaAndSpec, true); + } + + /** + * We need to lazily initialize the cache from the up-to-date serializer which has the current + * CatalogLoader available. + * + *

    This method must not be removed! + */ + @SuppressWarnings("unused") + private void initializeSerializerCache(TableSerializerCache cache) { + this.serializerCache = cache; } } + + @VisibleForTesting + TableSerializerCache getSerializerCache() { + return serializerCache; + } } diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java index b6d6f60ef673..0cc1c633f906 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -33,6 +33,7 @@ import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.ScanTaskParser; +import org.apache.iceberg.flink.util.SerializerHelper; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/util/SerializerHelper.java similarity index 92% rename from flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java rename to flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/util/SerializerHelper.java index 841969666ee5..3a161ea2631f 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/util/SerializerHelper.java @@ -16,20 +16,24 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.source.split; +package org.apache.iceberg.flink.util; import java.io.IOException; import java.io.Serializable; import java.io.UTFDataFormatException; +import org.apache.flink.annotation.Internal; import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.core.memory.DataOutputView; /** * Helper class to serialize and deserialize strings longer than 65K. The inspiration is mostly * taken from the class org.apache.flink.core.memory.DataInputSerializer.readUTF and * org.apache.flink.core.memory.DataOutputSerializer.writeUTF. */ -class SerializerHelper implements Serializable { +@Internal +public class SerializerHelper implements Serializable { private SerializerHelper() {} @@ -47,7 +51,7 @@ private SerializerHelper() {} * @param out the output stream to write the string to. * @param str the string value to be written. */ - public static void writeLongUTF(DataOutputSerializer out, String str) throws IOException { + public static void writeLongUTF(DataOutputView out, String str) throws IOException { int strlen = str.length(); long utflen = 0; int ch; @@ -85,7 +89,7 @@ public static void writeLongUTF(DataOutputSerializer out, String str) throws IOE * @return the string value read from the input stream. * @throws IOException if an I/O error occurs when reading from the input stream. */ - public static String readLongUTF(DataInputDeserializer in) throws IOException { + public static String readLongUTF(DataInputView in) throws IOException { int utflen = in.readInt(); byte[] bytearr = new byte[utflen]; char[] chararr = new char[utflen]; @@ -168,8 +172,7 @@ private static int getUTFBytesSize(int ch) { } } - private static void writeUTFBytes(DataOutputSerializer out, String str, int utflen) - throws IOException { + private static void writeUTFBytes(DataOutputView out, String str, int utflen) throws IOException { int strlen = str.length(); int ch; diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java index 30782e8d4170..07096b891a95 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java @@ -55,15 +55,19 @@ abstract class DynamicRecordInternalSerializerTestBase static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).bucket("id", 10).build(); private boolean writeFullSchemaAndSpec; + private final boolean writeLongUTF; - DynamicRecordInternalSerializerTestBase(boolean writeFullSchemaAndSpec) { + DynamicRecordInternalSerializerTestBase(boolean writeFullSchemaAndSpec, boolean writeLongUTF) { this.writeFullSchemaAndSpec = writeFullSchemaAndSpec; + this.writeLongUTF = writeLongUTF; } @Override protected TypeSerializer createSerializer() { return new DynamicRecordInternalSerializer( - new TableSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), writeFullSchemaAndSpec); + new TableSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), + writeFullSchemaAndSpec, + writeLongUTF); } @BeforeEach diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializer.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializer.java new file mode 100644 index 000000000000..388ff303339c --- /dev/null +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializer.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestDynamicRecordInternalSerializer { + + @RegisterExtension + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension("db", "table"); + + @Test + void testCurrentTypeSerializerSnapshotVersion() { + TypeSerializer serializer = createSerializer(); + assertThat(serializer).isNotNull().isInstanceOf(DynamicRecordInternalSerializer.class); + TypeSerializerSnapshot snapshot = serializer.snapshotConfiguration(); + assertThat(snapshot.getCurrentVersion()).isEqualTo(1); + } + + @Test + void testCurrentTypeSerializerSnapshotCompatibility() { + TypeSerializer serializer = createSerializer(); + assertThat(serializer).isNotNull().isInstanceOf(DynamicRecordInternalSerializer.class); + TypeSerializerSnapshot snapshot = serializer.snapshotConfiguration(); + assertThat( + snapshot + .resolveSchemaCompatibility(serializer.snapshotConfiguration()) + .isCompatibleAsIs()) + .isTrue(); + } + + @Test + void testRestoreFromOldVersion() throws IOException { + // Create a serialized snapshot of the TypeSerializer + final int oldVersion = 0; + OldTypeSerializerSnapshot oldTypeSerializerSnapshot = new OldTypeSerializerSnapshot(oldVersion); + assertThat(oldTypeSerializerSnapshot.getCurrentVersion()).isEqualTo(oldVersion); + DataOutputSerializer dataOutputSerializer = new DataOutputSerializer(128); + oldTypeSerializerSnapshot.writeSnapshot(dataOutputSerializer); + + // Load the serialized state + DynamicRecordInternalSerializer.DynamicRecordInternalTypeSerializerSnapshot restoreSnapshot = + (DynamicRecordInternalSerializer.DynamicRecordInternalTypeSerializerSnapshot) + createSerializer().snapshotConfiguration(); + restoreSnapshot.readSnapshot( + oldVersion, + new DataInputDeserializer(dataOutputSerializer.getSharedBuffer()), + getClass().getClassLoader()); + // Check that it matches the original one + assertThat(restoreSnapshot.getCurrentVersion()).isEqualTo(oldVersion); + assertThat( + restoreSnapshot + .resolveSchemaCompatibility(oldTypeSerializerSnapshot) + .isCompatibleAsIs()) + .isTrue(); + TypeSerializer restoreSerializer = restoreSnapshot.restoreSerializer(); + assertThat(restoreSerializer).isInstanceOf(DynamicRecordInternalSerializer.class); + assertThat(((DynamicRecordInternalSerializer) restoreSerializer).getSerializerCache()) + .isNotNull(); + + // Compare against the latest version of a snapshot + TypeSerializerSnapshot latestVersion = + createSerializer().snapshotConfiguration(); + assertThat(latestVersion.getCurrentVersion()).isEqualTo(1); + assertThat( + latestVersion + .resolveSchemaCompatibility(oldTypeSerializerSnapshot) + .isCompatibleAfterMigration()) + .isTrue(); + assertThat( + latestVersion.resolveSchemaCompatibility(restoreSnapshot).isCompatibleAfterMigration()) + .isTrue(); + } + + private DynamicRecordInternalSerializer createSerializer() { + return new DynamicRecordInternalSerializer( + new TableSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), true); + } + + private static class OldTypeSerializerSnapshot + extends DynamicRecordInternalSerializer.DynamicRecordInternalTypeSerializerSnapshot { + + private final int version; + + OldTypeSerializerSnapshot(int version) { + this.version = version; + } + + @Override + public int getCurrentVersion() { + return version; + } + } +} diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java index ab8ce98c3594..e7aa2d24d9d0 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java @@ -18,11 +18,11 @@ */ package org.apache.iceberg.flink.sink.dynamic; -/** Test writing DynamicRecord with the full schema */ +/** Test writing DynamicRecord with the full schema and standard UTF encoding */ class TestDynamicRecordInternalSerializerWriteSchema extends DynamicRecordInternalSerializerTestBase { TestDynamicRecordInternalSerializerWriteSchema() { - super(true); + super(true /* writeFullSchemaAndSpec */, false /* writeLongUTF */); } } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java index 1d8890546214..bff0fd5c6aad 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java @@ -18,11 +18,11 @@ */ package org.apache.iceberg.flink.sink.dynamic; -/** Test writing DynamicRecord with only the schema id. */ +/** Test writing DynamicRecord with only the schema id and standard UTF encoding */ class TestDynamicRecordInternalSerializerWriteSchemaId extends DynamicRecordInternalSerializerTestBase { TestDynamicRecordInternalSerializerWriteSchemaId() { - super(false); + super(false /* writeFullSchemaAndSpec */, false /* writeLongUTF */); } } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF.java new file mode 100644 index 000000000000..7a1ae3df3806 --- /dev/null +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +/** Test writing DynamicRecord with only the schema id and long UTF encoding */ +class TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF + extends DynamicRecordInternalSerializerTestBase { + + TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF() { + super(false /* writeFullSchemaAndSpec */, true /* writeLongUTF */); + } +} diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaLongUTF.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaLongUTF.java new file mode 100644 index 000000000000..faff8921db5d --- /dev/null +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaLongUTF.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +/** Test writing DynamicRecord with the full schema and long UTF encoding */ +class TestDynamicRecordInternalSerializerWriteSchemaLongUTF + extends DynamicRecordInternalSerializerTestBase { + + TestDynamicRecordInternalSerializerWriteSchemaLongUTF() { + super(true /* writeFullSchemaAndSpec */, true /* writeLongUTF */); + } +} From 4bd1fb8632ecde781fdf7e444137f0981c0e86bb Mon Sep 17 00:00:00 2001 From: aiborodin Date: Tue, 6 Jan 2026 00:45:28 +1100 Subject: [PATCH 145/201] Flink: DynamicSink: Report writer records/bytes send metrics (#14878) --- .../flink/sink/dynamic/DynamicWriter.java | 1 + .../sink/dynamic/DynamicWriterMetrics.java | 31 +++++++++++++++++-- .../flink/sink/dynamic/TestDynamicWriter.java | 2 +- 3 files changed, 30 insertions(+), 4 deletions(-) diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java index c2a303285801..907385797495 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java @@ -145,6 +145,7 @@ public void write(DynamicRecordInternal element, Context context) return taskWriterFactory.create(); }) .write(element.rowData()); + metrics.mainMetricsGroup().getNumRecordsSendCounter().inc(); } @Override diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java index 2e1f82df9d2d..d50a41512a15 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java @@ -18,24 +18,37 @@ */ package org.apache.iceberg.flink.sink.dynamic; +import java.util.Arrays; import java.util.Map; -import org.apache.flink.metrics.MetricGroup; +import java.util.function.ToLongFunction; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.flink.sink.IcebergStreamWriterMetrics; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.ScanTaskUtil; class DynamicWriterMetrics { private final Map metrics; - private final MetricGroup mainMetricsGroup; + private final SinkWriterMetricGroup mainMetricsGroup; - DynamicWriterMetrics(MetricGroup mainMetricsGroup) { + DynamicWriterMetrics(SinkWriterMetricGroup mainMetricsGroup) { this.mainMetricsGroup = mainMetricsGroup; this.metrics = Maps.newHashMap(); } + SinkWriterMetricGroup mainMetricsGroup() { + return this.mainMetricsGroup; + } + public void updateFlushResult(String fullTableName, WriteResult result) { writerMetrics(fullTableName).updateFlushResult(result); + + long bytesOutTotal = sum(result.dataFiles()) + sum(result.deleteFiles()); + this.mainMetricsGroup.getNumBytesSendCounter().inc(bytesOutTotal); } public void flushDuration(String fullTableName, long flushDurationMs) { @@ -46,4 +59,16 @@ IcebergStreamWriterMetrics writerMetrics(String fullTableName) { return metrics.computeIfAbsent( fullTableName, tableName -> new IcebergStreamWriterMetrics(mainMetricsGroup, tableName)); } + + private static long sum(DataFile[] files) { + return sum(files, DataFile::fileSizeInBytes); + } + + private static long sum(DeleteFile[] files) { + return sum(files, ScanTaskUtil::contentSizeInBytes); + } + + private static > long sum(T[] files, ToLongFunction sizeExtractor) { + return Arrays.stream(files).mapToLong(sizeExtractor).sum(); + } } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java index 689fd20483c1..d17848225f69 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java @@ -243,7 +243,7 @@ void testUniqueFileSuffixOnFactoryRecreation() throws Exception { 1024L, properties, 100, - new DynamicWriterMetrics(new UnregisteredMetricsGroup()), + new DynamicWriterMetrics(UnregisteredMetricsGroup.createSinkWriterMetricGroup()), 0, 0); return dynamicWriter; From 4bc934b9d3c4394566658cb2b11639f4daef7c8c Mon Sep 17 00:00:00 2001 From: Prashant Singh <35593236+singhpk234@users.noreply.github.com> Date: Mon, 5 Jan 2026 08:18:22 -0800 Subject: [PATCH 146/201] Spark 3.4 | 3.5: Enable remote scan planning (#14963) Co-authored-by: Prashant Kumar Singh --- .../extensions/TestRemoteScanPlanning.java | 58 +++++++++ .../spark/source/SparkScanBuilder.java | 5 +- .../apache/iceberg/spark/sql/TestSelect.java | 102 +++++++++------ .../extensions/TestRemoteScanPlanning.java | 58 +++++++++ .../spark/source/SparkScanBuilder.java | 5 +- .../apache/iceberg/spark/sql/TestSelect.java | 120 +++++++++++------- 6 files changed, 259 insertions(+), 89 deletions(-) create mode 100644 spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java create mode 100644 spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java new file mode 100644 index 000000000000..14e6c358898c --- /dev/null +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java @@ -0,0 +1,58 @@ +/* + * 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.extensions; + +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.RESTCatalogProperties; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.sql.TestSelect; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRemoteScanPlanning extends TestSelect { + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, binaryTableName = {3}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.REST.catalogName(), + SparkCatalogConfig.REST.implementation(), + ImmutableMap.builder() + .putAll(SparkCatalogConfig.REST.properties()) + .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) + // this flag is typically only set by the server, but we set it from the client for + // testing + .put(RESTCatalogProperties.REST_SCAN_PLANNING_ENABLED, "true") + .build(), + SparkCatalogConfig.REST.catalogName() + ".default.binary_table" + } + }; + } + + @TestTemplate + @Disabled( + "binary filter that is used by Spark is not working because ExpressionParser.fromJSON doesn't have the Schema to properly parse the filter expression") + public void testBinaryInFilter() { + super.testBinaryInFilter(); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 11b0ba58af51..dd914f1617bd 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -34,6 +34,7 @@ import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.MetricsModes; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RequiresRemoteScanPlanning; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SparkDistributedDataScan; @@ -760,7 +761,9 @@ public StructType readSchema() { } private BatchScan newBatchScan() { - if (table instanceof BaseTable && readConf.distributedPlanningEnabled()) { + if (table instanceof RequiresRemoteScanPlanning) { + return table.newBatchScan(); + } else if (table instanceof BaseTable && readConf.distributedPlanningEnabled()) { return new SparkDistributedDataScan(spark, table, readConf); } else { return table.newBatchScan(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index d4187a266b94..5fa894726716 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -107,7 +107,10 @@ public void testSelect() { List expected = ImmutableList.of(row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", Float.NaN)); - assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT * FROM %s ORDER BY id", tableName)); } @TestTemplate @@ -118,7 +121,10 @@ public void testSelectWithSpecifiedTargetSplitSize() { Table table = validationCatalog.loadTable(tableIdent); table.updateProperties().set("read.split.target-size", "1024").commit(); spark.sql("REFRESH TABLE " + tableName); - assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT * FROM %s ORDER BY id", tableName)); // Query failed when `SPLIT_SIZE` < 0 table.updateProperties().set(SPLIT_SIZE, "-1").commit(); @@ -158,16 +164,21 @@ public void selectWithLimit() { // verify that LIMIT is properly applied in case SupportsPushDownLimit.isPartiallyPushed() is // ever overridden in SparkScanBuilder - assertThat(sql("SELECT * FROM %s LIMIT 1", tableName)).containsExactly(first); - assertThat(sql("SELECT * FROM %s LIMIT 2", tableName)).containsExactly(first, second); - assertThat(sql("SELECT * FROM %s LIMIT 3", tableName)).containsExactly(first, second, third); + assertThat(sql("SELECT * FROM %s ORDER BY id LIMIT 1", tableName)).containsExactly(first); + assertThat(sql("SELECT * FROM %s ORDER BY id LIMIT 2", tableName)) + .containsExactly(first, second); + assertThat(sql("SELECT * FROM %s ORDER BY id LIMIT 3", tableName)) + .containsExactly(first, second, third); } @TestTemplate public void testProjection() { List expected = ImmutableList.of(row(1L), row(2L), row(3L)); - assertEquals("Should return all expected rows", expected, sql("SELECT id FROM %s", tableName)); + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT id FROM %s ORDER BY id", tableName)); assertThat(scanEventCount).as("Should create only one scan").isEqualTo(1); assertThat(lastScanEvent.filter()) @@ -209,14 +220,14 @@ public void testMetadataTables() { public void testSnapshotInTableName() { // get the snapshot ID of the last write and get the current row set as expected long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); String prefix = "snapshot_id_"; // read the table at the snapshot - List actual = sql("SELECT * FROM %s.%s", tableName, prefix + snapshotId); + List actual = sql("SELECT * FROM %s.%s ORDER BY id", tableName, prefix + snapshotId); assertEquals("Snapshot at specific ID, prefix " + prefix, expected, actual); // read the table using DataFrameReader option @@ -225,7 +236,8 @@ public void testSnapshotInTableName() { .read() .format("iceberg") .option(SparkReadOptions.SNAPSHOT_ID, snapshotId) - .load(tableName); + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at specific ID " + snapshotId, expected, fromDF); } @@ -235,14 +247,14 @@ 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); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); String prefix = "at_timestamp_"; // read the table at the snapshot - List actual = sql("SELECT * FROM %s.%s", tableName, prefix + timestamp); + List actual = sql("SELECT * FROM %s.%s ORDER BY id", tableName, prefix + timestamp); assertEquals("Snapshot at timestamp, prefix " + prefix, expected, actual); // read the table using DataFrameReader option @@ -251,7 +263,8 @@ public void testTimestampInTableName() { .read() .format("iceberg") .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) - .load(tableName); + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at timestamp " + timestamp, expected, fromDF); } @@ -260,19 +273,20 @@ public void testTimestampInTableName() { public void testVersionAsOf() { // get the snapshot ID of the last write and get the current row set as expected long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); // read the table at the snapshot - List actual1 = sql("SELECT * FROM %s VERSION AS OF %s", tableName, snapshotId); + List actual1 = + sql("SELECT * FROM %s VERSION AS OF %s ORDER BY id", tableName, snapshotId); assertEquals("Snapshot at specific ID", expected, actual1); // read the table at the snapshot // HIVE time travel syntax List actual2 = - sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF %s", tableName, snapshotId); + sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF %s ORDER BY id", tableName, snapshotId); assertEquals("Snapshot at specific ID", expected, actual2); // read the table using DataFrameReader option: versionAsOf @@ -281,7 +295,8 @@ public void testVersionAsOf() { .read() .format("iceberg") .option(SparkReadOptions.VERSION_AS_OF, snapshotId) - .load(tableName); + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at specific ID " + snapshotId, expected, fromDF); } @@ -291,28 +306,35 @@ public void testTagReference() { Table table = validationCatalog.loadTable(tableIdent); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createTag("test_tag", snapshotId).commit(); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot, read the table at the tag sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); - List actual1 = sql("SELECT * FROM %s VERSION AS OF 'test_tag'", tableName); + List actual1 = + sql("SELECT * FROM %s VERSION AS OF 'test_tag' ORDER BY id", tableName); assertEquals("Snapshot at specific tag reference name", expected, actual1); // read the table at the tag // HIVE time travel syntax - List actual2 = sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_tag'", tableName); + List actual2 = + sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_tag' ORDER BY id", tableName); assertEquals("Snapshot at specific tag reference name", expected, actual2); // Spark session catalog does not support extended table names if (!"spark_catalog".equals(catalogName)) { // read the table using the "tag_" prefix in the table name - List actual3 = sql("SELECT * FROM %s.tag_test_tag", tableName); + List actual3 = sql("SELECT * FROM %s.tag_test_tag ORDER BY id", tableName); assertEquals("Snapshot at specific tag reference name, prefix", expected, actual3); } // read the table using DataFrameReader option: tag Dataset df = - spark.read().format("iceberg").option(SparkReadOptions.TAG, "test_tag").load(tableName); + spark + .read() + .format("iceberg") + .option(SparkReadOptions.TAG, "test_tag") + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at specific tag reference name", expected, fromDF); } @@ -323,7 +345,7 @@ public void testUseSnapshotIdForTagReferenceAsOf() { long snapshotId1 = table.currentSnapshot().snapshotId(); // create a second snapshot, read the table at the snapshot - List actual = sql("SELECT * FROM %s", tableName); + List actual = sql("SELECT * FROM %s ORDER BY id", tableName); sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); table.refresh(); @@ -334,11 +356,11 @@ public void testUseSnapshotIdForTagReferenceAsOf() { // this means if a tag name matches a snapshot ID, it will always choose snapshotID to travel // to. List travelWithStringResult = - sql("SELECT * FROM %s VERSION AS OF '%s'", tableName, snapshotId1); + sql("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, snapshotId1); assertEquals("Snapshot at specific tag reference name", actual, travelWithStringResult); List travelWithLongResult = - sql("SELECT * FROM %s VERSION AS OF %s", tableName, snapshotId1); + sql("SELECT * FROM %s VERSION AS OF %s ORDER BY id", tableName, snapshotId1); assertEquals("Snapshot at specific tag reference name", actual, travelWithLongResult); } @@ -350,7 +372,7 @@ public void readAndWriteWithBranchAfterSchemaChange() { List expected = Arrays.asList(row(1L, "a", 1.0f), row(2L, "b", 2.0f), row(3L, "c", Float.NaN)); - assertThat(sql("SELECT * FROM %s", tableName)).containsExactlyElementsOf(expected); + assertThat(sql("SELECT * FROM %s", tableName)).containsExactlyInAnyOrderElementsOf(expected); // change schema on the table and add more data sql("ALTER TABLE %s DROP COLUMN float", tableName); @@ -361,16 +383,16 @@ public void readAndWriteWithBranchAfterSchemaChange() { // time-travel query using snapshot id should return the snapshot's schema long branchSnapshotId = table.refs().get(branchName).snapshotId(); - assertThat(sql("SELECT * FROM %s VERSION AS OF %s", tableName, branchSnapshotId)) + assertThat(sql("SELECT * FROM %s VERSION AS OF %s ORDER BY id", tableName, branchSnapshotId)) .containsExactlyElementsOf(expected); // querying the head of the branch should return the table's schema - assertThat(sql("SELECT * FROM %s VERSION AS OF '%s'", tableName, branchName)) + assertThat(sql("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, branchName)) .containsExactly(row(1L, "a", null), row(2L, "b", null), row(3L, "c", null)); if (!"spark_catalog".equals(catalogName)) { // querying the head of the branch using 'branch_' should return the table's schema - assertThat(sql("SELECT * FROM %s.branch_%s", tableName, branchName)) + assertThat(sql("SELECT * FROM %s.branch_%s ORDER BY id", tableName, branchName)) .containsExactly(row(1L, "a", null), row(2L, "b", null), row(3L, "c", null)); } @@ -380,7 +402,7 @@ public void readAndWriteWithBranchAfterSchemaChange() { tableName, branchName); // querying the head of the branch returns the table's schema - assertThat(sql("SELECT * FROM %s VERSION AS OF '%s'", tableName, branchName)) + assertThat(sql("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, branchName)) .containsExactlyInAnyOrder( row(1L, "a", null), row(2L, "b", null), @@ -405,23 +427,24 @@ public void testBranchReference() { Table table = validationCatalog.loadTable(tableIdent); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createBranch("test_branch", snapshotId).commit(); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot, read the table at the branch sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); - List actual1 = sql("SELECT * FROM %s VERSION AS OF 'test_branch'", tableName); + List actual1 = + sql("SELECT * FROM %s VERSION AS OF 'test_branch' ORDER BY id", tableName); assertEquals("Snapshot at specific branch reference name", expected, actual1); // read the table at the branch // HIVE time travel syntax List actual2 = - sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_branch'", tableName); + sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_branch' ORDER BY id", tableName); assertEquals("Snapshot at specific branch reference name", expected, actual2); // Spark session catalog does not support extended table names if (!"spark_catalog".equals(catalogName)) { // read the table using the "branch_" prefix in the table name - List actual3 = sql("SELECT * FROM %s.branch_test_branch", tableName); + List actual3 = sql("SELECT * FROM %s.branch_test_branch ORDER BY id", tableName); assertEquals("Snapshot at specific branch reference name, prefix", expected, actual3); } @@ -431,7 +454,8 @@ public void testBranchReference() { .read() .format("iceberg") .option(SparkReadOptions.BRANCH, "test_branch") - .load(tableName); + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at specific branch reference name", expected, fromDF); } @@ -453,30 +477,30 @@ public void testTimestampAsOf() { SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); String formattedDate = sdf.format(new Date(timestamp)); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); // read the table at the timestamp in long format i.e 1656507980463. List actualWithLongFormat = - sql("SELECT * FROM %s TIMESTAMP AS OF %s", tableName, timestampInSeconds); + sql("SELECT * FROM %s TIMESTAMP AS OF %s ORDER BY id", tableName, timestampInSeconds); assertEquals("Snapshot at timestamp", expected, actualWithLongFormat); // read the table at the timestamp in date format i.e 2022-06-29 18:40:37 List actualWithDateFormat = - sql("SELECT * FROM %s TIMESTAMP AS OF '%s'", tableName, formattedDate); + sql("SELECT * FROM %s TIMESTAMP AS OF '%s' ORDER BY id", tableName, formattedDate); assertEquals("Snapshot at timestamp", expected, actualWithDateFormat); // HIVE time travel syntax // read the table at the timestamp in long format i.e 1656507980463. List actualWithLongFormatInHiveSyntax = - sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF %s", tableName, timestampInSeconds); + sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF %s ORDER BY id", tableName, timestampInSeconds); assertEquals("Snapshot at specific ID", expected, actualWithLongFormatInHiveSyntax); // read the table at the timestamp in date format i.e 2022-06-29 18:40:37 List actualWithDateFormatInHiveSyntax = - sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF '%s'", tableName, formattedDate); + sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF '%s' ORDER BY id", tableName, formattedDate); assertEquals("Snapshot at specific ID", expected, actualWithDateFormatInHiveSyntax); // read the table using DataFrameReader option diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java new file mode 100644 index 000000000000..14e6c358898c --- /dev/null +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRemoteScanPlanning.java @@ -0,0 +1,58 @@ +/* + * 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.extensions; + +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.RESTCatalogProperties; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.sql.TestSelect; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRemoteScanPlanning extends TestSelect { + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, binaryTableName = {3}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.REST.catalogName(), + SparkCatalogConfig.REST.implementation(), + ImmutableMap.builder() + .putAll(SparkCatalogConfig.REST.properties()) + .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) + // this flag is typically only set by the server, but we set it from the client for + // testing + .put(RESTCatalogProperties.REST_SCAN_PLANNING_ENABLED, "true") + .build(), + SparkCatalogConfig.REST.catalogName() + ".default.binary_table" + } + }; + } + + @TestTemplate + @Disabled( + "binary filter that is used by Spark is not working because ExpressionParser.fromJSON doesn't have the Schema to properly parse the filter expression") + public void testBinaryInFilter() { + super.testBinaryInFilter(); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java index 11b0ba58af51..dd914f1617bd 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java @@ -34,6 +34,7 @@ import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.MetricsModes; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RequiresRemoteScanPlanning; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SparkDistributedDataScan; @@ -760,7 +761,9 @@ public StructType readSchema() { } private BatchScan newBatchScan() { - if (table instanceof BaseTable && readConf.distributedPlanningEnabled()) { + if (table instanceof RequiresRemoteScanPlanning) { + return table.newBatchScan(); + } else if (table instanceof BaseTable && readConf.distributedPlanningEnabled()) { return new SparkDistributedDataScan(spark, table, readConf); } else { return table.newBatchScan(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 2d4d63c1a127..e58cf72cc5bd 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -108,7 +108,10 @@ public void testSelect() { List expected = ImmutableList.of(row(1L, "a", 1.0F), row(2L, "b", 2.0F), row(3L, "c", Float.NaN)); - assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT * FROM %s ORDER BY id", tableName)); } @TestTemplate @@ -119,7 +122,10 @@ public void testSelectWithSpecifiedTargetSplitSize() { Table table = validationCatalog.loadTable(tableIdent); table.updateProperties().set("read.split.target-size", "1024").commit(); spark.sql("REFRESH TABLE " + tableName); - assertEquals("Should return all expected rows", expected, sql("SELECT * FROM %s", tableName)); + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT * FROM %s ORDER BY id", tableName)); // Query failed when `SPLIT_SIZE` < 0 table.updateProperties().set(SPLIT_SIZE, "-1").commit(); @@ -159,16 +165,20 @@ public void selectWithLimit() { // verify that LIMIT is properly applied in case SupportsPushDownLimit.isPartiallyPushed() is // ever overridden in SparkScanBuilder - assertThat(sql("SELECT * FROM %s LIMIT 1", tableName)).containsExactly(first); - assertThat(sql("SELECT * FROM %s LIMIT 2", tableName)).containsExactly(first, second); - assertThat(sql("SELECT * FROM %s LIMIT 3", tableName)).containsExactly(first, second, third); + assertThat(sql("SELECT * FROM %s ORDER BY id LIMIT 1", tableName)).containsExactly(first); + assertThat(sql("SELECT * FROM %s ORDER BY id LIMIT 2", tableName)) + .containsExactly(first, second); + assertThat(sql("SELECT * FROM %s ORDER BY id LIMIT 3", tableName)) + .containsExactly(first, second, third); } @TestTemplate public void testProjection() { List expected = ImmutableList.of(row(1L), row(2L), row(3L)); - - assertEquals("Should return all expected rows", expected, sql("SELECT id FROM %s", tableName)); + assertEquals( + "Should return all expected rows", + expected, + sql("SELECT id FROM %s ORDER BY id", tableName)); assertThat(scanEventCount).as("Should create only one scan").isEqualTo(1); assertThat(lastScanEvent.filter()) @@ -210,14 +220,14 @@ public void testMetadataTables() { public void testSnapshotInTableName() { // get the snapshot ID of the last write and get the current row set as expected long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); String prefix = "snapshot_id_"; // read the table at the snapshot - List actual = sql("SELECT * FROM %s.%s", tableName, prefix + snapshotId); + List actual = sql("SELECT * FROM %s.%s ORDER BY id", tableName, prefix + snapshotId); assertEquals("Snapshot at specific ID, prefix " + prefix, expected, actual); // read the table using DataFrameReader option @@ -226,7 +236,8 @@ public void testSnapshotInTableName() { .read() .format("iceberg") .option(SparkReadOptions.SNAPSHOT_ID, snapshotId) - .load(tableName); + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at specific ID " + snapshotId, expected, fromDF); } @@ -236,14 +247,14 @@ 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); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); String prefix = "at_timestamp_"; // read the table at the snapshot - List actual = sql("SELECT * FROM %s.%s", tableName, prefix + timestamp); + List actual = sql("SELECT * FROM %s.%s ORDER BY id", tableName, prefix + timestamp); assertEquals("Snapshot at timestamp, prefix " + prefix, expected, actual); // read the table using DataFrameReader option @@ -252,7 +263,8 @@ public void testTimestampInTableName() { .read() .format("iceberg") .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) - .load(tableName); + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at timestamp " + timestamp, expected, fromDF); } @@ -261,19 +273,20 @@ public void testTimestampInTableName() { public void testVersionAsOf() { // get the snapshot ID of the last write and get the current row set as expected long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); // read the table at the snapshot - List actual1 = sql("SELECT * FROM %s VERSION AS OF %s", tableName, snapshotId); + List actual1 = + sql("SELECT * FROM %s VERSION AS OF %s ORDER BY id", tableName, snapshotId); assertEquals("Snapshot at specific ID", expected, actual1); // read the table at the snapshot // HIVE time travel syntax List actual2 = - sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF %s", tableName, snapshotId); + sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF %s ORDER BY id", tableName, snapshotId); assertEquals("Snapshot at specific ID", expected, actual2); // read the table using DataFrameReader option: versionAsOf @@ -282,7 +295,8 @@ public void testVersionAsOf() { .read() .format("iceberg") .option(SparkReadOptions.VERSION_AS_OF, snapshotId) - .load(tableName); + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at specific ID " + snapshotId, expected, fromDF); } @@ -292,28 +306,35 @@ public void testTagReference() { Table table = validationCatalog.loadTable(tableIdent); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createTag("test_tag", snapshotId).commit(); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot, read the table at the tag sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); - List actual1 = sql("SELECT * FROM %s VERSION AS OF 'test_tag'", tableName); + List actual1 = + sql("SELECT * FROM %s VERSION AS OF 'test_tag' ORDER BY id", tableName); assertEquals("Snapshot at specific tag reference name", expected, actual1); // read the table at the tag // HIVE time travel syntax - List actual2 = sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_tag'", tableName); + List actual2 = + sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_tag' ORDER BY id", tableName); assertEquals("Snapshot at specific tag reference name", expected, actual2); // Spark session catalog does not support extended table names if (!"spark_catalog".equals(catalogName)) { // read the table using the "tag_" prefix in the table name - List actual3 = sql("SELECT * FROM %s.tag_test_tag", tableName); + List actual3 = sql("SELECT * FROM %s.tag_test_tag ORDER BY id", tableName); assertEquals("Snapshot at specific tag reference name, prefix", expected, actual3); } // read the table using DataFrameReader option: tag Dataset df = - spark.read().format("iceberg").option(SparkReadOptions.TAG, "test_tag").load(tableName); + spark + .read() + .format("iceberg") + .option(SparkReadOptions.TAG, "test_tag") + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at specific tag reference name", expected, fromDF); } @@ -324,7 +345,7 @@ public void testUseSnapshotIdForTagReferenceAsOf() { long snapshotId1 = table.currentSnapshot().snapshotId(); // create a second snapshot, read the table at the snapshot - List actual = sql("SELECT * FROM %s", tableName); + List actual = sql("SELECT * FROM %s ORDER BY id", tableName); sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); table.refresh(); @@ -335,11 +356,11 @@ public void testUseSnapshotIdForTagReferenceAsOf() { // this means if a tag name matches a snapshot ID, it will always choose snapshotID to travel // to. List travelWithStringResult = - sql("SELECT * FROM %s VERSION AS OF '%s'", tableName, snapshotId1); + sql("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, snapshotId1); assertEquals("Snapshot at specific tag reference name", actual, travelWithStringResult); List travelWithLongResult = - sql("SELECT * FROM %s VERSION AS OF %s", tableName, snapshotId1); + sql("SELECT * FROM %s VERSION AS OF %s ORDER BY id", tableName, snapshotId1); assertEquals("Snapshot at specific tag reference name", actual, travelWithLongResult); } @@ -348,23 +369,24 @@ public void testBranchReference() { Table table = validationCatalog.loadTable(tableIdent); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createBranch("test_branch", snapshotId).commit(); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot, read the table at the branch sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); - List actual1 = sql("SELECT * FROM %s VERSION AS OF 'test_branch'", tableName); + List actual1 = + sql("SELECT * FROM %s VERSION AS OF 'test_branch' ORDER BY id", tableName); assertEquals("Snapshot at specific branch reference name", expected, actual1); // read the table at the branch // HIVE time travel syntax List actual2 = - sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_branch'", tableName); + sql("SELECT * FROM %s FOR SYSTEM_VERSION AS OF 'test_branch' ORDER BY id", tableName); assertEquals("Snapshot at specific branch reference name", expected, actual2); // Spark session catalog does not support extended table names if (!"spark_catalog".equals(catalogName)) { // read the table using the "branch_" prefix in the table name - List actual3 = sql("SELECT * FROM %s.branch_test_branch", tableName); + List actual3 = sql("SELECT * FROM %s.branch_test_branch ORDER BY id", tableName); assertEquals("Snapshot at specific branch reference name, prefix", expected, actual3); } @@ -374,7 +396,8 @@ public void testBranchReference() { .read() .format("iceberg") .option(SparkReadOptions.BRANCH, "test_branch") - .load(tableName); + .load(tableName) + .orderBy("id"); List fromDF = rowsToJava(df.collectAsList()); assertEquals("Snapshot at specific branch reference name", expected, fromDF); } @@ -387,7 +410,7 @@ public void readAndWriteWithBranchAfterSchemaChange() { List expected = Arrays.asList(row(1L, "a", 1.0f), row(2L, "b", 2.0f), row(3L, "c", Float.NaN)); - assertThat(sql("SELECT * FROM %s", tableName)).containsExactlyElementsOf(expected); + assertThat(sql("SELECT * FROM %s", tableName)).containsExactlyInAnyOrderElementsOf(expected); // change schema on the table and add more data sql("ALTER TABLE %s DROP COLUMN float", tableName); @@ -398,16 +421,16 @@ public void readAndWriteWithBranchAfterSchemaChange() { // time-travel query using snapshot id should return the snapshot's schema long branchSnapshotId = table.refs().get(branchName).snapshotId(); - assertThat(sql("SELECT * FROM %s VERSION AS OF %s", tableName, branchSnapshotId)) + assertThat(sql("SELECT * FROM %s VERSION AS OF %s ORDER BY id", tableName, branchSnapshotId)) .containsExactlyElementsOf(expected); // querying the head of the branch should return the table's schema - assertThat(sql("SELECT * FROM %s VERSION AS OF '%s'", tableName, branchName)) + assertThat(sql("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, branchName)) .containsExactly(row(1L, "a", null), row(2L, "b", null), row(3L, "c", null)); if (!"spark_catalog".equals(catalogName)) { // querying the head of the branch using 'branch_' should return the table's schema - assertThat(sql("SELECT * FROM %s.branch_%s", tableName, branchName)) + assertThat(sql("SELECT * FROM %s.branch_%s ORDER BY id", tableName, branchName)) .containsExactly(row(1L, "a", null), row(2L, "b", null), row(3L, "c", null)); } @@ -417,7 +440,7 @@ public void readAndWriteWithBranchAfterSchemaChange() { tableName, branchName); // querying the head of the branch returns the table's schema - assertThat(sql("SELECT * FROM %s VERSION AS OF '%s'", tableName, branchName)) + assertThat(sql("SELECT * FROM %s VERSION AS OF '%s' ORDER BY id", tableName, branchName)) .containsExactlyInAnyOrder( row(1L, "a", null), row(2L, "b", null), @@ -454,30 +477,30 @@ public void testTimestampAsOf() { SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss"); String formattedDate = sdf.format(new Date(timestamp)); - List expected = sql("SELECT * FROM %s", tableName); + List expected = sql("SELECT * FROM %s ORDER BY id", tableName); // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); // read the table at the timestamp in long format i.e 1656507980463. List actualWithLongFormat = - sql("SELECT * FROM %s TIMESTAMP AS OF %s", tableName, timestampInSeconds); + sql("SELECT * FROM %s TIMESTAMP AS OF %s ORDER BY id", tableName, timestampInSeconds); assertEquals("Snapshot at timestamp", expected, actualWithLongFormat); // read the table at the timestamp in date format i.e 2022-06-29 18:40:37 List actualWithDateFormat = - sql("SELECT * FROM %s TIMESTAMP AS OF '%s'", tableName, formattedDate); + sql("SELECT * FROM %s TIMESTAMP AS OF '%s' ORDER BY id", tableName, formattedDate); assertEquals("Snapshot at timestamp", expected, actualWithDateFormat); // HIVE time travel syntax // read the table at the timestamp in long format i.e 1656507980463. List actualWithLongFormatInHiveSyntax = - sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF %s", tableName, timestampInSeconds); + sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF %s ORDER BY id", tableName, timestampInSeconds); assertEquals("Snapshot at specific ID", expected, actualWithLongFormatInHiveSyntax); // read the table at the timestamp in date format i.e 2022-06-29 18:40:37 List actualWithDateFormatInHiveSyntax = - sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF '%s'", tableName, formattedDate); + sql("SELECT * FROM %s FOR SYSTEM_TIME AS OF '%s' ORDER BY id", tableName, formattedDate); assertEquals("Snapshot at specific ID", expected, actualWithDateFormatInHiveSyntax); // read the table using DataFrameReader option @@ -657,24 +680,25 @@ public void simpleTypesInFilter() { + "(2, false, 2, 2L, 2.2, 2.4, '2.6', to_date('2022-02-02'), to_timestamp('2022-02-02T00:00:00')), " + "(3, true, 3, 3L, 3.3, 3.6, '3.9', to_date('2023-03-03'), to_timestamp('2023-03-03T00:00:00'))", tableName); - assertThat(sql("SELECT id FROM %s where id > 1", tableName)).containsExactly(row(2L), row(3L)); + assertThat(sql("SELECT id FROM %s where id > 1", tableName)) + .containsExactlyInAnyOrder(row(2L), row(3L)); assertThat(sql("SELECT id, boolean FROM %s where boolean = true", tableName)) - .containsExactly(row(1L, true), row(3L, true)); + .containsExactlyInAnyOrder(row(1L, true), row(3L, true)); assertThat(sql("SELECT long FROM %s where long > 1", tableName)) - .containsExactly(row(2L), row(3L)); + .containsExactlyInAnyOrder(row(2L), row(3L)); assertThat(sql("SELECT float FROM %s where float > 1.1f", tableName)) - .containsExactly(row(2.2f), row(3.3f)); + .containsExactlyInAnyOrder(row(2.2f), row(3.3f)); assertThat(sql("SELECT double FROM %s where double > 1.3", tableName)) - .containsExactly(row(2.4d), row(3.6d)); + .containsExactlyInAnyOrder(row(2.4d), row(3.6d)); assertThat(sql("SELECT string FROM %s where string > '1.5'", tableName)) - .containsExactly(row("2.6"), row("3.9")); + .containsExactlyInAnyOrder(row("2.6"), row("3.9")); java.sql.Date dateOne = java.sql.Date.valueOf("2022-02-02"); java.sql.Date dateTwo = java.sql.Date.valueOf("2023-03-03"); assertThat(sql("SELECT date FROM %s where date > to_date('2021-01-01')", tableName)) - .containsExactly(row(dateOne), row(dateTwo)); + .containsExactlyInAnyOrder(row(dateOne), row(dateTwo)); assertThat( sql("SELECT timestamp FROM %s where timestamp > to_timestamp('2021-01-01')", tableName)) - .containsExactly( + .containsExactlyInAnyOrder( row(new Timestamp(dateOne.getTime())), row(new Timestamp(dateTwo.getTime()))); sql("DROP TABLE IF EXISTS %s", tableName); From bc7bfa5de4743853d9647ad095322ba71e304221 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Mon, 5 Jan 2026 17:49:48 +0100 Subject: [PATCH 147/201] Flink: Backport: Dynamic Sink: Fix serialization issues with schemas larger than 2^16 bytes(#14967) Backports #14880 --- .../DynamicRecordInternalSerializer.java | 103 ++++++++++++--- .../source/split/IcebergSourceSplit.java | 1 + .../split => util}/SerializerHelper.java | 15 ++- ...namicRecordInternalSerializerTestBase.java | 8 +- .../TestDynamicRecordInternalSerializer.java | 119 ++++++++++++++++++ ...icRecordInternalSerializerWriteSchema.java | 4 +- ...RecordInternalSerializerWriteSchemaId.java | 4 +- ...nternalSerializerWriteSchemaIdLongUTF.java | 28 +++++ ...dInternalSerializerWriteSchemaLongUTF.java | 28 +++++ .../DynamicRecordInternalSerializer.java | 103 ++++++++++++--- .../source/split/IcebergSourceSplit.java | 1 + .../split => util}/SerializerHelper.java | 15 ++- ...namicRecordInternalSerializerTestBase.java | 8 +- .../TestDynamicRecordInternalSerializer.java | 119 ++++++++++++++++++ ...icRecordInternalSerializerWriteSchema.java | 4 +- ...RecordInternalSerializerWriteSchemaId.java | 4 +- ...nternalSerializerWriteSchemaIdLongUTF.java | 28 +++++ ...dInternalSerializerWriteSchemaLongUTF.java | 28 +++++ 18 files changed, 568 insertions(+), 52 deletions(-) rename flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/{source/split => util}/SerializerHelper.java (92%) create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializer.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaLongUTF.java rename flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/{source/split => util}/SerializerHelper.java (92%) create mode 100644 flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializer.java create mode 100644 flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF.java create mode 100644 flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaLongUTF.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java index 22b7742913e7..0d758ace1b19 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.Set; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; @@ -34,6 +35,9 @@ import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.flink.util.SerializerHelper; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @Internal @@ -43,18 +47,27 @@ class DynamicRecordInternalSerializer extends TypeSerializer duplicate() { return new DynamicRecordInternalSerializer( new TableSerializerCache(serializerCache.catalogLoader(), serializerCache.maximumSize()), - writeSchemaAndSpec); + writeSchemaAndSpec, + writeLongUTF); } @Override @@ -68,7 +81,12 @@ public void serialize(DynamicRecordInternal toSerialize, DataOutputView dataOutp dataOutputView.writeUTF(toSerialize.tableName()); dataOutputView.writeUTF(toSerialize.branch()); if (writeSchemaAndSpec) { - dataOutputView.writeUTF(SchemaParser.toJson(toSerialize.schema())); + if (writeLongUTF) { + SerializerHelper.writeLongUTF(dataOutputView, SchemaParser.toJson(toSerialize.schema())); + } else { + dataOutputView.writeUTF(SchemaParser.toJson(toSerialize.schema())); + } + dataOutputView.writeUTF(PartitionSpecParser.toJson(toSerialize.spec())); } else { dataOutputView.writeInt(toSerialize.schema().schemaId()); @@ -108,7 +126,12 @@ public DynamicRecordInternal deserialize(DataInputView dataInputView) throws IOE final PartitionSpec spec; final RowDataSerializer rowDataSerializer; if (writeSchemaAndSpec) { - schema = SchemaParser.fromJson(dataInputView.readUTF()); + if (writeLongUTF) { + schema = SchemaParser.fromJson(SerializerHelper.readLongUTF(dataInputView)); + } else { + schema = SchemaParser.fromJson(dataInputView.readUTF()); + } + spec = PartitionSpecParser.fromJson(schema, dataInputView.readUTF()); rowDataSerializer = serializerCache.serializer(tableName, schema, spec); } else { @@ -152,7 +175,12 @@ public DynamicRecordInternal deserialize(DynamicRecordInternal reuse, DataInputV final PartitionSpec spec; final RowDataSerializer rowDataSerializer; if (writeSchemaAndSpec) { - schema = SchemaParser.fromJson(dataInputView.readUTF()); + if (writeLongUTF) { + schema = SchemaParser.fromJson(SerializerHelper.readLongUTF(dataInputView)); + } else { + schema = SchemaParser.fromJson(dataInputView.readUTF()); + } + spec = PartitionSpecParser.fromJson(schema, dataInputView.readUTF()); reuse.setSchema(schema); reuse.setSpec(spec); @@ -245,25 +273,32 @@ public int getLength() { @Override public TypeSerializerSnapshot snapshotConfiguration() { - return new DynamicRecordInternalTypeSerializerSnapshot(writeSchemaAndSpec); + return new DynamicRecordInternalTypeSerializerSnapshot(writeSchemaAndSpec, serializerCache); } public static class DynamicRecordInternalTypeSerializerSnapshot implements TypeSerializerSnapshot { + private static final int MOST_RECENT_VERSION = 1; + private boolean writeSchemaAndSpec; + private int version; + private TableSerializerCache serializerCache; - // Zero args constructor is required to instantiate this class on restore + // Zero args constructor is required to instantiate this class on restore via readSnapshot(..) @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) public DynamicRecordInternalTypeSerializerSnapshot() {} - DynamicRecordInternalTypeSerializerSnapshot(boolean writeSchemaAndSpec) { + DynamicRecordInternalTypeSerializerSnapshot( + boolean writeSchemaAndSpec, TableSerializerCache serializerCache) { this.writeSchemaAndSpec = writeSchemaAndSpec; + this.serializerCache = serializerCache; + this.version = MOST_RECENT_VERSION; } @Override public int getCurrentVersion() { - return 0; + return version; } @Override @@ -274,22 +309,62 @@ public void writeSnapshot(DataOutputView out) throws IOException { @Override public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { + this.version = readVersion; this.writeSchemaAndSpec = in.readBoolean(); } @Override public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializerSnapshot oldSerializerSnapshot) { - return TypeSerializerSchemaCompatibility.compatibleAsIs(); + if (oldSerializerSnapshot.getCurrentVersion() == getCurrentVersion()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + // Old TypeSerializerSnapshots do not contain the serializer cache, but the newest one does. + // This will also ensure that we always use the up-to-date cache alongside with its catalog + // configuration. + Preconditions.checkNotNull(serializerCache, "serializerCache should not be null"); + try { + DynMethods.builder("initializeSerializerCache") + .hiddenImpl( + DynamicRecordInternalTypeSerializerSnapshot.class, TableSerializerCache.class) + .build() + .invoke(oldSerializerSnapshot, serializerCache); + } catch (Exception e) { + throw new RuntimeException( + "Failed to initialize serializerCache for reading data with old serializer", e); + } + + // This will first read data with the old serializer, then switch to the most recent one. + return TypeSerializerSchemaCompatibility.compatibleAfterMigration(); } @Override public TypeSerializer restoreSerializer() { - // Note: We pass in a null serializer cache which would create issues if we tried to use this - // restored serializer, but since we are using {@code - // TypeSerializerSchemaCompatibility.compatibleAsIs()} above, this serializer will never be - // used. A new one will be created via {@code DynamicRecordInternalType}. - return new DynamicRecordInternalSerializer(null, writeSchemaAndSpec); + if (getCurrentVersion() < MOST_RECENT_VERSION) { + // If this serializer is not the most recent one, we need to read old data with the correct + // parameters. + return new DynamicRecordInternalSerializer(serializerCache, writeSchemaAndSpec, false); + } + + // In all other cases, we just use the newest serializer. + return new DynamicRecordInternalSerializer(serializerCache, writeSchemaAndSpec, true); + } + + /** + * We need to lazily initialize the cache from the up-to-date serializer which has the current + * CatalogLoader available. + * + *

    This method must not be removed! + */ + @SuppressWarnings("unused") + private void initializeSerializerCache(TableSerializerCache cache) { + this.serializerCache = cache; } } + + @VisibleForTesting + TableSerializerCache getSerializerCache() { + return serializerCache; + } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java index b6d6f60ef673..0cc1c633f906 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -33,6 +33,7 @@ import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.ScanTaskParser; +import org.apache.iceberg.flink.util.SerializerHelper; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/SerializerHelper.java similarity index 92% rename from flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/SerializerHelper.java index 841969666ee5..3a161ea2631f 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/SerializerHelper.java @@ -16,20 +16,24 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.source.split; +package org.apache.iceberg.flink.util; import java.io.IOException; import java.io.Serializable; import java.io.UTFDataFormatException; +import org.apache.flink.annotation.Internal; import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.core.memory.DataOutputView; /** * Helper class to serialize and deserialize strings longer than 65K. The inspiration is mostly * taken from the class org.apache.flink.core.memory.DataInputSerializer.readUTF and * org.apache.flink.core.memory.DataOutputSerializer.writeUTF. */ -class SerializerHelper implements Serializable { +@Internal +public class SerializerHelper implements Serializable { private SerializerHelper() {} @@ -47,7 +51,7 @@ private SerializerHelper() {} * @param out the output stream to write the string to. * @param str the string value to be written. */ - public static void writeLongUTF(DataOutputSerializer out, String str) throws IOException { + public static void writeLongUTF(DataOutputView out, String str) throws IOException { int strlen = str.length(); long utflen = 0; int ch; @@ -85,7 +89,7 @@ public static void writeLongUTF(DataOutputSerializer out, String str) throws IOE * @return the string value read from the input stream. * @throws IOException if an I/O error occurs when reading from the input stream. */ - public static String readLongUTF(DataInputDeserializer in) throws IOException { + public static String readLongUTF(DataInputView in) throws IOException { int utflen = in.readInt(); byte[] bytearr = new byte[utflen]; char[] chararr = new char[utflen]; @@ -168,8 +172,7 @@ private static int getUTFBytesSize(int ch) { } } - private static void writeUTFBytes(DataOutputSerializer out, String str, int utflen) - throws IOException { + private static void writeUTFBytes(DataOutputView out, String str, int utflen) throws IOException { int strlen = str.length(); int ch; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java index 30782e8d4170..07096b891a95 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java @@ -55,15 +55,19 @@ abstract class DynamicRecordInternalSerializerTestBase static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).bucket("id", 10).build(); private boolean writeFullSchemaAndSpec; + private final boolean writeLongUTF; - DynamicRecordInternalSerializerTestBase(boolean writeFullSchemaAndSpec) { + DynamicRecordInternalSerializerTestBase(boolean writeFullSchemaAndSpec, boolean writeLongUTF) { this.writeFullSchemaAndSpec = writeFullSchemaAndSpec; + this.writeLongUTF = writeLongUTF; } @Override protected TypeSerializer createSerializer() { return new DynamicRecordInternalSerializer( - new TableSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), writeFullSchemaAndSpec); + new TableSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), + writeFullSchemaAndSpec, + writeLongUTF); } @BeforeEach diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializer.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializer.java new file mode 100644 index 000000000000..388ff303339c --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializer.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestDynamicRecordInternalSerializer { + + @RegisterExtension + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension("db", "table"); + + @Test + void testCurrentTypeSerializerSnapshotVersion() { + TypeSerializer serializer = createSerializer(); + assertThat(serializer).isNotNull().isInstanceOf(DynamicRecordInternalSerializer.class); + TypeSerializerSnapshot snapshot = serializer.snapshotConfiguration(); + assertThat(snapshot.getCurrentVersion()).isEqualTo(1); + } + + @Test + void testCurrentTypeSerializerSnapshotCompatibility() { + TypeSerializer serializer = createSerializer(); + assertThat(serializer).isNotNull().isInstanceOf(DynamicRecordInternalSerializer.class); + TypeSerializerSnapshot snapshot = serializer.snapshotConfiguration(); + assertThat( + snapshot + .resolveSchemaCompatibility(serializer.snapshotConfiguration()) + .isCompatibleAsIs()) + .isTrue(); + } + + @Test + void testRestoreFromOldVersion() throws IOException { + // Create a serialized snapshot of the TypeSerializer + final int oldVersion = 0; + OldTypeSerializerSnapshot oldTypeSerializerSnapshot = new OldTypeSerializerSnapshot(oldVersion); + assertThat(oldTypeSerializerSnapshot.getCurrentVersion()).isEqualTo(oldVersion); + DataOutputSerializer dataOutputSerializer = new DataOutputSerializer(128); + oldTypeSerializerSnapshot.writeSnapshot(dataOutputSerializer); + + // Load the serialized state + DynamicRecordInternalSerializer.DynamicRecordInternalTypeSerializerSnapshot restoreSnapshot = + (DynamicRecordInternalSerializer.DynamicRecordInternalTypeSerializerSnapshot) + createSerializer().snapshotConfiguration(); + restoreSnapshot.readSnapshot( + oldVersion, + new DataInputDeserializer(dataOutputSerializer.getSharedBuffer()), + getClass().getClassLoader()); + // Check that it matches the original one + assertThat(restoreSnapshot.getCurrentVersion()).isEqualTo(oldVersion); + assertThat( + restoreSnapshot + .resolveSchemaCompatibility(oldTypeSerializerSnapshot) + .isCompatibleAsIs()) + .isTrue(); + TypeSerializer restoreSerializer = restoreSnapshot.restoreSerializer(); + assertThat(restoreSerializer).isInstanceOf(DynamicRecordInternalSerializer.class); + assertThat(((DynamicRecordInternalSerializer) restoreSerializer).getSerializerCache()) + .isNotNull(); + + // Compare against the latest version of a snapshot + TypeSerializerSnapshot latestVersion = + createSerializer().snapshotConfiguration(); + assertThat(latestVersion.getCurrentVersion()).isEqualTo(1); + assertThat( + latestVersion + .resolveSchemaCompatibility(oldTypeSerializerSnapshot) + .isCompatibleAfterMigration()) + .isTrue(); + assertThat( + latestVersion.resolveSchemaCompatibility(restoreSnapshot).isCompatibleAfterMigration()) + .isTrue(); + } + + private DynamicRecordInternalSerializer createSerializer() { + return new DynamicRecordInternalSerializer( + new TableSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), true); + } + + private static class OldTypeSerializerSnapshot + extends DynamicRecordInternalSerializer.DynamicRecordInternalTypeSerializerSnapshot { + + private final int version; + + OldTypeSerializerSnapshot(int version) { + this.version = version; + } + + @Override + public int getCurrentVersion() { + return version; + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java index ab8ce98c3594..e7aa2d24d9d0 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java @@ -18,11 +18,11 @@ */ package org.apache.iceberg.flink.sink.dynamic; -/** Test writing DynamicRecord with the full schema */ +/** Test writing DynamicRecord with the full schema and standard UTF encoding */ class TestDynamicRecordInternalSerializerWriteSchema extends DynamicRecordInternalSerializerTestBase { TestDynamicRecordInternalSerializerWriteSchema() { - super(true); + super(true /* writeFullSchemaAndSpec */, false /* writeLongUTF */); } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java index 1d8890546214..bff0fd5c6aad 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java @@ -18,11 +18,11 @@ */ package org.apache.iceberg.flink.sink.dynamic; -/** Test writing DynamicRecord with only the schema id. */ +/** Test writing DynamicRecord with only the schema id and standard UTF encoding */ class TestDynamicRecordInternalSerializerWriteSchemaId extends DynamicRecordInternalSerializerTestBase { TestDynamicRecordInternalSerializerWriteSchemaId() { - super(false); + super(false /* writeFullSchemaAndSpec */, false /* writeLongUTF */); } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF.java new file mode 100644 index 000000000000..7a1ae3df3806 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +/** Test writing DynamicRecord with only the schema id and long UTF encoding */ +class TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF + extends DynamicRecordInternalSerializerTestBase { + + TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF() { + super(false /* writeFullSchemaAndSpec */, true /* writeLongUTF */); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaLongUTF.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaLongUTF.java new file mode 100644 index 000000000000..faff8921db5d --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaLongUTF.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +/** Test writing DynamicRecord with the full schema and long UTF encoding */ +class TestDynamicRecordInternalSerializerWriteSchemaLongUTF + extends DynamicRecordInternalSerializerTestBase { + + TestDynamicRecordInternalSerializerWriteSchemaLongUTF() { + super(true /* writeFullSchemaAndSpec */, true /* writeLongUTF */); + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java index 22b7742913e7..0d758ace1b19 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializer.java @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.Set; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; @@ -34,6 +35,9 @@ import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.flink.util.SerializerHelper; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @Internal @@ -43,18 +47,27 @@ class DynamicRecordInternalSerializer extends TypeSerializer duplicate() { return new DynamicRecordInternalSerializer( new TableSerializerCache(serializerCache.catalogLoader(), serializerCache.maximumSize()), - writeSchemaAndSpec); + writeSchemaAndSpec, + writeLongUTF); } @Override @@ -68,7 +81,12 @@ public void serialize(DynamicRecordInternal toSerialize, DataOutputView dataOutp dataOutputView.writeUTF(toSerialize.tableName()); dataOutputView.writeUTF(toSerialize.branch()); if (writeSchemaAndSpec) { - dataOutputView.writeUTF(SchemaParser.toJson(toSerialize.schema())); + if (writeLongUTF) { + SerializerHelper.writeLongUTF(dataOutputView, SchemaParser.toJson(toSerialize.schema())); + } else { + dataOutputView.writeUTF(SchemaParser.toJson(toSerialize.schema())); + } + dataOutputView.writeUTF(PartitionSpecParser.toJson(toSerialize.spec())); } else { dataOutputView.writeInt(toSerialize.schema().schemaId()); @@ -108,7 +126,12 @@ public DynamicRecordInternal deserialize(DataInputView dataInputView) throws IOE final PartitionSpec spec; final RowDataSerializer rowDataSerializer; if (writeSchemaAndSpec) { - schema = SchemaParser.fromJson(dataInputView.readUTF()); + if (writeLongUTF) { + schema = SchemaParser.fromJson(SerializerHelper.readLongUTF(dataInputView)); + } else { + schema = SchemaParser.fromJson(dataInputView.readUTF()); + } + spec = PartitionSpecParser.fromJson(schema, dataInputView.readUTF()); rowDataSerializer = serializerCache.serializer(tableName, schema, spec); } else { @@ -152,7 +175,12 @@ public DynamicRecordInternal deserialize(DynamicRecordInternal reuse, DataInputV final PartitionSpec spec; final RowDataSerializer rowDataSerializer; if (writeSchemaAndSpec) { - schema = SchemaParser.fromJson(dataInputView.readUTF()); + if (writeLongUTF) { + schema = SchemaParser.fromJson(SerializerHelper.readLongUTF(dataInputView)); + } else { + schema = SchemaParser.fromJson(dataInputView.readUTF()); + } + spec = PartitionSpecParser.fromJson(schema, dataInputView.readUTF()); reuse.setSchema(schema); reuse.setSpec(spec); @@ -245,25 +273,32 @@ public int getLength() { @Override public TypeSerializerSnapshot snapshotConfiguration() { - return new DynamicRecordInternalTypeSerializerSnapshot(writeSchemaAndSpec); + return new DynamicRecordInternalTypeSerializerSnapshot(writeSchemaAndSpec, serializerCache); } public static class DynamicRecordInternalTypeSerializerSnapshot implements TypeSerializerSnapshot { + private static final int MOST_RECENT_VERSION = 1; + private boolean writeSchemaAndSpec; + private int version; + private TableSerializerCache serializerCache; - // Zero args constructor is required to instantiate this class on restore + // Zero args constructor is required to instantiate this class on restore via readSnapshot(..) @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) public DynamicRecordInternalTypeSerializerSnapshot() {} - DynamicRecordInternalTypeSerializerSnapshot(boolean writeSchemaAndSpec) { + DynamicRecordInternalTypeSerializerSnapshot( + boolean writeSchemaAndSpec, TableSerializerCache serializerCache) { this.writeSchemaAndSpec = writeSchemaAndSpec; + this.serializerCache = serializerCache; + this.version = MOST_RECENT_VERSION; } @Override public int getCurrentVersion() { - return 0; + return version; } @Override @@ -274,22 +309,62 @@ public void writeSnapshot(DataOutputView out) throws IOException { @Override public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException { + this.version = readVersion; this.writeSchemaAndSpec = in.readBoolean(); } @Override public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( TypeSerializerSnapshot oldSerializerSnapshot) { - return TypeSerializerSchemaCompatibility.compatibleAsIs(); + if (oldSerializerSnapshot.getCurrentVersion() == getCurrentVersion()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + // Old TypeSerializerSnapshots do not contain the serializer cache, but the newest one does. + // This will also ensure that we always use the up-to-date cache alongside with its catalog + // configuration. + Preconditions.checkNotNull(serializerCache, "serializerCache should not be null"); + try { + DynMethods.builder("initializeSerializerCache") + .hiddenImpl( + DynamicRecordInternalTypeSerializerSnapshot.class, TableSerializerCache.class) + .build() + .invoke(oldSerializerSnapshot, serializerCache); + } catch (Exception e) { + throw new RuntimeException( + "Failed to initialize serializerCache for reading data with old serializer", e); + } + + // This will first read data with the old serializer, then switch to the most recent one. + return TypeSerializerSchemaCompatibility.compatibleAfterMigration(); } @Override public TypeSerializer restoreSerializer() { - // Note: We pass in a null serializer cache which would create issues if we tried to use this - // restored serializer, but since we are using {@code - // TypeSerializerSchemaCompatibility.compatibleAsIs()} above, this serializer will never be - // used. A new one will be created via {@code DynamicRecordInternalType}. - return new DynamicRecordInternalSerializer(null, writeSchemaAndSpec); + if (getCurrentVersion() < MOST_RECENT_VERSION) { + // If this serializer is not the most recent one, we need to read old data with the correct + // parameters. + return new DynamicRecordInternalSerializer(serializerCache, writeSchemaAndSpec, false); + } + + // In all other cases, we just use the newest serializer. + return new DynamicRecordInternalSerializer(serializerCache, writeSchemaAndSpec, true); + } + + /** + * We need to lazily initialize the cache from the up-to-date serializer which has the current + * CatalogLoader available. + * + *

    This method must not be removed! + */ + @SuppressWarnings("unused") + private void initializeSerializerCache(TableSerializerCache cache) { + this.serializerCache = cache; } } + + @VisibleForTesting + TableSerializerCache getSerializerCache() { + return serializerCache; + } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java index b6d6f60ef673..0cc1c633f906 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -33,6 +33,7 @@ import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.ScanTaskParser; +import org.apache.iceberg.flink.util.SerializerHelper; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/util/SerializerHelper.java similarity index 92% rename from flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java rename to flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/util/SerializerHelper.java index 841969666ee5..3a161ea2631f 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/util/SerializerHelper.java @@ -16,20 +16,24 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.flink.source.split; +package org.apache.iceberg.flink.util; import java.io.IOException; import java.io.Serializable; import java.io.UTFDataFormatException; +import org.apache.flink.annotation.Internal; import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.core.memory.DataOutputView; /** * Helper class to serialize and deserialize strings longer than 65K. The inspiration is mostly * taken from the class org.apache.flink.core.memory.DataInputSerializer.readUTF and * org.apache.flink.core.memory.DataOutputSerializer.writeUTF. */ -class SerializerHelper implements Serializable { +@Internal +public class SerializerHelper implements Serializable { private SerializerHelper() {} @@ -47,7 +51,7 @@ private SerializerHelper() {} * @param out the output stream to write the string to. * @param str the string value to be written. */ - public static void writeLongUTF(DataOutputSerializer out, String str) throws IOException { + public static void writeLongUTF(DataOutputView out, String str) throws IOException { int strlen = str.length(); long utflen = 0; int ch; @@ -85,7 +89,7 @@ public static void writeLongUTF(DataOutputSerializer out, String str) throws IOE * @return the string value read from the input stream. * @throws IOException if an I/O error occurs when reading from the input stream. */ - public static String readLongUTF(DataInputDeserializer in) throws IOException { + public static String readLongUTF(DataInputView in) throws IOException { int utflen = in.readInt(); byte[] bytearr = new byte[utflen]; char[] chararr = new char[utflen]; @@ -168,8 +172,7 @@ private static int getUTFBytesSize(int ch) { } } - private static void writeUTFBytes(DataOutputSerializer out, String str, int utflen) - throws IOException { + private static void writeUTFBytes(DataOutputView out, String str, int utflen) throws IOException { int strlen = str.length(); int ch; diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java index 30782e8d4170..07096b891a95 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordInternalSerializerTestBase.java @@ -55,15 +55,19 @@ abstract class DynamicRecordInternalSerializerTestBase static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).bucket("id", 10).build(); private boolean writeFullSchemaAndSpec; + private final boolean writeLongUTF; - DynamicRecordInternalSerializerTestBase(boolean writeFullSchemaAndSpec) { + DynamicRecordInternalSerializerTestBase(boolean writeFullSchemaAndSpec, boolean writeLongUTF) { this.writeFullSchemaAndSpec = writeFullSchemaAndSpec; + this.writeLongUTF = writeLongUTF; } @Override protected TypeSerializer createSerializer() { return new DynamicRecordInternalSerializer( - new TableSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), writeFullSchemaAndSpec); + new TableSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), + writeFullSchemaAndSpec, + writeLongUTF); } @BeforeEach diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializer.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializer.java new file mode 100644 index 000000000000..388ff303339c --- /dev/null +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializer.java @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestDynamicRecordInternalSerializer { + + @RegisterExtension + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension("db", "table"); + + @Test + void testCurrentTypeSerializerSnapshotVersion() { + TypeSerializer serializer = createSerializer(); + assertThat(serializer).isNotNull().isInstanceOf(DynamicRecordInternalSerializer.class); + TypeSerializerSnapshot snapshot = serializer.snapshotConfiguration(); + assertThat(snapshot.getCurrentVersion()).isEqualTo(1); + } + + @Test + void testCurrentTypeSerializerSnapshotCompatibility() { + TypeSerializer serializer = createSerializer(); + assertThat(serializer).isNotNull().isInstanceOf(DynamicRecordInternalSerializer.class); + TypeSerializerSnapshot snapshot = serializer.snapshotConfiguration(); + assertThat( + snapshot + .resolveSchemaCompatibility(serializer.snapshotConfiguration()) + .isCompatibleAsIs()) + .isTrue(); + } + + @Test + void testRestoreFromOldVersion() throws IOException { + // Create a serialized snapshot of the TypeSerializer + final int oldVersion = 0; + OldTypeSerializerSnapshot oldTypeSerializerSnapshot = new OldTypeSerializerSnapshot(oldVersion); + assertThat(oldTypeSerializerSnapshot.getCurrentVersion()).isEqualTo(oldVersion); + DataOutputSerializer dataOutputSerializer = new DataOutputSerializer(128); + oldTypeSerializerSnapshot.writeSnapshot(dataOutputSerializer); + + // Load the serialized state + DynamicRecordInternalSerializer.DynamicRecordInternalTypeSerializerSnapshot restoreSnapshot = + (DynamicRecordInternalSerializer.DynamicRecordInternalTypeSerializerSnapshot) + createSerializer().snapshotConfiguration(); + restoreSnapshot.readSnapshot( + oldVersion, + new DataInputDeserializer(dataOutputSerializer.getSharedBuffer()), + getClass().getClassLoader()); + // Check that it matches the original one + assertThat(restoreSnapshot.getCurrentVersion()).isEqualTo(oldVersion); + assertThat( + restoreSnapshot + .resolveSchemaCompatibility(oldTypeSerializerSnapshot) + .isCompatibleAsIs()) + .isTrue(); + TypeSerializer restoreSerializer = restoreSnapshot.restoreSerializer(); + assertThat(restoreSerializer).isInstanceOf(DynamicRecordInternalSerializer.class); + assertThat(((DynamicRecordInternalSerializer) restoreSerializer).getSerializerCache()) + .isNotNull(); + + // Compare against the latest version of a snapshot + TypeSerializerSnapshot latestVersion = + createSerializer().snapshotConfiguration(); + assertThat(latestVersion.getCurrentVersion()).isEqualTo(1); + assertThat( + latestVersion + .resolveSchemaCompatibility(oldTypeSerializerSnapshot) + .isCompatibleAfterMigration()) + .isTrue(); + assertThat( + latestVersion.resolveSchemaCompatibility(restoreSnapshot).isCompatibleAfterMigration()) + .isTrue(); + } + + private DynamicRecordInternalSerializer createSerializer() { + return new DynamicRecordInternalSerializer( + new TableSerializerCache(CATALOG_EXTENSION.catalogLoader(), 1), true); + } + + private static class OldTypeSerializerSnapshot + extends DynamicRecordInternalSerializer.DynamicRecordInternalTypeSerializerSnapshot { + + private final int version; + + OldTypeSerializerSnapshot(int version) { + this.version = version; + } + + @Override + public int getCurrentVersion() { + return version; + } + } +} diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java index ab8ce98c3594..e7aa2d24d9d0 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchema.java @@ -18,11 +18,11 @@ */ package org.apache.iceberg.flink.sink.dynamic; -/** Test writing DynamicRecord with the full schema */ +/** Test writing DynamicRecord with the full schema and standard UTF encoding */ class TestDynamicRecordInternalSerializerWriteSchema extends DynamicRecordInternalSerializerTestBase { TestDynamicRecordInternalSerializerWriteSchema() { - super(true); + super(true /* writeFullSchemaAndSpec */, false /* writeLongUTF */); } } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java index 1d8890546214..bff0fd5c6aad 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaId.java @@ -18,11 +18,11 @@ */ package org.apache.iceberg.flink.sink.dynamic; -/** Test writing DynamicRecord with only the schema id. */ +/** Test writing DynamicRecord with only the schema id and standard UTF encoding */ class TestDynamicRecordInternalSerializerWriteSchemaId extends DynamicRecordInternalSerializerTestBase { TestDynamicRecordInternalSerializerWriteSchemaId() { - super(false); + super(false /* writeFullSchemaAndSpec */, false /* writeLongUTF */); } } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF.java new file mode 100644 index 000000000000..7a1ae3df3806 --- /dev/null +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +/** Test writing DynamicRecord with only the schema id and long UTF encoding */ +class TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF + extends DynamicRecordInternalSerializerTestBase { + + TestDynamicRecordInternalSerializerWriteSchemaIdLongUTF() { + super(false /* writeFullSchemaAndSpec */, true /* writeLongUTF */); + } +} diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaLongUTF.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaLongUTF.java new file mode 100644 index 000000000000..faff8921db5d --- /dev/null +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicRecordInternalSerializerWriteSchemaLongUTF.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +/** Test writing DynamicRecord with the full schema and long UTF encoding */ +class TestDynamicRecordInternalSerializerWriteSchemaLongUTF + extends DynamicRecordInternalSerializerTestBase { + + TestDynamicRecordInternalSerializerWriteSchemaLongUTF() { + super(true /* writeFullSchemaAndSpec */, true /* writeLongUTF */); + } +} From 8f2ed2068967e9cadb7ae4e263cfef490c844b8e Mon Sep 17 00:00:00 2001 From: aiborodin Date: Tue, 6 Jan 2026 20:06:56 +1100 Subject: [PATCH 148/201] Flink: Backport: DynamicSink: Report writer records/bytes send metrics (#14971) Backports #14878 --- .../flink/sink/dynamic/DynamicWriter.java | 1 + .../sink/dynamic/DynamicWriterMetrics.java | 31 +++++++++++++++++-- .../flink/sink/dynamic/TestDynamicWriter.java | 2 +- .../flink/sink/dynamic/DynamicWriter.java | 1 + .../sink/dynamic/DynamicWriterMetrics.java | 31 +++++++++++++++++-- .../flink/sink/dynamic/TestDynamicWriter.java | 2 +- 6 files changed, 60 insertions(+), 8 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java index c2a303285801..907385797495 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java @@ -145,6 +145,7 @@ public void write(DynamicRecordInternal element, Context context) return taskWriterFactory.create(); }) .write(element.rowData()); + metrics.mainMetricsGroup().getNumRecordsSendCounter().inc(); } @Override diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java index 2e1f82df9d2d..d50a41512a15 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java @@ -18,24 +18,37 @@ */ package org.apache.iceberg.flink.sink.dynamic; +import java.util.Arrays; import java.util.Map; -import org.apache.flink.metrics.MetricGroup; +import java.util.function.ToLongFunction; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.flink.sink.IcebergStreamWriterMetrics; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.ScanTaskUtil; class DynamicWriterMetrics { private final Map metrics; - private final MetricGroup mainMetricsGroup; + private final SinkWriterMetricGroup mainMetricsGroup; - DynamicWriterMetrics(MetricGroup mainMetricsGroup) { + DynamicWriterMetrics(SinkWriterMetricGroup mainMetricsGroup) { this.mainMetricsGroup = mainMetricsGroup; this.metrics = Maps.newHashMap(); } + SinkWriterMetricGroup mainMetricsGroup() { + return this.mainMetricsGroup; + } + public void updateFlushResult(String fullTableName, WriteResult result) { writerMetrics(fullTableName).updateFlushResult(result); + + long bytesOutTotal = sum(result.dataFiles()) + sum(result.deleteFiles()); + this.mainMetricsGroup.getNumBytesSendCounter().inc(bytesOutTotal); } public void flushDuration(String fullTableName, long flushDurationMs) { @@ -46,4 +59,16 @@ IcebergStreamWriterMetrics writerMetrics(String fullTableName) { return metrics.computeIfAbsent( fullTableName, tableName -> new IcebergStreamWriterMetrics(mainMetricsGroup, tableName)); } + + private static long sum(DataFile[] files) { + return sum(files, DataFile::fileSizeInBytes); + } + + private static long sum(DeleteFile[] files) { + return sum(files, ScanTaskUtil::contentSizeInBytes); + } + + private static > long sum(T[] files, ToLongFunction sizeExtractor) { + return Arrays.stream(files).mapToLong(sizeExtractor).sum(); + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java index 689fd20483c1..d17848225f69 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java @@ -243,7 +243,7 @@ void testUniqueFileSuffixOnFactoryRecreation() throws Exception { 1024L, properties, 100, - new DynamicWriterMetrics(new UnregisteredMetricsGroup()), + new DynamicWriterMetrics(UnregisteredMetricsGroup.createSinkWriterMetricGroup()), 0, 0); return dynamicWriter; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java index c2a303285801..907385797495 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java @@ -145,6 +145,7 @@ public void write(DynamicRecordInternal element, Context context) return taskWriterFactory.create(); }) .write(element.rowData()); + metrics.mainMetricsGroup().getNumRecordsSendCounter().inc(); } @Override diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java index 2e1f82df9d2d..d50a41512a15 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriterMetrics.java @@ -18,24 +18,37 @@ */ package org.apache.iceberg.flink.sink.dynamic; +import java.util.Arrays; import java.util.Map; -import org.apache.flink.metrics.MetricGroup; +import java.util.function.ToLongFunction; +import org.apache.flink.metrics.groups.SinkWriterMetricGroup; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.flink.sink.IcebergStreamWriterMetrics; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.ScanTaskUtil; class DynamicWriterMetrics { private final Map metrics; - private final MetricGroup mainMetricsGroup; + private final SinkWriterMetricGroup mainMetricsGroup; - DynamicWriterMetrics(MetricGroup mainMetricsGroup) { + DynamicWriterMetrics(SinkWriterMetricGroup mainMetricsGroup) { this.mainMetricsGroup = mainMetricsGroup; this.metrics = Maps.newHashMap(); } + SinkWriterMetricGroup mainMetricsGroup() { + return this.mainMetricsGroup; + } + public void updateFlushResult(String fullTableName, WriteResult result) { writerMetrics(fullTableName).updateFlushResult(result); + + long bytesOutTotal = sum(result.dataFiles()) + sum(result.deleteFiles()); + this.mainMetricsGroup.getNumBytesSendCounter().inc(bytesOutTotal); } public void flushDuration(String fullTableName, long flushDurationMs) { @@ -46,4 +59,16 @@ IcebergStreamWriterMetrics writerMetrics(String fullTableName) { return metrics.computeIfAbsent( fullTableName, tableName -> new IcebergStreamWriterMetrics(mainMetricsGroup, tableName)); } + + private static long sum(DataFile[] files) { + return sum(files, DataFile::fileSizeInBytes); + } + + private static long sum(DeleteFile[] files) { + return sum(files, ScanTaskUtil::contentSizeInBytes); + } + + private static > long sum(T[] files, ToLongFunction sizeExtractor) { + return Arrays.stream(files).mapToLong(sizeExtractor).sum(); + } } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java index 689fd20483c1..d17848225f69 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriter.java @@ -243,7 +243,7 @@ void testUniqueFileSuffixOnFactoryRecreation() throws Exception { 1024L, properties, 100, - new DynamicWriterMetrics(new UnregisteredMetricsGroup()), + new DynamicWriterMetrics(UnregisteredMetricsGroup.createSinkWriterMetricGroup()), 0, 0); return dynamicWriter; From 42cac92c4847407b8b56f976b83a0602d80dce0b Mon Sep 17 00:00:00 2001 From: GuoYu <511955993@qq.com> Date: Tue, 6 Jan 2026 20:26:31 +0800 Subject: [PATCH 149/201] Flink: Fix equalityFieldColumns always null in IcebergSink (#14952) --- .../iceberg/flink/sink/IcebergSink.java | 15 +++++++-- ...estFlinkIcebergSinkV2DistributionMode.java | 33 +++++++++++++++++++ 2 files changed, 45 insertions(+), 3 deletions(-) diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java index 593230262fde..470bbc41ded2 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -89,6 +89,7 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.SerializableSupplier; import org.slf4j.Logger; @@ -162,7 +163,9 @@ public class IcebergSink private final transient FlinkMaintenanceConfig flinkMaintenanceConfig; private final Table table; - private final Set equalityFieldColumns = null; + // This should only be used for logging/error messages. For any actual logic always use + // equalityFieldIds instead. + private final Set equalityFieldColumns; private IcebergSink( TableLoader tableLoader, @@ -176,7 +179,8 @@ private IcebergSink( Set equalityFieldIds, String branch, boolean overwriteMode, - FlinkMaintenanceConfig flinkMaintenanceConfig) { + FlinkMaintenanceConfig flinkMaintenanceConfig, + Set equalityFieldColumns) { this.tableLoader = tableLoader; this.snapshotProperties = snapshotProperties; this.uidSuffix = uidSuffix; @@ -198,6 +202,7 @@ private IcebergSink( this.sinkId = UUID.randomUUID().toString(); this.compactMode = flinkWriteConf.compactMode(); this.flinkMaintenanceConfig = flinkMaintenanceConfig; + this.equalityFieldColumns = equalityFieldColumns; } @Override @@ -666,6 +671,9 @@ IcebergSink build() { FlinkMaintenanceConfig flinkMaintenanceConfig = new FlinkMaintenanceConfig(table, writeOptions, readableConfig); + Set equalityFieldColumnsSet = + equalityFieldColumns != null ? Sets.newHashSet(equalityFieldColumns) : null; + return new IcebergSink( tableLoader, table, @@ -680,7 +688,8 @@ IcebergSink build() { equalityFieldIds, flinkWriteConf.branch(), overwriteMode, - flinkMaintenanceConfig); + flinkMaintenanceConfig, + equalityFieldColumnsSet); } /** diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java index 0feb4cc282d2..89f2c7b0daad 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java @@ -562,6 +562,39 @@ public void testRangeDistributionStatisticsMigration() throws Exception { } } + @TestTemplate + public void testHashDistributionWithPartitionNotInEqualityFields() { + assumeThat(partitioned).isTrue(); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + if (isTableSchema) { + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_TABLE_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(writeParallelism) + .distributionMode(DistributionMode.HASH) + .upsert(false) + .equalityFieldColumns(ImmutableList.of("id")) + .append(); + } else { + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(writeParallelism) + .distributionMode(DistributionMode.HASH) + .upsert(false) + .equalityFieldColumns(ImmutableList.of("id")) + .append(); + } + + assertThatThrownBy(env::execute) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining( + "In 'hash' distribution mode with equality fields set, source column 'data' of partition field '1000: data: identity(2)' should be included in equality fields: '[id]'"); + } + private BoundedTestSource createRangeDistributionBoundedSource( List> rowsPerCheckpoint) { return new BoundedTestSource<>(rowsPerCheckpoint); From 4fe8ae2d3688fab62270b24dd8c45fb36cef6cd6 Mon Sep 17 00:00:00 2001 From: GuoYu <511955993@qq.com> Date: Wed, 7 Jan 2026 01:08:55 +0800 Subject: [PATCH 150/201] Flink: Backport fix equalityFieldColumns always null in IcebergSink (#14975) --- .../iceberg/flink/sink/IcebergSink.java | 15 +++++++-- ...estFlinkIcebergSinkV2DistributionMode.java | 33 +++++++++++++++++++ .../iceberg/flink/sink/IcebergSink.java | 15 +++++++-- ...estFlinkIcebergSinkV2DistributionMode.java | 33 +++++++++++++++++++ 4 files changed, 90 insertions(+), 6 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java index f27db4f2b4cf..34220791900f 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -88,6 +88,7 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.SerializableSupplier; import org.slf4j.Logger; @@ -161,7 +162,9 @@ public class IcebergSink private final transient FlinkMaintenanceConfig flinkMaintenanceConfig; private final Table table; - private final Set equalityFieldColumns = null; + // This should only be used for logging/error messages. For any actual logic always use + // equalityFieldIds instead. + private final Set equalityFieldColumns; private IcebergSink( TableLoader tableLoader, @@ -175,7 +178,8 @@ private IcebergSink( Set equalityFieldIds, String branch, boolean overwriteMode, - FlinkMaintenanceConfig flinkMaintenanceConfig) { + FlinkMaintenanceConfig flinkMaintenanceConfig, + Set equalityFieldColumns) { this.tableLoader = tableLoader; this.snapshotProperties = snapshotProperties; this.uidSuffix = uidSuffix; @@ -197,6 +201,7 @@ private IcebergSink( this.sinkId = UUID.randomUUID().toString(); this.compactMode = flinkWriteConf.compactMode(); this.flinkMaintenanceConfig = flinkMaintenanceConfig; + this.equalityFieldColumns = equalityFieldColumns; } @Override @@ -665,6 +670,9 @@ IcebergSink build() { FlinkMaintenanceConfig flinkMaintenanceConfig = new FlinkMaintenanceConfig(table, writeOptions, readableConfig); + Set equalityFieldColumnsSet = + equalityFieldColumns != null ? Sets.newHashSet(equalityFieldColumns) : null; + return new IcebergSink( tableLoader, table, @@ -679,7 +687,8 @@ IcebergSink build() { equalityFieldIds, flinkWriteConf.branch(), overwriteMode, - flinkMaintenanceConfig); + flinkMaintenanceConfig, + equalityFieldColumnsSet); } /** diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java index 0feb4cc282d2..89f2c7b0daad 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java @@ -562,6 +562,39 @@ public void testRangeDistributionStatisticsMigration() throws Exception { } } + @TestTemplate + public void testHashDistributionWithPartitionNotInEqualityFields() { + assumeThat(partitioned).isTrue(); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + if (isTableSchema) { + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_TABLE_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(writeParallelism) + .distributionMode(DistributionMode.HASH) + .upsert(false) + .equalityFieldColumns(ImmutableList.of("id")) + .append(); + } else { + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(writeParallelism) + .distributionMode(DistributionMode.HASH) + .upsert(false) + .equalityFieldColumns(ImmutableList.of("id")) + .append(); + } + + assertThatThrownBy(env::execute) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining( + "In 'hash' distribution mode with equality fields set, source column 'data' of partition field '1000: data: identity(2)' should be included in equality fields: '[id]'"); + } + private BoundedTestSource createRangeDistributionBoundedSource( List> rowsPerCheckpoint) { return new BoundedTestSource<>(rowsPerCheckpoint); diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java index 593230262fde..470bbc41ded2 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -89,6 +89,7 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.SerializableSupplier; import org.slf4j.Logger; @@ -162,7 +163,9 @@ public class IcebergSink private final transient FlinkMaintenanceConfig flinkMaintenanceConfig; private final Table table; - private final Set equalityFieldColumns = null; + // This should only be used for logging/error messages. For any actual logic always use + // equalityFieldIds instead. + private final Set equalityFieldColumns; private IcebergSink( TableLoader tableLoader, @@ -176,7 +179,8 @@ private IcebergSink( Set equalityFieldIds, String branch, boolean overwriteMode, - FlinkMaintenanceConfig flinkMaintenanceConfig) { + FlinkMaintenanceConfig flinkMaintenanceConfig, + Set equalityFieldColumns) { this.tableLoader = tableLoader; this.snapshotProperties = snapshotProperties; this.uidSuffix = uidSuffix; @@ -198,6 +202,7 @@ private IcebergSink( this.sinkId = UUID.randomUUID().toString(); this.compactMode = flinkWriteConf.compactMode(); this.flinkMaintenanceConfig = flinkMaintenanceConfig; + this.equalityFieldColumns = equalityFieldColumns; } @Override @@ -666,6 +671,9 @@ IcebergSink build() { FlinkMaintenanceConfig flinkMaintenanceConfig = new FlinkMaintenanceConfig(table, writeOptions, readableConfig); + Set equalityFieldColumnsSet = + equalityFieldColumns != null ? Sets.newHashSet(equalityFieldColumns) : null; + return new IcebergSink( tableLoader, table, @@ -680,7 +688,8 @@ IcebergSink build() { equalityFieldIds, flinkWriteConf.branch(), overwriteMode, - flinkMaintenanceConfig); + flinkMaintenanceConfig, + equalityFieldColumnsSet); } /** diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java index 0feb4cc282d2..89f2c7b0daad 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2DistributionMode.java @@ -562,6 +562,39 @@ public void testRangeDistributionStatisticsMigration() throws Exception { } } + @TestTemplate + public void testHashDistributionWithPartitionNotInEqualityFields() { + assumeThat(partitioned).isTrue(); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + if (isTableSchema) { + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_TABLE_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(writeParallelism) + .distributionMode(DistributionMode.HASH) + .upsert(false) + .equalityFieldColumns(ImmutableList.of("id")) + .append(); + } else { + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(writeParallelism) + .distributionMode(DistributionMode.HASH) + .upsert(false) + .equalityFieldColumns(ImmutableList.of("id")) + .append(); + } + + assertThatThrownBy(env::execute) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining( + "In 'hash' distribution mode with equality fields set, source column 'data' of partition field '1000: data: identity(2)' should be included in equality fields: '[id]'"); + } + private BoundedTestSource createRangeDistributionBoundedSource( List> rowsPerCheckpoint) { return new BoundedTestSource<>(rowsPerCheckpoint); From d75451833a91c2e747b1f9f6e141b83e7679614f Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Tue, 6 Jan 2026 11:50:13 -0800 Subject: [PATCH 151/201] Core: Reduce manifest logging noise on drop table (#14969) --- core/src/main/java/org/apache/iceberg/CatalogUtil.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index a96234629232..5390dfdd39a7 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -40,7 +40,6 @@ import org.apache.iceberg.io.SupportsStorageCredentials; import org.apache.iceberg.metrics.LoggingMetricsReporter; import org.apache.iceberg.metrics.MetricsReporter; -import org.apache.iceberg.relocated.com.google.common.base.Joiner; 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.Iterables; @@ -112,7 +111,12 @@ public static void dropTableData(FileIO io, TableMetadata metadata) { } } - LOG.info("Manifests to delete: {}", Joiner.on(", ").join(manifestsToDelete)); + LOG.info("{} Manifests to delete ", manifestsToDelete.size()); + if (LOG.isDebugEnabled()) { + for (ManifestFile manifest : manifestsToDelete) { + LOG.debug("Deleting manifest file: {}", manifest.path()); + } + } // run all of the deletes From bde85b0c347692cd1d2b322f6ab970583e8e3a77 Mon Sep 17 00:00:00 2001 From: Joy Haldar Date: Wed, 7 Jan 2026 02:02:51 +0530 Subject: [PATCH 152/201] API, Spark: Optimize NOT IN and != predicate evaluation for fields containing a single-value (#14593) --- .../InclusiveMetricsEvaluator.java | 44 +++++ .../TestInclusiveMetricsEvaluator.java | 168 ++++++++++++++++++ .../iceberg/spark/source/TestSparkScan.java | 7 +- .../iceberg/spark/source/TestSparkScan.java | 7 +- .../iceberg/spark/source/TestSparkScan.java | 7 +- .../iceberg/spark/source/TestSparkScan.java | 7 +- 6 files changed, 224 insertions(+), 16 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java b/api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java index aa0441f49011..81cbbe785519 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java +++ b/api/src/main/java/org/apache/iceberg/expressions/InclusiveMetricsEvaluator.java @@ -327,6 +327,14 @@ public Boolean eq(Bound term, Literal lit) { public Boolean notEq(Bound term, Literal lit) { // because the bounds are not necessarily a min or max value, this cannot be answered using // them. notEq(col, X) with (X, Y) doesn't guarantee that X is a value in col. + // However, when min == max and the file has no nulls or NaN values, we can safely prune + // if that value equals the literal. + T value = uniqueValue(term); + + if (value != null && lit.comparator().compare(value, lit.value()) == 0) { + return ROWS_CANNOT_MATCH; + } + return ROWS_MIGHT_MATCH; } @@ -381,6 +389,14 @@ public Boolean in(Bound term, Set literalSet) { public Boolean notIn(Bound term, Set literalSet) { // because the bounds are not necessarily a min or max value, this cannot be answered using // them. notIn(col, {X, ...}) with (X, Y) doesn't guarantee that X is a value in col. + // However, when min == max and the file has no nulls or NaN values, we can safely prune + // if that value is in the exclusion set. + T value = uniqueValue(term); + + if (value != null && literalSet.contains(value)) { + return ROWS_CANNOT_MATCH; + } + return ROWS_MIGHT_MATCH; } @@ -490,6 +506,34 @@ private boolean containsNaNsOnly(Integer id) { && nanCounts.get(id).equals(valueCounts.get(id)); } + /** + * Returns the column's single value if all rows contain the same value. Defined as a column + * with no nulls, no NaNs, and lower bound equals upper bound. Returns null otherwise. + */ + private T uniqueValue(Bound term) { + int id = term.ref().fieldId(); + if (mayContainNull(id)) { + return null; + } + + T lower = lowerBound(term); + T upper = upperBound(term); + + if (lower == null || upper == null || NaNUtil.isNaN(lower) || NaNUtil.isNaN(upper)) { + return null; + } + + if (nanCounts != null && nanCounts.containsKey(id) && nanCounts.get(id) != 0) { + return null; + } + + if (!lower.equals(upper)) { + return null; + } + + return lower; + } + private T lowerBound(Bound term) { if (term instanceof BoundReference) { return parseLowerBound((BoundReference) term); diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java index 8ae34c4c16ce..5f0ca2659fbf 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java @@ -970,4 +970,172 @@ public void testNotNullInNestedStruct() { .as("Should not read: optional_address.optional_street2 is optional") .isFalse(); } + + @Test + public void testNotEqWithSingleValue() { + DataFile rangeOfValues = + new TestDataFile( + "range_of_values.avro", + Row.of(), + 10, + ImmutableMap.of(3, 10L), + ImmutableMap.of(3, 0L), + ImmutableMap.of(3, 0L), + ImmutableMap.of(3, toByteBuffer(StringType.get(), "aaa")), + ImmutableMap.of(3, toByteBuffer(StringType.get(), "zzz"))); + + boolean shouldRead = + new InclusiveMetricsEvaluator(SCHEMA, notEqual("required", "aaa")).eval(rangeOfValues); + assertThat(shouldRead) + .as("Should read: file has range of values, cannot exclude based on literal") + .isTrue(); + + DataFile singleValueFile = + new TestDataFile( + "single_value.avro", + Row.of(), + 10, + ImmutableMap.of(3, 10L), + ImmutableMap.of(3, 0L), + ImmutableMap.of(3, 0L), + ImmutableMap.of(3, toByteBuffer(StringType.get(), "abc")), + ImmutableMap.of(3, toByteBuffer(StringType.get(), "abc"))); + + shouldRead = + new InclusiveMetricsEvaluator(SCHEMA, notEqual("required", "abc")).eval(singleValueFile); + assertThat(shouldRead) + .as("Should not read: file contains single value equal to literal") + .isFalse(); + + shouldRead = + new InclusiveMetricsEvaluator(SCHEMA, notEqual("required", "def")).eval(singleValueFile); + assertThat(shouldRead) + .as("Should read: file contains single value not equal to literal") + .isTrue(); + + DataFile singleValueWithNulls = + new TestDataFile( + "single_value_nulls.avro", + Row.of(), + 10, + ImmutableMap.of(3, 10L), + ImmutableMap.of(3, 2L), + ImmutableMap.of(3, 0L), + ImmutableMap.of(3, toByteBuffer(StringType.get(), "abc")), + ImmutableMap.of(3, toByteBuffer(StringType.get(), "abc"))); + + shouldRead = + new InclusiveMetricsEvaluator(SCHEMA, notEqual("required", "abc")) + .eval(singleValueWithNulls); + assertThat(shouldRead).as("Should read: file has nulls which match != predicate").isTrue(); + + DataFile singleValueWithNaN = + new TestDataFile( + "single_value_nan.avro", + Row.of(), + 10, + ImmutableMap.of(9, 10L), + ImmutableMap.of(9, 0L), + ImmutableMap.of(9, 2L), + ImmutableMap.of(9, toByteBuffer(Types.FloatType.get(), 5.0F)), + ImmutableMap.of(9, toByteBuffer(Types.FloatType.get(), 5.0F))); + + shouldRead = + new InclusiveMetricsEvaluator(SCHEMA, notEqual("no_nans", 5.0F)).eval(singleValueWithNaN); + assertThat(shouldRead).as("Should read: file has NaN values which match != predicate").isTrue(); + + DataFile singleValueNaNBounds = + new TestDataFile( + "single_value_nan_bounds.avro", + Row.of(), + 10, + ImmutableMap.of(9, 10L), + ImmutableMap.of(9, 0L), + ImmutableMap.of(9, 0L), + ImmutableMap.of(9, toByteBuffer(Types.FloatType.get(), Float.NaN)), + ImmutableMap.of(9, toByteBuffer(Types.FloatType.get(), Float.NaN))); + + shouldRead = + new InclusiveMetricsEvaluator(SCHEMA, notEqual("no_nans", 5.0F)).eval(singleValueNaNBounds); + assertThat(shouldRead).as("Should read: bounds are NaN").isTrue(); + } + + @Test + public void testNotInWithSingleValue() { + DataFile rangeOfValues = + new TestDataFile( + "range_of_values.avro", + Row.of(), + 10, + ImmutableMap.of(3, 10L), + ImmutableMap.of(3, 0L), + ImmutableMap.of(3, 0L), + ImmutableMap.of(3, toByteBuffer(StringType.get(), "aaa")), + ImmutableMap.of(3, toByteBuffer(StringType.get(), "zzz"))); + + boolean shouldRead = + new InclusiveMetricsEvaluator(SCHEMA, notIn("required", "aaa", "bbb")).eval(rangeOfValues); + assertThat(shouldRead) + .as("Should read: file has range of values, cannot exclude based on literal") + .isTrue(); + + DataFile singleValueFile = + new TestDataFile( + "single_value.avro", + Row.of(), + 10, + ImmutableMap.of(3, 10L), + ImmutableMap.of(3, 0L), + ImmutableMap.of(3, 0L), + ImmutableMap.of(3, toByteBuffer(StringType.get(), "abc")), + ImmutableMap.of(3, toByteBuffer(StringType.get(), "abc"))); + + shouldRead = + new InclusiveMetricsEvaluator(SCHEMA, notIn("required", "abc", "def")) + .eval(singleValueFile); + assertThat(shouldRead) + .as("Should not read: file contains single value in exclusion list") + .isFalse(); + + shouldRead = + new InclusiveMetricsEvaluator(SCHEMA, notIn("required", "def", "ghi")) + .eval(singleValueFile); + assertThat(shouldRead) + .as("Should read: file contains single value not in exclusion list") + .isTrue(); + + DataFile singleValueWithNulls = + new TestDataFile( + "single_value_nulls.avro", + Row.of(), + 10, + ImmutableMap.of(3, 10L), + ImmutableMap.of(3, 2L), + ImmutableMap.of(3, 0L), + ImmutableMap.of(3, toByteBuffer(StringType.get(), "abc")), + ImmutableMap.of(3, toByteBuffer(StringType.get(), "abc"))); + + shouldRead = + new InclusiveMetricsEvaluator(SCHEMA, notIn("required", "abc", "def")) + .eval(singleValueWithNulls); + assertThat(shouldRead).as("Should read: file has nulls which match NOT IN predicate").isTrue(); + + DataFile singleValueWithNaN = + new TestDataFile( + "single_value_nan.avro", + Row.of(), + 10, + ImmutableMap.of(9, 10L), + ImmutableMap.of(9, 0L), + ImmutableMap.of(9, 2L), + ImmutableMap.of(9, toByteBuffer(Types.FloatType.get(), 5.0F)), + ImmutableMap.of(9, toByteBuffer(Types.FloatType.get(), 5.0F))); + + shouldRead = + new InclusiveMetricsEvaluator(SCHEMA, notIn("no_nans", 5.0F, 10.0F)) + .eval(singleValueWithNaN); + assertThat(shouldRead) + .as("Should read: file has NaN values which match NOT IN predicate") + .isTrue(); + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java index a248d4e12827..0a3c09dffa00 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java @@ -461,8 +461,7 @@ public void testUnpartitionedYears() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - // notEq can't be answered using column bounds because they are not exact - assertThat(scan.planInputPartitions()).hasSize(10); + assertThat(scan.planInputPartitions()).hasSize(5); } @TestTemplate @@ -771,7 +770,7 @@ public void testUnpartitionedTruncateString() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - assertThat(scan.planInputPartitions()).hasSize(10); + assertThat(scan.planInputPartitions()).hasSize(5); // NOT NotEqual builder = scanBuilder(); @@ -990,7 +989,7 @@ public void testUnpartitionedOr() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - assertThat(scan.planInputPartitions()).hasSize(10); + assertThat(scan.planInputPartitions()).hasSize(5); } @TestTemplate diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java index 1ddf9318f608..417a84d82769 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java @@ -461,8 +461,7 @@ public void testUnpartitionedYears() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - // notEq can't be answered using column bounds because they are not exact - assertThat(scan.planInputPartitions()).hasSize(10); + assertThat(scan.planInputPartitions()).hasSize(5); } @TestTemplate @@ -771,7 +770,7 @@ public void testUnpartitionedTruncateString() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - assertThat(scan.planInputPartitions()).hasSize(10); + assertThat(scan.planInputPartitions()).hasSize(5); // NOT NotEqual builder = scanBuilder(); @@ -990,7 +989,7 @@ public void testUnpartitionedOr() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - assertThat(scan.planInputPartitions()).hasSize(10); + assertThat(scan.planInputPartitions()).hasSize(5); } @TestTemplate diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java index 1ddf9318f608..417a84d82769 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkScan.java @@ -461,8 +461,7 @@ public void testUnpartitionedYears() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - // notEq can't be answered using column bounds because they are not exact - assertThat(scan.planInputPartitions()).hasSize(10); + assertThat(scan.planInputPartitions()).hasSize(5); } @TestTemplate @@ -771,7 +770,7 @@ public void testUnpartitionedTruncateString() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - assertThat(scan.planInputPartitions()).hasSize(10); + assertThat(scan.planInputPartitions()).hasSize(5); // NOT NotEqual builder = scanBuilder(); @@ -990,7 +989,7 @@ public void testUnpartitionedOr() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - assertThat(scan.planInputPartitions()).hasSize(10); + assertThat(scan.planInputPartitions()).hasSize(5); } @TestTemplate 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 1ddf9318f608..417a84d82769 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 @@ -461,8 +461,7 @@ public void testUnpartitionedYears() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - // notEq can't be answered using column bounds because they are not exact - assertThat(scan.planInputPartitions()).hasSize(10); + assertThat(scan.planInputPartitions()).hasSize(5); } @TestTemplate @@ -771,7 +770,7 @@ public void testUnpartitionedTruncateString() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - assertThat(scan.planInputPartitions()).hasSize(10); + assertThat(scan.planInputPartitions()).hasSize(5); // NOT NotEqual builder = scanBuilder(); @@ -990,7 +989,7 @@ public void testUnpartitionedOr() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - assertThat(scan.planInputPartitions()).hasSize(10); + assertThat(scan.planInputPartitions()).hasSize(5); } @TestTemplate From 7bfe144173cf45995b0dfc2639b20838cac6de4c Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 6 Jan 2026 13:02:08 -0800 Subject: [PATCH 153/201] Flink: fix VisibleForTesting import in ZkLockFactory (#14977) --- .../org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java | 2 +- .../org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java | 2 +- .../org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java index 27c0a43a858a..7797da0caac3 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; -import org.apache.curator.shaded.com.google.common.annotations.VisibleForTesting; import org.apache.flink.shaded.curator5.org.apache.curator.RetryPolicy; import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework; import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFrameworkFactory; @@ -31,6 +30,7 @@ import org.apache.flink.shaded.curator5.org.apache.curator.retry.RetryNTimes; import org.apache.flink.shaded.curator5.org.apache.curator.retry.RetryOneTime; import org.apache.flink.shaded.curator5.org.apache.curator.retry.RetryUntilElapsed; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java index 27c0a43a858a..7797da0caac3 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; -import org.apache.curator.shaded.com.google.common.annotations.VisibleForTesting; import org.apache.flink.shaded.curator5.org.apache.curator.RetryPolicy; import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework; import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFrameworkFactory; @@ -31,6 +30,7 @@ import org.apache.flink.shaded.curator5.org.apache.curator.retry.RetryNTimes; import org.apache.flink.shaded.curator5.org.apache.curator.retry.RetryOneTime; import org.apache.flink.shaded.curator5.org.apache.curator.retry.RetryUntilElapsed; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java index 27c0a43a858a..7797da0caac3 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/ZkLockFactory.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.concurrent.TimeUnit; -import org.apache.curator.shaded.com.google.common.annotations.VisibleForTesting; import org.apache.flink.shaded.curator5.org.apache.curator.RetryPolicy; import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFramework; import org.apache.flink.shaded.curator5.org.apache.curator.framework.CuratorFrameworkFactory; @@ -31,6 +30,7 @@ import org.apache.flink.shaded.curator5.org.apache.curator.retry.RetryNTimes; import org.apache.flink.shaded.curator5.org.apache.curator.retry.RetryOneTime; import org.apache.flink.shaded.curator5.org.apache.curator.retry.RetryUntilElapsed; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From 234af35ae143dae6369eca03365dc166b92d0b84 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Tue, 6 Jan 2026 20:17:01 -0500 Subject: [PATCH 154/201] site: fix live loading in make serve-dev --- site/README.md | 1 - site/dev/serve-dev.sh | 4 +--- site/dev/serve.sh | 2 +- 3 files changed, 2 insertions(+), 5 deletions(-) diff --git a/site/README.md b/site/README.md index c0fd351adcee..212b0a706fd3 100644 --- a/site/README.md +++ b/site/README.md @@ -141,7 +141,6 @@ make serve-dev This development mode: - **Only builds `nightly` and `latest` versions** - Skips all historical versions - **Significantly reduces build time** - Typically 5-10x faster than building all versions -- **Uses the `--dirty` flag** - Only rebuilds changed files for even faster iteration - **Perfect for iterative development** - Great for working on documentation content The development mode sets the `ICEBERG_DEV_MODE=true` environment variable and uses a simplified mkdocs configuration (`mkdocs-dev.yml`) that only includes the most recent versions. diff --git a/site/dev/serve-dev.sh b/site/dev/serve-dev.sh index 8e3ef3d1ac75..f98afd307363 100755 --- a/site/dev/serve-dev.sh +++ b/site/dev/serve-dev.sh @@ -32,6 +32,4 @@ echo "" ./dev/setup_env.sh -# Using mkdocs serve with --dirty flag for even faster rebuilds -# The --dirty flag means only changed files are rebuilt -"${VENV_DIR}/bin/python3" -m mkdocs serve --dirty --watch . -f mkdocs-dev.yml +"${VENV_DIR}/bin/python3" -m mkdocs serve -f mkdocs-dev.yml --livereload --watch . diff --git a/site/dev/serve.sh b/site/dev/serve.sh index 2be2e5f62661..3e670e72b563 100755 --- a/site/dev/serve.sh +++ b/site/dev/serve.sh @@ -23,4 +23,4 @@ set -e ./dev/lint.sh -"${VENV_DIR}/bin/python3" -m mkdocs serve --dirty --watch . +"${VENV_DIR}/bin/python3" -m mkdocs serve --livereload --watch . From 46c871ccbaa6a4494433778fc38af889afefa735 Mon Sep 17 00:00:00 2001 From: Stas Pak Date: Wed, 7 Jan 2026 00:22:14 -0800 Subject: [PATCH 155/201] Spark: Add Spark app name to env context (#14976) --- core/src/main/java/org/apache/iceberg/CatalogProperties.java | 1 + .../iceberg/spark/extensions/TestRewriteDataFilesProcedure.java | 1 + .../extensions/TestRewritePositionDeleteFilesProcedure.java | 1 + .../src/main/java/org/apache/iceberg/spark/SparkCatalog.java | 1 + 4 files changed, 4 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/CatalogProperties.java b/core/src/main/java/org/apache/iceberg/CatalogProperties.java index c17ed59b2674..f35c90c4e80c 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogProperties.java +++ b/core/src/main/java/org/apache/iceberg/CatalogProperties.java @@ -155,6 +155,7 @@ private CatalogProperties() {} public static final String LOCK_TABLE = "lock.table"; public static final String APP_ID = "app-id"; + public static final String APP_NAME = "app-name"; public static final String USER = "user"; public static final String AUTH_SESSION_TIMEOUT_MS = "auth.session-timeout-ms"; diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index e23a06a76d33..66c95621bf2d 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -976,6 +976,7 @@ public void testRewriteDataFilesSummary() { Map summary = snapshotSummary(); assertThat(summary) .containsKey(CatalogProperties.APP_ID) + .containsKey(CatalogProperties.APP_NAME) .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") .hasEntrySatisfying( EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("4.1")); diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java index cd824c191080..311cf763eeef 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java @@ -239,6 +239,7 @@ public void testRewriteSummary() throws Exception { Map summary = snapshotSummary(); assertThat(summary) .containsKey(CatalogProperties.APP_ID) + .containsKey(CatalogProperties.APP_NAME) .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") .hasEntrySatisfying( EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("4.1")); 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 dba3ea6ecc1c..da22607d05b0 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 @@ -797,6 +797,7 @@ public final void initialize(String name, CaseInsensitiveStringMap options) { EnvironmentContext.put( EnvironmentContext.ENGINE_VERSION, sparkSession.sparkContext().version()); EnvironmentContext.put(CatalogProperties.APP_ID, sparkSession.sparkContext().applicationId()); + EnvironmentContext.put(CatalogProperties.APP_NAME, sparkSession.sparkContext().appName()); } @Override From 055a73a92cc2f14b8adbd7b0851d6c5b8538db5c Mon Sep 17 00:00:00 2001 From: Thomas Powell Date: Wed, 7 Jan 2026 08:33:45 +0000 Subject: [PATCH 156/201] AWS: Merge catalog properties with properties prefixed with client.credentials-provider. (#14608) --- .../iceberg/aws/AwsClientProperties.java | 13 +++--- .../iceberg/aws/TestAwsClientProperties.java | 21 ++++++---- .../org/apache/iceberg/util/PropertyUtil.java | 15 +++++++ .../apache/iceberg/util/TestPropertyUtil.java | 41 +++++++++++++++++++ 4 files changed, 76 insertions(+), 14 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/util/TestPropertyUtil.java diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java index cf73e80f44c1..056959df2301 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.Map; +import java.util.function.Predicate; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.aws.s3.VendedCredentialsProvider; import org.apache.iceberg.common.DynClasses; @@ -28,7 +29,6 @@ import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.rest.RESTUtil; import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.SerializableMap; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; @@ -97,7 +97,6 @@ public class AwsClientProperties implements Serializable { private final String refreshCredentialsEndpoint; private final boolean refreshCredentialsEnabled; private final boolean legacyMd5pluginEnabled; - private final Map allProperties; public AwsClientProperties() { this.clientRegion = null; @@ -106,15 +105,18 @@ public AwsClientProperties() { this.refreshCredentialsEndpoint = null; this.refreshCredentialsEnabled = true; this.legacyMd5pluginEnabled = false; - this.allProperties = null; } public AwsClientProperties(Map properties) { - this.allProperties = SerializableMap.copyOf(properties); this.clientRegion = properties.get(CLIENT_REGION); this.clientCredentialsProvider = properties.get(CLIENT_CREDENTIALS_PROVIDER); + // Retain all non-prefixed properties and override with prefixed properties this.clientCredentialsProviderProperties = - PropertyUtil.propertiesWithPrefix(properties, CLIENT_CREDENTIAL_PROVIDER_PREFIX); + PropertyUtil.mergeProperties( + PropertyUtil.filterProperties( + properties, + Predicate.not(property -> property.startsWith(CLIENT_CREDENTIAL_PROVIDER_PREFIX))), + PropertyUtil.propertiesWithPrefix(properties, CLIENT_CREDENTIAL_PROVIDER_PREFIX)); this.refreshCredentialsEndpoint = RESTUtil.resolveEndpoint( properties.get(CatalogProperties.URI), properties.get(REFRESH_CREDENTIALS_ENDPOINT)); @@ -211,7 +213,6 @@ public void applyClientCredentialConfigurati public AwsCredentialsProvider credentialsProvider( String accessKeyId, String secretAccessKey, String sessionToken) { if (refreshCredentialsEnabled && !Strings.isNullOrEmpty(refreshCredentialsEndpoint)) { - clientCredentialsProviderProperties.putAll(allProperties); clientCredentialsProviderProperties.put( VendedCredentialsProvider.URI, refreshCredentialsEndpoint); return credentialsProvider(VendedCredentialsProvider.class.getName()); diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientProperties.java b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientProperties.java index 8c177ab5274e..cac8d892badb 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientProperties.java @@ -180,7 +180,7 @@ public void refreshCredentialsEndpointWithOverridingOAuthToken() { Map properties = ImmutableMap.of( CatalogProperties.URI, - "http://localhost:1234/v1", + "http://localhost:1234/v1/catalog", AwsClientProperties.REFRESH_CREDENTIALS_ENDPOINT, "http://localhost:1234/v1/credentials", OAuth2Properties.TOKEN, @@ -189,17 +189,22 @@ public void refreshCredentialsEndpointWithOverridingOAuthToken() { "specific-token"); AwsClientProperties awsClientProperties = new AwsClientProperties(properties); - Map expectedProperties = - ImmutableMap.builder() - .putAll(properties) - .put("credentials.uri", "http://localhost:1234/v1/credentials") - .build(); - AwsCredentialsProvider provider = awsClientProperties.credentialsProvider("key", "secret", "token"); assertThat(provider).isInstanceOf(VendedCredentialsProvider.class); VendedCredentialsProvider vendedCredentialsProvider = (VendedCredentialsProvider) provider; - assertThat(vendedCredentialsProvider).extracting("properties").isEqualTo(expectedProperties); + assertThat(vendedCredentialsProvider) + .extracting("properties") + .isEqualTo( + ImmutableMap.of( + AwsClientProperties.REFRESH_CREDENTIALS_ENDPOINT, + "http://localhost:1234/v1/credentials", + "credentials.uri", + "http://localhost:1234/v1/credentials", + CatalogProperties.URI, + "http://localhost:1234/v1/catalog", + OAuth2Properties.TOKEN, + "specific-token")); } @Test diff --git a/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java b/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java index 633b0a6ae739..0498a34a0b0e 100644 --- a/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/PropertyUtil.java @@ -191,6 +191,21 @@ public static Map filterProperties( .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } + public static Map mergeProperties( + Map properties, Map overrides) { + if (overrides == null || overrides.isEmpty()) { + return properties; + } + + if (properties == null || properties.isEmpty()) { + return overrides; + } + + Map merged = Maps.newHashMap(properties); + merged.putAll(overrides); + return merged; + } + public static Map applySchemaChanges( Map properties, List deletedColumns, diff --git a/core/src/test/java/org/apache/iceberg/util/TestPropertyUtil.java b/core/src/test/java/org/apache/iceberg/util/TestPropertyUtil.java new file mode 100644 index 000000000000..91c6759a3c8f --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/util/TestPropertyUtil.java @@ -0,0 +1,41 @@ +/* + * 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.util; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.junit.jupiter.api.Test; + +public class TestPropertyUtil { + + @Test + void mergeProperties() { + Map properties = Map.of("k1", "v1", "k2", "v2"); + Map overrides = Map.of("k1", "v11", "k3", "v3"); + + assertThat(PropertyUtil.mergeProperties(null, null)).isNull(); + assertThat(PropertyUtil.mergeProperties(properties, null)).isEqualTo(properties); + assertThat(PropertyUtil.mergeProperties(properties, Map.of())).isEqualTo(properties); + assertThat(PropertyUtil.mergeProperties(null, overrides)).isEqualTo(overrides); + assertThat(PropertyUtil.mergeProperties(Map.of(), overrides)).isEqualTo(overrides); + assertThat(PropertyUtil.mergeProperties(properties, overrides)) + .containsExactlyInAnyOrderEntriesOf(Map.of("k1", "v11", "k2", "v2", "k3", "v3")); + } +} From b07c1e570d5db28a937b3fc632ef2f2788932818 Mon Sep 17 00:00:00 2001 From: Varun Lakhyani <130844282+varun-lakhyani@users.noreply.github.com> Date: Wed, 7 Jan 2026 16:49:22 +0530 Subject: [PATCH 157/201] Spark: Backport: Add Spark app name to env context for Spark v3.4, 3.5, 4.0 (#14981) Backports #14976 --- .../iceberg/spark/extensions/TestRewriteDataFilesProcedure.java | 1 + .../extensions/TestRewritePositionDeleteFilesProcedure.java | 1 + .../src/main/java/org/apache/iceberg/spark/SparkCatalog.java | 1 + .../iceberg/spark/extensions/TestRewriteDataFilesProcedure.java | 1 + .../extensions/TestRewritePositionDeleteFilesProcedure.java | 1 + .../src/main/java/org/apache/iceberg/spark/SparkCatalog.java | 1 + .../iceberg/spark/extensions/TestRewriteDataFilesProcedure.java | 1 + .../extensions/TestRewritePositionDeleteFilesProcedure.java | 1 + .../src/main/java/org/apache/iceberg/spark/SparkCatalog.java | 1 + 9 files changed, 9 insertions(+) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index 7ba93935349d..d011ad062843 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -926,6 +926,7 @@ public void testRewriteDataFilesSummary() { Map summary = snapshotSummary(); assertThat(summary) .containsKey(CatalogProperties.APP_ID) + .containsKey(CatalogProperties.APP_NAME) .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") .hasEntrySatisfying( EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("3.4")); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java index bec5e06b37a1..0ff3a949ae51 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java @@ -239,6 +239,7 @@ public void testRewriteSummary() throws Exception { Map summary = snapshotSummary(); assertThat(summary) .containsKey(CatalogProperties.APP_ID) + .containsKey(CatalogProperties.APP_NAME) .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") .hasEntrySatisfying( EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("3.4")); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 0f49c54544f8..c07508568dbf 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -797,6 +797,7 @@ public final void initialize(String name, CaseInsensitiveStringMap options) { EnvironmentContext.put( EnvironmentContext.ENGINE_VERSION, sparkSession.sparkContext().version()); EnvironmentContext.put(CatalogProperties.APP_ID, sparkSession.sparkContext().applicationId()); + EnvironmentContext.put(CatalogProperties.APP_NAME, sparkSession.sparkContext().appName()); } @Override diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index ae88e04e447e..00734dd1dc27 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -978,6 +978,7 @@ public void testRewriteDataFilesSummary() { Map summary = snapshotSummary(); assertThat(summary) .containsKey(CatalogProperties.APP_ID) + .containsKey(CatalogProperties.APP_NAME) .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") .hasEntrySatisfying( EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("3.5")); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java index eae9208022a1..feafaff27b45 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java @@ -239,6 +239,7 @@ public void testRewriteSummary() throws Exception { Map summary = snapshotSummary(); assertThat(summary) .containsKey(CatalogProperties.APP_ID) + .containsKey(CatalogProperties.APP_NAME) .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") .hasEntrySatisfying( EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("3.5")); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 5e1cd977954b..913fe25e6dd1 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -797,6 +797,7 @@ public final void initialize(String name, CaseInsensitiveStringMap options) { EnvironmentContext.put( EnvironmentContext.ENGINE_VERSION, sparkSession.sparkContext().version()); EnvironmentContext.put(CatalogProperties.APP_ID, sparkSession.sparkContext().applicationId()); + EnvironmentContext.put(CatalogProperties.APP_NAME, sparkSession.sparkContext().appName()); } @Override diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index 3aabd635bb69..15bbfb02eb89 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -976,6 +976,7 @@ public void testRewriteDataFilesSummary() { Map summary = snapshotSummary(); assertThat(summary) .containsKey(CatalogProperties.APP_ID) + .containsKey(CatalogProperties.APP_NAME) .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") .hasEntrySatisfying( EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("4.0")); diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java index 4a6d2ff1979e..006379adda56 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java @@ -239,6 +239,7 @@ public void testRewriteSummary() throws Exception { Map summary = snapshotSummary(); assertThat(summary) .containsKey(CatalogProperties.APP_ID) + .containsKey(CatalogProperties.APP_NAME) .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") .hasEntrySatisfying( EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("4.0")); diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index dba3ea6ecc1c..da22607d05b0 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -797,6 +797,7 @@ public final void initialize(String name, CaseInsensitiveStringMap options) { EnvironmentContext.put( EnvironmentContext.ENGINE_VERSION, sparkSession.sparkContext().version()); EnvironmentContext.put(CatalogProperties.APP_ID, sparkSession.sparkContext().applicationId()); + EnvironmentContext.put(CatalogProperties.APP_NAME, sparkSession.sparkContext().appName()); } @Override From 51d548a4fc101df3494e17a5f7ff5c615c5979cc Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Wed, 7 Jan 2026 13:58:23 +0100 Subject: [PATCH 158/201] API, Core: Scan API for partition stats (#14640) --- .../apache/iceberg/PartitionStatistics.java | 65 +++ .../iceberg/PartitionStatisticsScan.java | 59 +++ .../main/java/org/apache/iceberg/Table.java | 12 + .../iceberg/BasePartitionStatistics.java | 201 ++++++++ .../iceberg/BasePartitionStatisticsScan.java | 86 ++++ .../java/org/apache/iceberg/BaseTable.java | 5 + .../org/apache/iceberg/PartitionStats.java | 6 + .../apache/iceberg/PartitionStatsHandler.java | 2 + .../PartitionStatisticsScanTestBase.java | 480 ++++++++++++++++++ .../iceberg/PartitionStatisticsTestBase.java | 114 +++++ .../PartitionStatsHandlerTestBase.java | 112 +--- .../avro/TestAvroPartitionStatisticsScan.java | 29 ++ .../orc/TestOrcPartitionStatisticsScan.java | 59 +++ .../TestParquetPartitionStatisticsScan.java | 30 ++ 14 files changed, 1164 insertions(+), 96 deletions(-) create mode 100644 api/src/main/java/org/apache/iceberg/PartitionStatistics.java create mode 100644 api/src/main/java/org/apache/iceberg/PartitionStatisticsScan.java create mode 100644 core/src/main/java/org/apache/iceberg/BasePartitionStatistics.java create mode 100644 core/src/main/java/org/apache/iceberg/BasePartitionStatisticsScan.java create mode 100644 core/src/test/java/org/apache/iceberg/PartitionStatisticsScanTestBase.java create mode 100644 core/src/test/java/org/apache/iceberg/PartitionStatisticsTestBase.java create mode 100644 core/src/test/java/org/apache/iceberg/avro/TestAvroPartitionStatisticsScan.java create mode 100644 orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatisticsScan.java create mode 100644 parquet/src/test/java/org/apache/iceberg/parquet/TestParquetPartitionStatisticsScan.java diff --git a/api/src/main/java/org/apache/iceberg/PartitionStatistics.java b/api/src/main/java/org/apache/iceberg/PartitionStatistics.java new file mode 100644 index 000000000000..10df7303d500 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/PartitionStatistics.java @@ -0,0 +1,65 @@ +/* + * 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; + +/** Interface for partition statistics returned from a {@link PartitionStatisticsScan}. */ +public interface PartitionStatistics extends StructLike { + + /** Returns the partition of these partition statistics */ + StructLike partition(); + + /** Returns the spec ID of the partition of these partition statistics */ + Integer specId(); + + /** Returns the number of data records in the partition */ + Long dataRecordCount(); + + /** Returns the number of data files in the partition */ + Integer dataFileCount(); + + /** Returns the total size of data files in bytes in the partition */ + Long totalDataFileSizeInBytes(); + + /** + * Returns the number of positional delete records in the partition. Also includes dv record count + * as per spec + */ + Long positionDeleteRecordCount(); + + /** Returns the number of positional delete files in the partition */ + Integer positionDeleteFileCount(); + + /** Returns the number of equality delete records in the partition */ + Long equalityDeleteRecordCount(); + + /** Returns the number of equality delete files in the partition */ + Integer equalityDeleteFileCount(); + + /** Returns the total number of records in the partition */ + Long totalRecords(); + + /** Returns the timestamp in milliseconds when the partition was last updated */ + Long lastUpdatedAt(); + + /** Returns the ID of the snapshot that last updated this partition */ + Long lastUpdatedSnapshotId(); + + /** Returns the number of delete vectors in the partition */ + Integer dvCount(); +} diff --git a/api/src/main/java/org/apache/iceberg/PartitionStatisticsScan.java b/api/src/main/java/org/apache/iceberg/PartitionStatisticsScan.java new file mode 100644 index 000000000000..18d8b2031821 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/PartitionStatisticsScan.java @@ -0,0 +1,59 @@ +/* + * 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; + +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; + +/** API for configuring partition statistics scan. */ +public interface PartitionStatisticsScan { + + /** + * Create a new scan from this scan's configuration that will use the given snapshot by ID. + * + * @param snapshotId a snapshot ID + * @return a new scan based on this with the given snapshot ID + * @throws IllegalArgumentException if the snapshot cannot be found + */ + PartitionStatisticsScan useSnapshot(long snapshotId); + + /** + * Create a new scan from the results of this, where partitions are filtered by the {@link + * Expression}. + * + * @param filter a filter expression + * @return a new scan based on this with results filtered by the expression + */ + PartitionStatisticsScan filter(Expression filter); + + /** + * Create a new scan from this with the schema as its projection. + * + * @param schema a projection schema + * @return a new scan based on this with the given projection + */ + PartitionStatisticsScan project(Schema schema); + + /** + * Scans a partition statistics file belonging to a particular snapshot + * + * @return an Iterable of partition statistics + */ + CloseableIterable scan(); +} diff --git a/api/src/main/java/org/apache/iceberg/Table.java b/api/src/main/java/org/apache/iceberg/Table.java index 97ea9ba76526..3c0689e89288 100644 --- a/api/src/main/java/org/apache/iceberg/Table.java +++ b/api/src/main/java/org/apache/iceberg/Table.java @@ -83,6 +83,18 @@ default IncrementalChangelogScan newIncrementalChangelogScan() { throw new UnsupportedOperationException("Incremental changelog scan is not supported"); } + /** + * Create a new {@link PartitionStatisticsScan} for this table. + * + *

    Once a partition statistics scan is created, it can be refined to project columns and filter + * data. + * + * @return a partition statistics scan for this table + */ + default PartitionStatisticsScan newPartitionStatisticsScan() { + throw new UnsupportedOperationException("Partition statistics scan is not supported"); + } + /** * Return the {@link Schema schema} for this table. * diff --git a/core/src/main/java/org/apache/iceberg/BasePartitionStatistics.java b/core/src/main/java/org/apache/iceberg/BasePartitionStatistics.java new file mode 100644 index 000000000000..c17718281b57 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/BasePartitionStatistics.java @@ -0,0 +1,201 @@ +/* + * 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; + +import org.apache.iceberg.avro.SupportsIndexProjection; +import org.apache.iceberg.types.Types; + +public class BasePartitionStatistics extends SupportsIndexProjection + implements PartitionStatistics { + + private StructLike partition; + private Integer specId; + private Long dataRecordCount; + private Integer dataFileCount; + private Long totalDataFileSizeInBytes; + private Long positionDeleteRecordCount; + private Integer positionDeleteFileCount; + private Long equalityDeleteRecordCount; + private Integer equalityDeleteFileCount; + private Long totalRecordCount; // Not calculated, as it needs scanning the data. Remains null + private Long lastUpdatedAt; + private Long lastUpdatedSnapshotId; + private Integer dvCount; + + private static final int STATS_COUNT = 13; + + /** Used by internal readers to instantiate this class with a projection schema. */ + public BasePartitionStatistics(Types.StructType projection) { + super(STATS_COUNT); + } + + @Override + public StructLike partition() { + return partition; + } + + @Override + public Integer specId() { + return specId; + } + + @Override + public Long dataRecordCount() { + return dataRecordCount; + } + + @Override + public Integer dataFileCount() { + return dataFileCount; + } + + @Override + public Long totalDataFileSizeInBytes() { + return totalDataFileSizeInBytes; + } + + @Override + public Long positionDeleteRecordCount() { + return positionDeleteRecordCount; + } + + @Override + public Integer positionDeleteFileCount() { + return positionDeleteFileCount; + } + + @Override + public Long equalityDeleteRecordCount() { + return equalityDeleteRecordCount; + } + + @Override + public Integer equalityDeleteFileCount() { + return equalityDeleteFileCount; + } + + @Override + public Long totalRecords() { + return totalRecordCount; + } + + @Override + public Long lastUpdatedAt() { + return lastUpdatedAt; + } + + @Override + public Long lastUpdatedSnapshotId() { + return lastUpdatedSnapshotId; + } + + @Override + public Integer dvCount() { + return dvCount; + } + + @Override + protected T internalGet(int pos, Class javaClass) { + return javaClass.cast(getByPos(pos)); + } + + private Object getByPos(int pos) { + switch (pos) { + case 0: + return partition; + case 1: + return specId; + case 2: + return dataRecordCount; + case 3: + return dataFileCount; + case 4: + return totalDataFileSizeInBytes; + case 5: + return positionDeleteRecordCount; + case 6: + return positionDeleteFileCount; + case 7: + return equalityDeleteRecordCount; + case 8: + return equalityDeleteFileCount; + case 9: + return totalRecordCount; + case 10: + return lastUpdatedAt; + case 11: + return lastUpdatedSnapshotId; + case 12: + return dvCount; + default: + throw new UnsupportedOperationException("Unknown position: " + pos); + } + } + + @Override + protected void internalSet(int pos, T value) { + if (value == null) { + return; + } + + switch (pos) { + case 0: + this.partition = (StructLike) value; + break; + case 1: + this.specId = (int) value; + break; + case 2: + this.dataRecordCount = (long) value; + break; + case 3: + this.dataFileCount = (int) value; + break; + case 4: + this.totalDataFileSizeInBytes = (long) value; + break; + case 5: + this.positionDeleteRecordCount = (long) value; + break; + case 6: + this.positionDeleteFileCount = (int) value; + break; + case 7: + this.equalityDeleteRecordCount = (long) value; + break; + case 8: + this.equalityDeleteFileCount = (int) value; + break; + case 9: + this.totalRecordCount = (Long) value; + break; + case 10: + this.lastUpdatedAt = (Long) value; + break; + case 11: + this.lastUpdatedSnapshotId = (Long) value; + break; + case 12: + this.dvCount = (int) value; + break; + default: + throw new UnsupportedOperationException("Unknown position: " + pos); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/BasePartitionStatisticsScan.java b/core/src/main/java/org/apache/iceberg/BasePartitionStatisticsScan.java new file mode 100644 index 000000000000..075a1a85d394 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/BasePartitionStatisticsScan.java @@ -0,0 +1,86 @@ +/* + * 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; + +import java.util.Optional; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Types; + +public class BasePartitionStatisticsScan implements PartitionStatisticsScan { + + private final Table table; + private Long snapshotId; + + public BasePartitionStatisticsScan(Table table) { + this.table = table; + } + + @Override + public PartitionStatisticsScan useSnapshot(long newSnapshotId) { + Preconditions.checkArgument( + table.snapshot(newSnapshotId) != null, "Cannot find snapshot with ID %s", newSnapshotId); + + this.snapshotId = newSnapshotId; + return this; + } + + @Override + public PartitionStatisticsScan filter(Expression newFilter) { + throw new UnsupportedOperationException("Filtering is not supported"); + } + + @Override + public PartitionStatisticsScan project(Schema newSchema) { + throw new UnsupportedOperationException("Projection is not supported"); + } + + @Override + public CloseableIterable scan() { + if (snapshotId == null) { + if (table.currentSnapshot() == null) { + return CloseableIterable.empty(); + } + + snapshotId = table.currentSnapshot().snapshotId(); + } + + Optional statsFile = + table.partitionStatisticsFiles().stream() + .filter(f -> f.snapshotId() == snapshotId) + .findFirst(); + + if (statsFile.isEmpty()) { + return CloseableIterable.empty(); + } + + Types.StructType partitionType = Partitioning.partitionType(table); + Schema schema = PartitionStatsHandler.schema(partitionType, TableUtil.formatVersion(table)); + + FileFormat fileFormat = FileFormat.fromFileName(statsFile.get().path()); + Preconditions.checkNotNull( + fileFormat != null, "Unable to determine format of file: %s", statsFile.get().path()); + + return InternalData.read(fileFormat, table.io().newInputFile(statsFile.get().path())) + .project(schema) + .setRootType(BasePartitionStatistics.class) + .build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/BaseTable.java b/core/src/main/java/org/apache/iceberg/BaseTable.java index 23299a962ce5..c489c3bfb517 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseTable.java @@ -90,6 +90,11 @@ public IncrementalChangelogScan newIncrementalChangelogScan() { return new BaseIncrementalChangelogScan(this); } + @Override + public PartitionStatisticsScan newPartitionStatisticsScan() { + return new BasePartitionStatisticsScan(this); + } + @Override public Schema schema() { return ops.current().schema(); diff --git a/core/src/main/java/org/apache/iceberg/PartitionStats.java b/core/src/main/java/org/apache/iceberg/PartitionStats.java index 9051c8535c7e..e8a4e18916bc 100644 --- a/core/src/main/java/org/apache/iceberg/PartitionStats.java +++ b/core/src/main/java/org/apache/iceberg/PartitionStats.java @@ -20,6 +20,12 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +/** + * Class to hold partition statistics values. + * + * @deprecated will be removed in 1.12.0. Use {@link BasePartitionStatistics instead} + */ +@Deprecated public class PartitionStats implements StructLike { private static final int STATS_COUNT = 13; diff --git a/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java b/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java index 4e7c1b104ee8..7259a1f0684b 100644 --- a/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java +++ b/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java @@ -275,7 +275,9 @@ static PartitionStatisticsFile writePartitionStatsFile( * * @param schema The {@link Schema} of the partition statistics file. * @param inputFile An {@link InputFile} pointing to the partition stats file. + * @deprecated will be removed in 1.12.0, use {@link PartitionStatisticsScan} instead */ + @Deprecated public static CloseableIterable readPartitionStatsFile( Schema schema, InputFile inputFile) { Preconditions.checkArgument(schema != null, "Invalid schema: null"); diff --git a/core/src/test/java/org/apache/iceberg/PartitionStatisticsScanTestBase.java b/core/src/test/java/org/apache/iceberg/PartitionStatisticsScanTestBase.java new file mode 100644 index 000000000000..89eb70959c6d --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/PartitionStatisticsScanTestBase.java @@ -0,0 +1,480 @@ +/* + * 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; + +import static org.apache.iceberg.PartitionStatsHandler.PARTITION_FIELD_ID; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Types; +import org.assertj.core.groups.Tuple; +import org.junit.jupiter.api.Test; + +public abstract class PartitionStatisticsScanTestBase extends PartitionStatisticsTestBase { + + public abstract FileFormat format(); + + private final Map fileFormatProperty = + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format().name()); + + @Test + public void testEmptyTable() throws Exception { + Table testTable = + TestTables.create( + tempDir("scan_empty_table"), "scan_empty_table", SCHEMA, SPEC, 2, fileFormatProperty); + + assertThat(Lists.newArrayList(testTable.newPartitionStatisticsScan().scan())).isEmpty(); + } + + @Test + public void testInvalidSnapshotId() throws Exception { + Table testTable = + TestTables.create( + tempDir("scan_invalid_snapshot"), + "scan_invalid_snapshot", + SCHEMA, + SPEC, + 2, + fileFormatProperty); + + assertThatThrownBy(() -> testTable.newPartitionStatisticsScan().useSnapshot(1234L).scan()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find snapshot with ID 1234"); + } + + @Test + public void testNoStatsForSnapshot() throws Exception { + Table testTable = + TestTables.create( + tempDir("scan_no_stats"), "scan_no_stats", SCHEMA, SPEC, 2, fileFormatProperty); + + DataFile dataFile = + DataFiles.builder(SPEC) + .withPath("some_path") + .withFileSizeInBytes(15) + .withFormat(format()) + .withRecordCount(1) + .build(); + testTable.newAppend().appendFile(dataFile).commit(); + long snapshotId = testTable.currentSnapshot().snapshotId(); + + assertThat(testTable.newPartitionStatisticsScan().useSnapshot(snapshotId).scan()).isEmpty(); + } + + @Test + public void testReadingStatsWithInvalidSchema() throws Exception { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + Table testTable = + TestTables.create( + tempDir("scan_with_old_schema"), + "scan_with_old_schema", + SCHEMA, + spec, + 2, + fileFormatProperty); + Types.StructType partitionType = Partitioning.partitionType(testTable); + Schema oldSchema = invalidOldSchema(partitionType); + + // Add a dummy file to the table to have a snapshot + DataFile dataFile = + DataFiles.builder(spec) + .withPath("some_path") + .withFileSizeInBytes(15) + .withFormat(FileFormat.PARQUET) + .withRecordCount(1) + .build(); + testTable.newAppend().appendFile(dataFile).commit(); + long snapshotId = testTable.currentSnapshot().snapshotId(); + + testTable + .updatePartitionStatistics() + .setPartitionStatistics( + PartitionStatsHandler.writePartitionStatsFile( + testTable, + snapshotId, + oldSchema, + Collections.singletonList(randomStats(partitionType)))) + .commit(); + + try (CloseableIterable recordIterator = + testTable.newPartitionStatisticsScan().useSnapshot(snapshotId).scan()) { + + if (format() == FileFormat.PARQUET) { + assertThatThrownBy(() -> Lists.newArrayList(recordIterator)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Not a primitive type: struct"); + } else if (format() == FileFormat.AVRO) { + assertThatThrownBy(() -> Lists.newArrayList(recordIterator)) + .isInstanceOf(ClassCastException.class) + .hasMessageContaining("Integer cannot be cast to class org.apache.iceberg.StructLike"); + } + } + } + + @Test + public void testV2toV3SchemaEvolution() throws Exception { + Table testTable = + TestTables.create( + tempDir("scan_with_schema_evolution"), + "scan_with_schema_evolution", + SCHEMA, + SPEC, + 2, + fileFormatProperty); + + // write stats file using v2 schema + DataFile dataFile = + FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("foo", "A")); + testTable.newAppend().appendFile(dataFile).commit(); + + testTable + .updatePartitionStatistics() + .setPartitionStatistics( + PartitionStatsHandler.computeAndWriteStatsFile( + testTable, testTable.currentSnapshot().snapshotId())) + .commit(); + + Types.StructType partitionSchema = Partitioning.partitionType(testTable); + + // read with v2 schema + List partitionStatsV2; + try (CloseableIterable recordIterator = + testTable.newPartitionStatisticsScan().scan()) { + partitionStatsV2 = Lists.newArrayList(recordIterator); + } + + testTable.updateProperties().set(TableProperties.FORMAT_VERSION, "3").commit(); + + // read with v3 schema + List partitionStatsV3; + try (CloseableIterable recordIterator = + testTable.newPartitionStatisticsScan().scan()) { + partitionStatsV3 = Lists.newArrayList(recordIterator); + } + + assertThat(partitionStatsV2).hasSameSizeAs(partitionStatsV3); + Comparator comparator = Comparators.forType(partitionSchema); + for (int i = 0; i < partitionStatsV2.size(); i++) { + assertThat(isEqual(comparator, partitionStatsV2.get(i), partitionStatsV3.get(i))).isTrue(); + } + } + + @SuppressWarnings("checkstyle:MethodLength") + @Test + public void testScanPartitionStatsForCurrentSnapshot() throws Exception { + Table testTable = + TestTables.create( + tempDir("scan_partition_stats"), + "scan_partition_stats", + SCHEMA, + SPEC, + 2, + fileFormatProperty); + + DataFile dataFile1 = + FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("foo", "A")); + DataFile dataFile2 = + FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("foo", "B")); + DataFile dataFile3 = + FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("bar", "A")); + DataFile dataFile4 = + FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("bar", "B")); + + for (int i = 0; i < 3; i++) { + // insert same set of seven records thrice to have a new manifest files + testTable + .newAppend() + .appendFile(dataFile1) + .appendFile(dataFile2) + .appendFile(dataFile3) + .appendFile(dataFile4) + .commit(); + } + + Snapshot snapshot1 = testTable.currentSnapshot(); + Schema recordSchema = PartitionStatsHandler.schema(Partitioning.partitionType(testTable), 2); + + Types.StructType partitionType = + recordSchema.findField(PARTITION_FIELD_ID).type().asStructType(); + computeAndValidatePartitionStats( + testTable, + testTable.currentSnapshot().snapshotId(), + Tuple.tuple( + partitionRecord(partitionType, "foo", "A"), + 0, + 3 * dataFile1.recordCount(), + 3, + 3 * dataFile1.fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + null, + snapshot1.timestampMillis(), + snapshot1.snapshotId(), + null), + Tuple.tuple( + partitionRecord(partitionType, "foo", "B"), + 0, + 3 * dataFile2.recordCount(), + 3, + 3 * dataFile2.fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + null, + snapshot1.timestampMillis(), + snapshot1.snapshotId(), + null), + Tuple.tuple( + partitionRecord(partitionType, "bar", "A"), + 0, + 3 * dataFile3.recordCount(), + 3, + 3 * dataFile3.fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + null, + snapshot1.timestampMillis(), + snapshot1.snapshotId(), + null), + Tuple.tuple( + partitionRecord(partitionType, "bar", "B"), + 0, + 3 * dataFile4.recordCount(), + 3, + 3 * dataFile4.fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + null, + snapshot1.timestampMillis(), + snapshot1.snapshotId(), + null)); + + DeleteFile posDelete = + FileGenerationUtil.generatePositionDeleteFile(testTable, TestHelpers.Row.of("bar", "A")); + testTable.newRowDelta().addDeletes(posDelete).commit(); + // snapshot2 is unused in the result as same partition was updated by snapshot4 + + DeleteFile eqDelete = + FileGenerationUtil.generateEqualityDeleteFile(testTable, TestHelpers.Row.of("foo", "A")); + testTable.newRowDelta().addDeletes(eqDelete).commit(); + Snapshot snapshot3 = testTable.currentSnapshot(); + + testTable.updateProperties().set(TableProperties.FORMAT_VERSION, "3").commit(); + DeleteFile dv = FileGenerationUtil.generateDV(testTable, dataFile3); + testTable.newRowDelta().addDeletes(dv).commit(); + Snapshot snapshot4 = testTable.currentSnapshot(); + + computeAndValidatePartitionStats( + testTable, + testTable.currentSnapshot().snapshotId(), + Tuple.tuple( + partitionRecord(partitionType, "foo", "A"), + 0, + 3 * dataFile1.recordCount(), + 3, + 3 * dataFile1.fileSizeInBytes(), + 0L, + 0, + eqDelete.recordCount(), + 1, + null, + snapshot3.timestampMillis(), + snapshot3.snapshotId(), + 0), + Tuple.tuple( + partitionRecord(partitionType, "foo", "B"), + 0, + 3 * dataFile2.recordCount(), + 3, + 3 * dataFile2.fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + null, + snapshot1.timestampMillis(), + snapshot1.snapshotId(), + 0), + Tuple.tuple( + partitionRecord(partitionType, "bar", "A"), + 0, + 3 * dataFile3.recordCount(), + 3, + 3 * dataFile3.fileSizeInBytes(), + posDelete.recordCount() + dv.recordCount(), + 1, + 0L, + 0, + null, + snapshot4.timestampMillis(), + snapshot4.snapshotId(), + 1), // dv count + Tuple.tuple( + partitionRecord(partitionType, "bar", "B"), + 0, + 3 * dataFile4.recordCount(), + 3, + 3 * dataFile4.fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + null, + snapshot1.timestampMillis(), + snapshot1.snapshotId(), + 0)); + } + + @Test + public void testScanPartitionStatsForOlderSnapshot() throws Exception { + Table testTable = + TestTables.create( + tempDir("scan_older_snapshot"), + "scan_older_snapshot", + SCHEMA, + SPEC, + 2, + fileFormatProperty); + + DataFile dataFile1 = + FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("foo", "A")); + DataFile dataFile2 = + FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("foo", "B")); + + testTable.newAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); + + Snapshot firstSnapshot = testTable.currentSnapshot(); + + testTable.newAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); + + Schema recordSchema = PartitionStatsHandler.schema(Partitioning.partitionType(testTable), 2); + + Types.StructType partitionType = + recordSchema.findField(PARTITION_FIELD_ID).type().asStructType(); + + computeAndValidatePartitionStats( + testTable, + firstSnapshot.snapshotId(), + Tuple.tuple( + partitionRecord(partitionType, "foo", "A"), + 0, + dataFile1.recordCount(), + 1, + dataFile1.fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + null, + firstSnapshot.timestampMillis(), + firstSnapshot.snapshotId(), + null), + Tuple.tuple( + partitionRecord(partitionType, "foo", "B"), + 0, + dataFile2.recordCount(), + 1, + dataFile2.fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + null, + firstSnapshot.timestampMillis(), + firstSnapshot.snapshotId(), + null)); + } + + private static void computeAndValidatePartitionStats( + Table testTable, long snapshotId, Tuple... expectedValues) throws IOException { + PartitionStatisticsFile result = + PartitionStatsHandler.computeAndWriteStatsFile(testTable, snapshotId); + testTable.updatePartitionStatistics().setPartitionStatistics(result).commit(); + assertThat(result.snapshotId()).isEqualTo(snapshotId); + + PartitionStatisticsScan statScan = testTable.newPartitionStatisticsScan(); + if (testTable.currentSnapshot().snapshotId() != snapshotId) { + statScan.useSnapshot(snapshotId); + } + + List partitionStats; + try (CloseableIterable recordIterator = statScan.scan()) { + partitionStats = Lists.newArrayList(recordIterator); + } + + assertThat(partitionStats) + .extracting( + PartitionStatistics::partition, + PartitionStatistics::specId, + PartitionStatistics::dataRecordCount, + PartitionStatistics::dataFileCount, + PartitionStatistics::totalDataFileSizeInBytes, + PartitionStatistics::positionDeleteRecordCount, + PartitionStatistics::positionDeleteFileCount, + PartitionStatistics::equalityDeleteRecordCount, + PartitionStatistics::equalityDeleteFileCount, + PartitionStatistics::totalRecords, + PartitionStatistics::lastUpdatedAt, + PartitionStatistics::lastUpdatedSnapshotId, + PartitionStatistics::dvCount) + .containsExactlyInAnyOrder(expectedValues); + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + protected static boolean isEqual( + Comparator partitionComparator, + PartitionStatistics stats1, + PartitionStatistics stats2) { + if (stats1 == stats2) { + return true; + } else if (stats1 == null || stats2 == null) { + return false; + } + + return partitionComparator.compare(stats1.partition(), stats2.partition()) == 0 + && Objects.equals(stats1.specId(), stats2.specId()) + && Objects.equals(stats1.dataRecordCount(), stats2.dataRecordCount()) + && Objects.equals(stats1.dataFileCount(), stats2.dataFileCount()) + && Objects.equals(stats1.totalDataFileSizeInBytes(), stats2.totalDataFileSizeInBytes()) + && Objects.equals(stats1.positionDeleteRecordCount(), stats2.positionDeleteRecordCount()) + && Objects.equals(stats1.positionDeleteFileCount(), stats2.positionDeleteFileCount()) + && Objects.equals(stats1.equalityDeleteRecordCount(), stats2.equalityDeleteRecordCount()) + && Objects.equals(stats1.equalityDeleteFileCount(), stats2.equalityDeleteFileCount()) + && Objects.equals(stats1.totalRecords(), stats2.totalRecords()) + && Objects.equals(stats1.lastUpdatedAt(), stats2.lastUpdatedAt()) + && Objects.equals(stats1.lastUpdatedSnapshotId(), stats2.lastUpdatedSnapshotId()); + } +} diff --git a/core/src/test/java/org/apache/iceberg/PartitionStatisticsTestBase.java b/core/src/test/java/org/apache/iceberg/PartitionStatisticsTestBase.java new file mode 100644 index 000000000000..72a5405d7f1b --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/PartitionStatisticsTestBase.java @@ -0,0 +1,114 @@ +/* + * 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; + +import static org.apache.iceberg.PartitionStatsHandler.DATA_FILE_COUNT; +import static org.apache.iceberg.PartitionStatsHandler.DATA_RECORD_COUNT; +import static org.apache.iceberg.PartitionStatsHandler.EQUALITY_DELETE_FILE_COUNT; +import static org.apache.iceberg.PartitionStatsHandler.EQUALITY_DELETE_RECORD_COUNT; +import static org.apache.iceberg.PartitionStatsHandler.LAST_UPDATED_AT; +import static org.apache.iceberg.PartitionStatsHandler.LAST_UPDATED_SNAPSHOT_ID; +import static org.apache.iceberg.PartitionStatsHandler.PARTITION_FIELD_NAME; +import static org.apache.iceberg.PartitionStatsHandler.POSITION_DELETE_FILE_COUNT; +import static org.apache.iceberg.PartitionStatsHandler.POSITION_DELETE_RECORD_COUNT; +import static org.apache.iceberg.PartitionStatsHandler.SPEC_ID; +import static org.apache.iceberg.PartitionStatsHandler.TOTAL_DATA_FILE_SIZE_IN_BYTES; +import static org.apache.iceberg.PartitionStatsHandler.TOTAL_RECORD_COUNT; +import static org.apache.iceberg.types.Types.NestedField.optional; + +import java.io.File; +import java.io.IOException; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.io.TempDir; + +public abstract class PartitionStatisticsTestBase { + + @TempDir private File temp; + + // positions in StructLike + protected static final int DATA_RECORD_COUNT_POSITION = 2; + protected static final int DATA_FILE_COUNT_POSITION = 3; + protected static final int TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION = 4; + protected static final int POSITION_DELETE_RECORD_COUNT_POSITION = 5; + protected static final int POSITION_DELETE_FILE_COUNT_POSITION = 6; + protected static final int EQUALITY_DELETE_RECORD_COUNT_POSITION = 7; + protected static final int EQUALITY_DELETE_FILE_COUNT_POSITION = 8; + protected static final int TOTAL_RECORD_COUNT_POSITION = 9; + protected static final int LAST_UPDATED_AT_POSITION = 10; + protected static final int LAST_UPDATED_SNAPSHOT_ID_POSITION = 11; + protected static final int DV_COUNT_POSITION = 12; + + protected static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + + protected static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("c2").identity("c3").build(); + + private static final Random RANDOM = ThreadLocalRandom.current(); + + protected Schema invalidOldSchema(Types.StructType unifiedPartitionType) { + // field ids starts from 0 instead of 1 + return new Schema( + Types.NestedField.required(0, PARTITION_FIELD_NAME, unifiedPartitionType), + Types.NestedField.required(1, SPEC_ID.name(), Types.IntegerType.get()), + Types.NestedField.required(2, DATA_RECORD_COUNT.name(), Types.LongType.get()), + Types.NestedField.required(3, DATA_FILE_COUNT.name(), Types.IntegerType.get()), + Types.NestedField.required(4, TOTAL_DATA_FILE_SIZE_IN_BYTES.name(), Types.LongType.get()), + Types.NestedField.optional(5, POSITION_DELETE_RECORD_COUNT.name(), Types.LongType.get()), + Types.NestedField.optional(6, POSITION_DELETE_FILE_COUNT.name(), Types.IntegerType.get()), + Types.NestedField.optional(7, EQUALITY_DELETE_RECORD_COUNT.name(), Types.LongType.get()), + Types.NestedField.optional(8, EQUALITY_DELETE_FILE_COUNT.name(), Types.IntegerType.get()), + Types.NestedField.optional(9, TOTAL_RECORD_COUNT.name(), Types.LongType.get()), + Types.NestedField.optional(10, LAST_UPDATED_AT.name(), Types.LongType.get()), + Types.NestedField.optional(11, LAST_UPDATED_SNAPSHOT_ID.name(), Types.LongType.get())); + } + + protected PartitionStats randomStats(Types.StructType partitionType) { + PartitionData partitionData = new PartitionData(partitionType); + partitionData.set(0, RANDOM.nextInt()); + + return randomStats(partitionData); + } + + protected PartitionStats randomStats(PartitionData partitionData) { + PartitionStats stats = new PartitionStats(partitionData, RANDOM.nextInt(10)); + stats.set(DATA_RECORD_COUNT_POSITION, RANDOM.nextLong()); + stats.set(DATA_FILE_COUNT_POSITION, RANDOM.nextInt()); + stats.set(TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION, 1024L * RANDOM.nextInt(20)); + return stats; + } + + protected File tempDir(String folderName) throws IOException { + return java.nio.file.Files.createTempDirectory(temp.toPath(), folderName).toFile(); + } + + protected static StructLike partitionRecord( + Types.StructType partitionType, String val1, String val2) { + GenericRecord record = GenericRecord.create(partitionType); + record.set(0, val1); + record.set(1, val2); + return record; + } +} diff --git a/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java b/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java index 71fdc9507d58..9b93013a9b06 100644 --- a/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java +++ b/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java @@ -18,25 +18,12 @@ */ package org.apache.iceberg; -import static org.apache.iceberg.PartitionStatsHandler.DATA_FILE_COUNT; -import static org.apache.iceberg.PartitionStatsHandler.DATA_RECORD_COUNT; -import static org.apache.iceberg.PartitionStatsHandler.EQUALITY_DELETE_FILE_COUNT; -import static org.apache.iceberg.PartitionStatsHandler.EQUALITY_DELETE_RECORD_COUNT; -import static org.apache.iceberg.PartitionStatsHandler.LAST_UPDATED_AT; -import static org.apache.iceberg.PartitionStatsHandler.LAST_UPDATED_SNAPSHOT_ID; import static org.apache.iceberg.PartitionStatsHandler.PARTITION_FIELD_ID; -import static org.apache.iceberg.PartitionStatsHandler.PARTITION_FIELD_NAME; -import static org.apache.iceberg.PartitionStatsHandler.POSITION_DELETE_FILE_COUNT; -import static org.apache.iceberg.PartitionStatsHandler.POSITION_DELETE_RECORD_COUNT; -import static org.apache.iceberg.PartitionStatsHandler.SPEC_ID; -import static org.apache.iceberg.PartitionStatsHandler.TOTAL_DATA_FILE_SIZE_IN_BYTES; -import static org.apache.iceberg.PartitionStatsHandler.TOTAL_RECORD_COUNT; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import java.io.File; import java.io.IOException; import java.math.BigDecimal; import java.nio.ByteBuffer; @@ -46,10 +33,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Random; import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; -import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -61,10 +45,9 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.io.TempDir; @ExtendWith(ParameterizedTestExtension.class) -public abstract class PartitionStatsHandlerTestBase { +public abstract class PartitionStatsHandlerTestBase extends PartitionStatisticsTestBase { public abstract FileFormat format(); @@ -75,35 +58,9 @@ protected static List formatVersions() { @Parameter protected int formatVersion; - private static final Schema SCHEMA = - new Schema( - optional(1, "c1", Types.IntegerType.get()), - optional(2, "c2", Types.StringType.get()), - optional(3, "c3", Types.StringType.get())); - - protected static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("c2").identity("c3").build(); - - @TempDir public File temp; - - private static final Random RANDOM = ThreadLocalRandom.current(); - private final Map fileFormatProperty = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format().name()); - // position in StructLike - private static final int DATA_RECORD_COUNT_POSITION = 2; - private static final int DATA_FILE_COUNT_POSITION = 3; - private static final int TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION = 4; - private static final int POSITION_DELETE_RECORD_COUNT_POSITION = 5; - private static final int POSITION_DELETE_FILE_COUNT_POSITION = 6; - private static final int EQUALITY_DELETE_RECORD_COUNT_POSITION = 7; - private static final int EQUALITY_DELETE_FILE_COUNT_POSITION = 8; - private static final int TOTAL_RECORD_COUNT_POSITION = 9; - private static final int LAST_UPDATED_AT_POSITION = 10; - private static final int LAST_UPDATED_SNAPSHOT_ID_POSITION = 11; - private static final int DV_COUNT_POSITION = 12; - @Test public void testPartitionStatsOnEmptyTable() throws Exception { Table testTable = @@ -223,10 +180,7 @@ public void testAllDatatypePartitionWriting() throws Exception { partitionData.set(13, new BigDecimal("12345678901234567890.1234567890")); partitionData.set(14, Literal.of("10:10:10").to(Types.TimeType.get()).value()); - PartitionStats partitionStats = new PartitionStats(partitionData, RANDOM.nextInt(10)); - partitionStats.set(DATA_RECORD_COUNT_POSITION, RANDOM.nextLong()); - partitionStats.set(DATA_FILE_COUNT_POSITION, RANDOM.nextInt()); - partitionStats.set(TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION, 1024L * RANDOM.nextInt(20)); + PartitionStats partitionStats = randomStats(partitionData); List expected = Collections.singletonList(partitionStats); PartitionStatisticsFile statisticsFile = PartitionStatsHandler.writePartitionStatsFile(testTable, 42L, dataSchema, expected); @@ -262,14 +216,8 @@ public void testOptionalFieldsWriting() throws Exception { ImmutableList.Builder partitionListBuilder = ImmutableList.builder(); for (int i = 0; i < 5; i++) { - PartitionData partitionData = - new PartitionData(dataSchema.findField(PARTITION_FIELD_ID).type().asStructType()); - partitionData.set(0, RANDOM.nextInt()); - - PartitionStats stats = new PartitionStats(partitionData, RANDOM.nextInt(10)); - stats.set(DATA_RECORD_COUNT_POSITION, RANDOM.nextLong()); - stats.set(DATA_FILE_COUNT_POSITION, RANDOM.nextInt()); - stats.set(TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION, 1024L * RANDOM.nextInt(20)); + PartitionStats stats = + randomStats(dataSchema.findField(PARTITION_FIELD_ID).type().asStructType()); stats.set(POSITION_DELETE_RECORD_COUNT_POSITION, null); stats.set(POSITION_DELETE_FILE_COUNT_POSITION, null); stats.set(EQUALITY_DELETE_RECORD_COUNT_POSITION, null); @@ -315,8 +263,12 @@ public void testOptionalFieldsWriting() throws Exception { } } + /** + * @deprecated will be removed in 1.12.0 + */ @SuppressWarnings("checkstyle:MethodLength") @Test + @Deprecated public void testPartitionStats() throws Exception { Table testTable = TestTables.create( @@ -611,7 +563,11 @@ public void testLatestStatsFileWithBranch() throws Exception { assertThat(PartitionStatsHandler.latestStatsFile(testTable, snapshotBranchBId)).isNull(); } + /** + * @deprecated will be removed in 1.12.0 + */ @Test + @Deprecated public void testReadingStatsWithInvalidSchema() throws Exception { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); Table testTable = @@ -677,7 +633,11 @@ public void testFullComputeFallbackWithInvalidStats() throws Exception { assertThat(partitionStats.get(0).dataFileCount()).isEqualTo(2); } + /** + * @deprecated will be removed in 1.12.0 + */ @Test + @Deprecated public void testV2toV3SchemaEvolution() throws Exception { Table testTable = TestTables.create( @@ -718,14 +678,6 @@ public void testV2toV3SchemaEvolution() throws Exception { } } - private static StructLike partitionRecord( - Types.StructType partitionType, String val1, String val2) { - GenericRecord record = GenericRecord.create(partitionType); - record.set(0, val1); - record.set(1, val2); - return record; - } - private static void computeAndValidatePartitionStats( Table testTable, Schema recordSchema, Tuple... expectedValues) throws IOException { // compute and commit partition stats file @@ -760,38 +712,6 @@ private static void computeAndValidatePartitionStats( .containsExactlyInAnyOrder(expectedValues); } - private File tempDir(String folderName) throws IOException { - return java.nio.file.Files.createTempDirectory(temp.toPath(), folderName).toFile(); - } - - private Schema invalidOldSchema(Types.StructType unifiedPartitionType) { - // field ids starts from 0 instead of 1 - return new Schema( - Types.NestedField.required(0, PARTITION_FIELD_NAME, unifiedPartitionType), - Types.NestedField.required(1, SPEC_ID.name(), Types.IntegerType.get()), - Types.NestedField.required(2, DATA_RECORD_COUNT.name(), Types.LongType.get()), - Types.NestedField.required(3, DATA_FILE_COUNT.name(), Types.IntegerType.get()), - Types.NestedField.required(4, TOTAL_DATA_FILE_SIZE_IN_BYTES.name(), Types.LongType.get()), - Types.NestedField.optional(5, POSITION_DELETE_RECORD_COUNT.name(), Types.LongType.get()), - Types.NestedField.optional(6, POSITION_DELETE_FILE_COUNT.name(), Types.IntegerType.get()), - Types.NestedField.optional(7, EQUALITY_DELETE_RECORD_COUNT.name(), Types.LongType.get()), - Types.NestedField.optional(8, EQUALITY_DELETE_FILE_COUNT.name(), Types.IntegerType.get()), - Types.NestedField.optional(9, TOTAL_RECORD_COUNT.name(), Types.LongType.get()), - Types.NestedField.optional(10, LAST_UPDATED_AT.name(), Types.LongType.get()), - Types.NestedField.optional(11, LAST_UPDATED_SNAPSHOT_ID.name(), Types.LongType.get())); - } - - private PartitionStats randomStats(Types.StructType partitionType) { - PartitionData partitionData = new PartitionData(partitionType); - partitionData.set(0, RANDOM.nextInt()); - - PartitionStats stats = new PartitionStats(partitionData, RANDOM.nextInt(10)); - stats.set(DATA_RECORD_COUNT_POSITION, RANDOM.nextLong()); - stats.set(DATA_FILE_COUNT_POSITION, RANDOM.nextInt()); - stats.set(TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION, 1024L * RANDOM.nextInt(20)); - return stats; - } - @SuppressWarnings("checkstyle:CyclomaticComplexity") private static boolean isEqual( Comparator partitionComparator, PartitionStats stats1, PartitionStats stats2) { diff --git a/core/src/test/java/org/apache/iceberg/avro/TestAvroPartitionStatisticsScan.java b/core/src/test/java/org/apache/iceberg/avro/TestAvroPartitionStatisticsScan.java new file mode 100644 index 000000000000..54e03180cedb --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroPartitionStatisticsScan.java @@ -0,0 +1,29 @@ +/* + * 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.avro; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionStatisticsScanTestBase; + +public class TestAvroPartitionStatisticsScan extends PartitionStatisticsScanTestBase { + + public FileFormat format() { + return FileFormat.AVRO; + } +} diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatisticsScan.java b/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatisticsScan.java new file mode 100644 index 000000000000..2040f046ee5d --- /dev/null +++ b/orc/src/test/java/org/apache/iceberg/orc/TestOrcPartitionStatisticsScan.java @@ -0,0 +1,59 @@ +/* + * 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.orc; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionStatisticsScanTestBase; + +public class TestOrcPartitionStatisticsScan extends PartitionStatisticsScanTestBase { + @Override + public FileFormat format() { + return FileFormat.ORC; + } + + @Override + public void testScanPartitionStatsForCurrentSnapshot() throws Exception { + assertThatThrownBy(super::testScanPartitionStatsForCurrentSnapshot) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Cannot write using unregistered internal data format: ORC"); + } + + @Override + public void testScanPartitionStatsForOlderSnapshot() throws Exception { + assertThatThrownBy(super::testScanPartitionStatsForOlderSnapshot) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Cannot write using unregistered internal data format: ORC"); + } + + @Override + public void testReadingStatsWithInvalidSchema() throws Exception { + assertThatThrownBy(super::testReadingStatsWithInvalidSchema) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Cannot write using unregistered internal data format: ORC"); + } + + @Override + public void testV2toV3SchemaEvolution() throws Exception { + assertThatThrownBy(super::testV2toV3SchemaEvolution) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Cannot write using unregistered internal data format: ORC"); + } +} diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetPartitionStatisticsScan.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetPartitionStatisticsScan.java new file mode 100644 index 000000000000..5152e31b28e5 --- /dev/null +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetPartitionStatisticsScan.java @@ -0,0 +1,30 @@ +/* + * 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.parquet; + +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionStatisticsScanTestBase; + +public class TestParquetPartitionStatisticsScan extends PartitionStatisticsScanTestBase { + + @Override + public FileFormat format() { + return FileFormat.PARQUET; + } +} From 1dce77c788a761f74e7ff8a4f5ce2d0a75b06359 Mon Sep 17 00:00:00 2001 From: Ayush Saxena Date: Wed, 7 Jan 2026 20:05:54 +0530 Subject: [PATCH 159/201] Data: Handle TIMESTAMP_NANO in InternalRecordWrapper (#14974) --- .../iceberg/data/InternalRecordWrapper.java | 6 +++ .../apache/iceberg/RecordWrapperTestBase.java | 20 ++++++++++ .../iceberg/TestInternalRecordWrapper.java | 40 +++++++++++++++++++ .../iceberg/flink/TestRowDataWrapper.java | 13 ++++++ .../iceberg/flink/TestRowDataWrapper.java | 13 ++++++ .../iceberg/flink/TestRowDataWrapper.java | 13 ++++++ .../spark/source/TestInternalRowWrapper.java | 12 ++++++ .../spark/source/TestInternalRowWrapper.java | 12 ++++++ .../spark/source/TestInternalRowWrapper.java | 12 ++++++ .../spark/source/TestInternalRowWrapper.java | 12 ++++++ 10 files changed, 153 insertions(+) create mode 100644 data/src/test/java/org/apache/iceberg/TestInternalRecordWrapper.java diff --git a/data/src/main/java/org/apache/iceberg/data/InternalRecordWrapper.java b/data/src/main/java/org/apache/iceberg/data/InternalRecordWrapper.java index 871000279848..828bd58ec9c6 100644 --- a/data/src/main/java/org/apache/iceberg/data/InternalRecordWrapper.java +++ b/data/src/main/java/org/apache/iceberg/data/InternalRecordWrapper.java @@ -59,6 +59,12 @@ private static Function converter(Type type) { } else { return timestamp -> DateTimeUtil.microsFromTimestamp((LocalDateTime) timestamp); } + case TIMESTAMP_NANO: + if (((Types.TimestampNanoType) type).shouldAdjustToUTC()) { + return timestamp -> DateTimeUtil.nanosFromTimestamptz((OffsetDateTime) timestamp); + } else { + return timestamp -> DateTimeUtil.nanosFromTimestamp((LocalDateTime) timestamp); + } case FIXED: return bytes -> ByteBuffer.wrap((byte[]) bytes); case STRUCT: diff --git a/data/src/test/java/org/apache/iceberg/RecordWrapperTestBase.java b/data/src/test/java/org/apache/iceberg/RecordWrapperTestBase.java index f51d1ebed4e4..df0fe6f1879d 100644 --- a/data/src/test/java/org/apache/iceberg/RecordWrapperTestBase.java +++ b/data/src/test/java/org/apache/iceberg/RecordWrapperTestBase.java @@ -52,6 +52,16 @@ public abstract class RecordWrapperTestBase { required(101, "ts0", Types.TimestampType.withoutZone()), required(102, "ts1", Types.TimestampType.withoutZone())); + private static final Types.StructType TIMESTAMP_NS_WITHOUT_ZONE = + Types.StructType.of( + required(101, "ts0", Types.TimestampNanoType.withoutZone()), + required(102, "ts1", Types.TimestampNanoType.withoutZone())); + + private static final Types.StructType TIMESTAMP_NS_WITH_ZONE = + Types.StructType.of( + required(101, "ts0", Types.TimestampNanoType.withZone()), + required(102, "ts1", Types.TimestampNanoType.withZone())); + protected static final Types.StructType TIME = Types.StructType.of( required(100, "time0", Types.TimeType.get()), @@ -67,6 +77,16 @@ public void testTimestampWithoutZone() { generateAndValidate(new Schema(TIMESTAMP_WITHOUT_ZONE.fields())); } + @Test + public void testTimestampNanoWithoutZone() { + generateAndValidate(new Schema(TIMESTAMP_NS_WITHOUT_ZONE.fields())); + } + + @Test + public void testTimestampNanoWithZone() { + generateAndValidate(new Schema(TIMESTAMP_NS_WITH_ZONE.fields())); + } + @Test public void testTime() { generateAndValidate(new Schema(TIME.fields())); diff --git a/data/src/test/java/org/apache/iceberg/TestInternalRecordWrapper.java b/data/src/test/java/org/apache/iceberg/TestInternalRecordWrapper.java new file mode 100644 index 000000000000..14047da7c4ae --- /dev/null +++ b/data/src/test/java/org/apache/iceberg/TestInternalRecordWrapper.java @@ -0,0 +1,40 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; + +public class TestInternalRecordWrapper extends RecordWrapperTestBase { + @Override + protected void generateAndValidate(Schema schema, AssertMethod assertMethod) { + Record record = RandomGenericData.generate(schema, 1, 101L).iterator().next(); + StructLike wrapped = new InternalRecordWrapper(schema.asStruct()).wrap(record); + + for (int i = 0; i < schema.columns().size(); i++) { + Object val = wrapped.get(i, Object.class); + if (val != null) { + assertThat(val).isInstanceOf(schema.columns().get(i).type().typeId().javaClass()); + } + } + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java index 0e7635a33e87..cd6964b5ed0f 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java @@ -30,6 +30,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.data.RandomRowData; import org.apache.iceberg.util.StructLikeWrapper; +import org.junit.jupiter.api.Disabled; public class TestRowDataWrapper extends RecordWrapperTestBase { @@ -59,6 +60,18 @@ public void testTime() { }); } + @Disabled + @Override + public void testTimestampNanoWithoutZone() { + // Flink does not support nanosecond timestamp without zone. + } + + @Disabled + @Override + public void testTimestampNanoWithZone() { + // Flink does not support nanosecond timestamp with zone. + } + @Override protected void generateAndValidate( Schema schema, RecordWrapperTestBase.AssertMethod assertMethod) { diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java index 0e7635a33e87..cd6964b5ed0f 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java @@ -30,6 +30,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.data.RandomRowData; import org.apache.iceberg.util.StructLikeWrapper; +import org.junit.jupiter.api.Disabled; public class TestRowDataWrapper extends RecordWrapperTestBase { @@ -59,6 +60,18 @@ public void testTime() { }); } + @Disabled + @Override + public void testTimestampNanoWithoutZone() { + // Flink does not support nanosecond timestamp without zone. + } + + @Disabled + @Override + public void testTimestampNanoWithZone() { + // Flink does not support nanosecond timestamp with zone. + } + @Override protected void generateAndValidate( Schema schema, RecordWrapperTestBase.AssertMethod assertMethod) { diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java index 0e7635a33e87..abcc2d1da199 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java @@ -30,6 +30,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.data.RandomRowData; import org.apache.iceberg.util.StructLikeWrapper; +import org.junit.jupiter.api.Disabled; public class TestRowDataWrapper extends RecordWrapperTestBase { @@ -91,4 +92,16 @@ protected void generateAndValidate( assertThat(actual).isExhausted(); assertThat(expected).isExhausted(); } + + @Disabled + @Override + public void testTimestampNanoWithoutZone() { + // Flink does not support nanosecond timestamp without zone. + } + + @Disabled + @Override + public void testTimestampNanoWithZone() { + // Flink does not support nanosecond timestamp with zone. + } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java index 05e1cae51666..63fef0f2e37a 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java @@ -47,6 +47,18 @@ public void testTime() { // Spark does not support time fields. } + @Disabled + @Override + public void testTimestampNanoWithoutZone() { + // Spark does not support nanosecond timestamp without zone. + } + + @Disabled + @Override + public void testTimestampNanoWithZone() { + // Spark does not support nanosecond timestamp with zone. + } + @Override protected void generateAndValidate(Schema schema, AssertMethod assertMethod) { int numRecords = 100; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java index 05e1cae51666..63fef0f2e37a 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java @@ -47,6 +47,18 @@ public void testTime() { // Spark does not support time fields. } + @Disabled + @Override + public void testTimestampNanoWithoutZone() { + // Spark does not support nanosecond timestamp without zone. + } + + @Disabled + @Override + public void testTimestampNanoWithZone() { + // Spark does not support nanosecond timestamp with zone. + } + @Override protected void generateAndValidate(Schema schema, AssertMethod assertMethod) { int numRecords = 100; diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java index 05e1cae51666..63fef0f2e37a 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java @@ -47,6 +47,18 @@ public void testTime() { // Spark does not support time fields. } + @Disabled + @Override + public void testTimestampNanoWithoutZone() { + // Spark does not support nanosecond timestamp without zone. + } + + @Disabled + @Override + public void testTimestampNanoWithZone() { + // Spark does not support nanosecond timestamp with zone. + } + @Override protected void generateAndValidate(Schema schema, AssertMethod assertMethod) { int numRecords = 100; diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java index 05e1cae51666..63fef0f2e37a 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java @@ -47,6 +47,18 @@ public void testTime() { // Spark does not support time fields. } + @Disabled + @Override + public void testTimestampNanoWithoutZone() { + // Spark does not support nanosecond timestamp without zone. + } + + @Disabled + @Override + public void testTimestampNanoWithZone() { + // Spark does not support nanosecond timestamp with zone. + } + @Override protected void generateAndValidate(Schema schema, AssertMethod assertMethod) { int numRecords = 100; From b3b6657352bc83329add02baf6154a6df13c3830 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Wed, 7 Jan 2026 11:58:56 -0600 Subject: [PATCH 160/201] Site: Add Iceberg Summit 2026 section to homepage (#14988) --- site/docs/assets/stylesheets/home.css | 50 +++++++++++++++++++++++++++ site/overrides/home.html | 49 ++++++++++++++++++++++++++ 2 files changed, 99 insertions(+) diff --git a/site/docs/assets/stylesheets/home.css b/site/docs/assets/stylesheets/home.css index 98166fdfc0d2..1c45c0e72025 100644 --- a/site/docs/assets/stylesheets/home.css +++ b/site/docs/assets/stylesheets/home.css @@ -30,6 +30,42 @@ text-align: center; /* Center text horizontally */ } +/* Summit Box Styles */ +.summit-box { + margin: 20px auto 15px auto; + padding: 10px 15px 15px 15px; + max-width: 600px; + background: rgba(255, 255, 255, 0.05); + border-radius: 12px; + border: 2px solid rgba(255, 255, 255, 0.2); + box-shadow: 0 4px 15px rgba(0, 0, 0, 0.1); + backdrop-filter: blur(10px); +} + +.summit-box h4 { + text-align: center; + font-weight: 600; + font-size: 20px; + margin-top: 0; + margin-bottom: 10px; +} + +.summit-link-item { + transition: transform 0.3s ease, box-shadow 0.3s ease; + box-shadow: 0 3px 8px rgba(0, 0, 0, 0.2); +} + +.summit-link-item:hover { + transform: translateY(-3px); + box-shadow: 0 6px 15px rgba(0, 0, 0, 0.3); +} + +.summit-link-item a { + display: block; + width: 100%; + height: 100%; +} + /* Media query for smaller screens */ @media (max-width: 767px) { .col-6 { @@ -42,4 +78,18 @@ div#termynal-expressive-sql { left: 0; } + + .summit-box { + max-width: 100%; + margin: 15px 10px; + } + + .summit-links { + flex-direction: column !important; + } + + .summit-link-item { + min-width: 100% !important; + max-width: 100% !important; + } } diff --git a/site/overrides/home.html b/site/overrides/home.html index 65d971e0a134..8d6e49176963 100644 --- a/site/overrides/home.html +++ b/site/overrides/home.html @@ -37,6 +37,27 @@

    Apache Iceberg™

    The open table format for analytic datasets.


    + + +
      {% for social in config.extra.social %}
    • @@ -310,6 +331,34 @@

      Data Compaction

      src="assets/javascript/termynal.js" data-termynal-container="#termynal|#termynal-data-compaction|#termynal-expressive-sql|#termynal-time-travel"> + + + {% endblock %} {% block content %} From aee89008aaebb99eb26ddbd3f79220a30a7ef6c9 Mon Sep 17 00:00:00 2001 From: Thomas Powell Date: Wed, 7 Jan 2026 21:18:25 +0000 Subject: [PATCH 161/201] Include key metadata in manifest tables (#14750) * Include key metadata in Spark manifest table. * Update AllManifestsTable to include key_metadata. * spark 3.4 * Switch to ByteBuffers.toByteArray --------- Co-authored-by: Thomas Powell --- .../org/apache/iceberg/AllManifestsTable.java | 26 +++++++++++-------- .../iceberg/AllManifestsTableTaskParser.java | 9 +++++-- .../TestAllManifestsTableTaskParser.java | 10 ++++--- .../spark/actions/BaseSparkAction.java | 3 ++- .../spark/actions/ManifestFileBean.java | 12 ++++++++- .../spark/actions/BaseSparkAction.java | 3 ++- .../spark/actions/ManifestFileBean.java | 12 ++++++++- .../spark/actions/BaseSparkAction.java | 3 ++- .../spark/actions/ManifestFileBean.java | 12 ++++++++- 9 files changed, 68 insertions(+), 22 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/AllManifestsTable.java b/core/src/main/java/org/apache/iceberg/AllManifestsTable.java index 16f38c0aa2f1..2435de62f0f9 100644 --- a/core/src/main/java/org/apache/iceberg/AllManifestsTable.java +++ b/core/src/main/java/org/apache/iceberg/AllManifestsTable.java @@ -38,6 +38,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; import org.apache.iceberg.util.StructProjection; /** @@ -76,7 +77,8 @@ public class AllManifestsTable extends BaseMetadataTable { Types.NestedField.required(11, "contains_nan", Types.BooleanType.get()), Types.NestedField.optional(12, "lower_bound", Types.StringType.get()), Types.NestedField.optional(13, "upper_bound", Types.StringType.get())))), - REF_SNAPSHOT_ID); + REF_SNAPSHOT_ID, + Types.NestedField.optional(19, "key_metadata", Types.BinaryType.get())); AllManifestsTable(Table table) { this(table, table.name() + ".all_manifests"); @@ -138,7 +140,7 @@ protected CloseableIterable doPlanFiles() { io, schema(), specs, - snap.manifestListLocation(), + new BaseManifestListFile(snap.manifestListLocation(), snap.keyId()), filter, snap.snapshotId()); } else { @@ -161,7 +163,7 @@ static class ManifestListReadTask implements DataTask { private final FileIO io; private final Schema schema; private final Map specs; - private final String manifestListLocation; + private final ManifestListFile manifestList; private final Expression residual; private final long referenceSnapshotId; private DataFile lazyDataFile = null; @@ -171,14 +173,14 @@ static class ManifestListReadTask implements DataTask { FileIO io, Schema schema, Map specs, - String manifestListLocation, + ManifestListFile manifestList, Expression residual, long referenceSnapshotId) { this.dataTableSchema = dataTableSchema; this.io = io; this.schema = schema; this.specs = specs; - this.manifestListLocation = manifestListLocation; + this.manifestList = manifestList; this.residual = residual; this.referenceSnapshotId = referenceSnapshotId; } @@ -191,7 +193,7 @@ public List deletes() { @Override public CloseableIterable rows() { try (CloseableIterable manifests = - InternalData.read(FileFormat.AVRO, io.newInputFile(manifestListLocation)) + InternalData.read(FileFormat.AVRO, io.newInputFile(manifestList)) .setRootType(GenericManifestFile.class) .setCustomType( ManifestFile.PARTITION_SUMMARIES_ELEMENT_ID, GenericPartitionFieldSummary.class) @@ -209,7 +211,8 @@ public CloseableIterable rows() { return CloseableIterable.transform(rowIterable, projection::wrap); } catch (IOException e) { - throw new RuntimeIOException(e, "Cannot read manifest list file: %s", manifestListLocation); + throw new RuntimeIOException( + e, "Cannot read manifest list file: %s", manifestList.location()); } } @@ -218,7 +221,7 @@ public DataFile file() { if (lazyDataFile == null) { this.lazyDataFile = DataFiles.builder(PartitionSpec.unpartitioned()) - .withInputFile(io.newInputFile(manifestListLocation)) + .withInputFile(io.newInputFile(manifestList)) .withRecordCount(1) .withFormat(FileFormat.AVRO) .build(); @@ -271,8 +274,8 @@ Map specsById() { return specs; } - String manifestListLocation() { - return manifestListLocation; + ManifestListFile manifestList() { + return manifestList; } long referenceSnapshotId() { @@ -295,7 +298,8 @@ static StaticDataTask.Row manifestFileToRow( manifest.content() == ManifestContent.DELETES ? manifest.existingFilesCount() : 0, manifest.content() == ManifestContent.DELETES ? manifest.deletedFilesCount() : 0, ManifestsTable.partitionSummariesToRows(spec, manifest.partitions()), - referenceSnapshotId); + referenceSnapshotId, + manifest.keyMetadata() == null ? null : ByteBuffers.toByteArray(manifest.keyMetadata())); } private static class SnapshotEvaluator { diff --git a/core/src/main/java/org/apache/iceberg/AllManifestsTableTaskParser.java b/core/src/main/java/org/apache/iceberg/AllManifestsTableTaskParser.java index 24c28d95466c..e6539f2d714f 100644 --- a/core/src/main/java/org/apache/iceberg/AllManifestsTableTaskParser.java +++ b/core/src/main/java/org/apache/iceberg/AllManifestsTableTaskParser.java @@ -37,6 +37,7 @@ class AllManifestsTableTaskParser { private static final String SCHEMA = "schema"; private static final String SPECS = "partition-specs"; private static final String MANIFEST_LIST_LOCATION = "manifest-list-Location"; + private static final String MANIFEST_LIST_KEY_ID = "manifest-list-key-id"; private static final String RESIDUAL = "residual-filter"; private static final String REFERENCE_SNAPSHOT_ID = "reference-snapshot-id"; @@ -63,7 +64,10 @@ static void toJson(AllManifestsTable.ManifestListReadTask task, JsonGenerator ge generator.writeEndArray(); - generator.writeStringField(MANIFEST_LIST_LOCATION, task.manifestListLocation()); + generator.writeStringField(MANIFEST_LIST_LOCATION, task.manifestList().location()); + if (task.manifestList().encryptionKeyID() != null) { + generator.writeStringField(MANIFEST_LIST_KEY_ID, task.manifestList().encryptionKeyID()); + } generator.writeFieldName(RESIDUAL); ExpressionParser.toJson(task.residual(), generator); @@ -92,6 +96,7 @@ static AllManifestsTable.ManifestListReadTask fromJson(JsonNode jsonNode) { Map specsById = PartitionUtil.indexSpecs(specsBuilder.build()); String manifestListLocation = JsonUtil.getString(MANIFEST_LIST_LOCATION, jsonNode); + String manifestListKeyId = JsonUtil.getStringOrNull(MANIFEST_LIST_KEY_ID, jsonNode); Expression residualFilter = ExpressionParser.fromJson(JsonUtil.get(RESIDUAL, jsonNode)); long referenceSnapshotId = JsonUtil.getLong(REFERENCE_SNAPSHOT_ID, jsonNode); @@ -100,7 +105,7 @@ static AllManifestsTable.ManifestListReadTask fromJson(JsonNode jsonNode) { fileIO, schema, specsById, - manifestListLocation, + new BaseManifestListFile(manifestListLocation, manifestListKeyId), residualFilter, referenceSnapshotId); } diff --git a/core/src/test/java/org/apache/iceberg/TestAllManifestsTableTaskParser.java b/core/src/test/java/org/apache/iceberg/TestAllManifestsTableTaskParser.java index 2f057d7bd5a8..dde6bab6aa9f 100644 --- a/core/src/test/java/org/apache/iceberg/TestAllManifestsTableTaskParser.java +++ b/core/src/test/java/org/apache/iceberg/TestAllManifestsTableTaskParser.java @@ -91,7 +91,7 @@ private AllManifestsTable.ManifestListReadTask createTask() { fileIO, AllManifestsTable.MANIFEST_FILE_SCHEMA, specsById, - "/path/manifest-list-file.avro", + new BaseManifestListFile("/path/manifest-list-file.avro", "a"), Expressions.equal("id", 1), 1L); } @@ -121,10 +121,12 @@ private String taskJson() { + "{\"id\":11,\"name\":\"contains_nan\",\"required\":true,\"type\":\"boolean\"}," + "{\"id\":12,\"name\":\"lower_bound\",\"required\":false,\"type\":\"string\"}," + "{\"id\":13,\"name\":\"upper_bound\",\"required\":false,\"type\":\"string\"}]},\"element-required\":true}}," - + "{\"id\":18,\"name\":\"reference_snapshot_id\",\"required\":true,\"type\":\"long\"}]}," + + "{\"id\":18,\"name\":\"reference_snapshot_id\",\"required\":true,\"type\":\"long\"}," + + "{\"id\":19,\"name\":\"key_metadata\",\"required\":false,\"type\":\"binary\"}]}," + "\"partition-specs\":[{\"spec-id\":0,\"fields\":[{\"name\":\"data_bucket\"," + "\"transform\":\"bucket[16]\",\"source-id\":4,\"field-id\":1000}]}]," + "\"manifest-list-Location\":\"/path/manifest-list-file.avro\"," + + "\"manifest-list-key-id\":\"a\"," + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}," + "\"reference-snapshot-id\":1}"; } @@ -145,7 +147,9 @@ private void assertTaskEquals( .isEqualTo(expected.schema().asStruct()); assertThat(actual.specsById()).isEqualTo(expected.specsById()); - assertThat(actual.manifestListLocation()).isEqualTo(expected.manifestListLocation()); + assertThat(actual.manifestList().location()).isEqualTo(expected.manifestList().location()); + assertThat(actual.manifestList().encryptionKeyID()) + .isEqualTo(expected.manifestList().encryptionKeyID()); assertThat(actual.residual().toString()).isEqualTo(expected.residual().toString()); assertThat(actual.referenceSnapshotId()).isEqualTo(expected.referenceSnapshotId()); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index 65c605519d3d..6a977c0c0a2a 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -162,7 +162,8 @@ protected Dataset contentFileDS(Table table, Set snapshotIds) { "length", "0 as sequenceNumber", "partition_spec_id as partitionSpecId", - "added_snapshot_id as addedSnapshotId") + "added_snapshot_id as addedSnapshotId", + "key_metadata as keyMetadata") .dropDuplicates("path") .repartition(numShufflePartitions) // avoid adaptive execution combining tasks .as(ManifestFileBean.ENCODER); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java index fd4639897743..599e27b71c45 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java @@ -37,6 +37,7 @@ public class ManifestFileBean implements ManifestFile, Serializable { private Integer content = null; private Long sequenceNumber = null; private Long firstRowId = null; + private byte[] keyMetadata = null; public static ManifestFileBean fromManifest(ManifestFile manifest) { ManifestFileBean bean = new ManifestFileBean(); @@ -48,6 +49,7 @@ public static ManifestFileBean fromManifest(ManifestFile manifest) { bean.setContent(manifest.content().id()); bean.setSequenceNumber(manifest.sequenceNumber()); bean.setFirstRowId(manifest.firstRowId()); + bean.setKeyMetadata(manifest.keyMetadata() == null ? null : manifest.keyMetadata().array()); return bean; } @@ -104,6 +106,14 @@ public void setFirstRowId(Long firstRowId) { this.firstRowId = firstRowId; } + public byte[] getKeyMetadata() { + return keyMetadata; + } + + public void setKeyMetadata(byte[] keyMetadata) { + this.keyMetadata = keyMetadata; + } + @Override public String path() { return path; @@ -176,7 +186,7 @@ public List partitions() { @Override public ByteBuffer keyMetadata() { - return null; + return keyMetadata == null ? null : ByteBuffer.wrap(keyMetadata); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index a8e82d101fbf..f53859ef97d6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -161,7 +161,8 @@ protected Dataset contentFileDS(Table table, Set snapshotIds) { "length", "0 as sequenceNumber", "partition_spec_id as partitionSpecId", - "added_snapshot_id as addedSnapshotId") + "added_snapshot_id as addedSnapshotId", + "key_metadata as keyMetadata") .dropDuplicates("path") .repartition(numShufflePartitions) // avoid adaptive execution combining tasks .as(ManifestFileBean.ENCODER); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java index fd4639897743..599e27b71c45 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java @@ -37,6 +37,7 @@ public class ManifestFileBean implements ManifestFile, Serializable { private Integer content = null; private Long sequenceNumber = null; private Long firstRowId = null; + private byte[] keyMetadata = null; public static ManifestFileBean fromManifest(ManifestFile manifest) { ManifestFileBean bean = new ManifestFileBean(); @@ -48,6 +49,7 @@ public static ManifestFileBean fromManifest(ManifestFile manifest) { bean.setContent(manifest.content().id()); bean.setSequenceNumber(manifest.sequenceNumber()); bean.setFirstRowId(manifest.firstRowId()); + bean.setKeyMetadata(manifest.keyMetadata() == null ? null : manifest.keyMetadata().array()); return bean; } @@ -104,6 +106,14 @@ public void setFirstRowId(Long firstRowId) { this.firstRowId = firstRowId; } + public byte[] getKeyMetadata() { + return keyMetadata; + } + + public void setKeyMetadata(byte[] keyMetadata) { + this.keyMetadata = keyMetadata; + } + @Override public String path() { return path; @@ -176,7 +186,7 @@ public List partitions() { @Override public ByteBuffer keyMetadata() { - return null; + return keyMetadata == null ? null : ByteBuffer.wrap(keyMetadata); } @Override diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index a8e82d101fbf..f53859ef97d6 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -161,7 +161,8 @@ protected Dataset contentFileDS(Table table, Set snapshotIds) { "length", "0 as sequenceNumber", "partition_spec_id as partitionSpecId", - "added_snapshot_id as addedSnapshotId") + "added_snapshot_id as addedSnapshotId", + "key_metadata as keyMetadata") .dropDuplicates("path") .repartition(numShufflePartitions) // avoid adaptive execution combining tasks .as(ManifestFileBean.ENCODER); diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java index fd4639897743..599e27b71c45 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java @@ -37,6 +37,7 @@ public class ManifestFileBean implements ManifestFile, Serializable { private Integer content = null; private Long sequenceNumber = null; private Long firstRowId = null; + private byte[] keyMetadata = null; public static ManifestFileBean fromManifest(ManifestFile manifest) { ManifestFileBean bean = new ManifestFileBean(); @@ -48,6 +49,7 @@ public static ManifestFileBean fromManifest(ManifestFile manifest) { bean.setContent(manifest.content().id()); bean.setSequenceNumber(manifest.sequenceNumber()); bean.setFirstRowId(manifest.firstRowId()); + bean.setKeyMetadata(manifest.keyMetadata() == null ? null : manifest.keyMetadata().array()); return bean; } @@ -104,6 +106,14 @@ public void setFirstRowId(Long firstRowId) { this.firstRowId = firstRowId; } + public byte[] getKeyMetadata() { + return keyMetadata; + } + + public void setKeyMetadata(byte[] keyMetadata) { + this.keyMetadata = keyMetadata; + } + @Override public String path() { return path; @@ -176,7 +186,7 @@ public List partitions() { @Override public ByteBuffer keyMetadata() { - return null; + return keyMetadata == null ? null : ByteBuffer.wrap(keyMetadata); } @Override From 88d833b06bdb07101e458b8fdb7e60fe9e1a9c9a Mon Sep 17 00:00:00 2001 From: Ashok Date: Thu, 8 Jan 2026 07:00:01 +0530 Subject: [PATCH 162/201] Core: Handle NotFound exception for missing metadata file (#13143) --- .../apache/iceberg/rest/ErrorHandlers.java | 3 ++ .../apache/iceberg/catalog/CatalogTests.java | 38 +++++++++++++++++++ .../iceberg/inmemory/TestInMemoryCatalog.java | 30 +++++++++++++++ .../iceberg/rest/RESTCatalogAdapter.java | 2 + .../apache/iceberg/rest/TestRESTCatalog.java | 22 +++++++++++ 5 files changed, 95 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java index b1575035fcc0..543e548529dd 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java @@ -31,6 +31,7 @@ import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.exceptions.NotAuthorizedException; +import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.RESTException; import org.apache.iceberg.exceptions.ServiceFailureException; import org.apache.iceberg.exceptions.ServiceUnavailableException; @@ -124,6 +125,8 @@ public void accept(ErrorResponse error) { case 404: if (NoSuchNamespaceException.class.getSimpleName().equals(error.type())) { throw new NoSuchNamespaceException("%s", error.message()); + } else if (NotFoundException.class.getSimpleName().equals(error.type())) { + throw new NotFoundException("%s", error.message()); } else { throw new NoSuchTableException("%s", error.message()); } diff --git a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java index 5d20bc15a9c1..833b2fb0b46f 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java @@ -26,6 +26,11 @@ import java.io.IOException; import java.io.UncheckedIOException; +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -63,6 +68,7 @@ import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.io.CloseableIterable; @@ -81,6 +87,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.CharSequenceSet; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -960,6 +967,37 @@ public void testLoadMissingTable() { .hasMessageStartingWith("Table does not exist: ns.tbl"); } + @Test + public void testLoadTableWithMissingMetadataFile(@TempDir Path tempDir) throws IOException { + C catalog = catalog(); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(TBL.namespace()); + } + + catalog.buildTable(TBL, SCHEMA).create(); + assertThat(catalog.tableExists(TBL)).as("Table should exist").isTrue(); + + Table table = catalog.loadTable(TBL); + String metadataFileLocation = + ((HasTableOperations) table).operations().current().metadataFileLocation(); + Path renamedMetadataFile = tempDir.resolve("tmp.json"); + renamedMetadataFile.toFile().deleteOnExit(); + Files.writeString(renamedMetadataFile, "metadata"); + Path metadataFilePath = + metadataFileLocation.startsWith("file:") + ? Paths.get(URI.create(metadataFileLocation)) + : Paths.get(metadataFileLocation); + try { + Files.move(metadataFilePath, renamedMetadataFile, StandardCopyOption.REPLACE_EXISTING); + assertThatThrownBy(() -> catalog.loadTable(TBL)) + .isInstanceOf(NotFoundException.class) + .hasMessageContaining("Failed to open input stream for file: " + metadataFileLocation); + } finally { + Files.move(renamedMetadataFile, metadataFilePath, StandardCopyOption.REPLACE_EXISTING); + } + } + @Test public void testRenameTable() { C catalog = catalog(); diff --git a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryCatalog.java b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryCatalog.java index 705ff3dc8699..c2c683e7d882 100644 --- a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryCatalog.java +++ b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryCatalog.java @@ -18,11 +18,21 @@ */ package org.apache.iceberg.inmemory; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.nio.file.Path; import java.util.Map; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.Table; import org.apache.iceberg.catalog.CatalogTests; +import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestInMemoryCatalog extends CatalogTests { private InMemoryCatalog catalog; @@ -71,4 +81,24 @@ protected boolean requiresNamespaceCreate() { protected boolean supportsEmptyNamespace() { return true; } + + @Test + @Override + public void testLoadTableWithMissingMetadataFile(@TempDir Path tempDir) throws IOException { + + if (requiresNamespaceCreate()) { + catalog.createNamespace(TBL.namespace()); + } + + catalog.buildTable(TBL, SCHEMA).create(); + assertThat(catalog.tableExists(TBL)).as("Table should exist").isTrue(); + Table table = catalog.loadTable(TBL); + String metadataFileLocation = + ((HasTableOperations) table).operations().current().metadataFileLocation(); + table.io().deleteFile(metadataFileLocation); + assertThatThrownBy(() -> catalog.loadTable(TBL)) + .isInstanceOf(NotFoundException.class) + .hasMessage("No in-memory file found for location: " + metadataFileLocation); + table.io().newOutputFile(metadataFileLocation).create(); + } } diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java index 524b3e760ca6..e62937b6df6e 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -53,6 +53,7 @@ import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.exceptions.NotAuthorizedException; +import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.RESTException; import org.apache.iceberg.exceptions.UnprocessableEntityException; import org.apache.iceberg.exceptions.ValidationException; @@ -97,6 +98,7 @@ public class RESTCatalogAdapter extends BaseHTTPClient { .put(ForbiddenException.class, 403) .put(NoSuchNamespaceException.class, 404) .put(NoSuchTableException.class, 404) + .put(NotFoundException.class, 404) .put(NoSuchViewException.class, 404) .put(NoSuchIcebergTableException.class, 404) .put(UnsupportedOperationException.class, 406) diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index df4ba3214aea..d202680e5626 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -59,6 +59,7 @@ import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; +import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataUpdate; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -3453,6 +3454,27 @@ public T execute( return local; } + @Test + @Override + public void testLoadTableWithMissingMetadataFile(@TempDir Path tempDir) { + + if (requiresNamespaceCreate()) { + restCatalog.createNamespace(TBL.namespace()); + } + + restCatalog.buildTable(TBL, SCHEMA).create(); + assertThat(restCatalog.tableExists(TBL)).as("Table should exist").isTrue(); + + Table table = restCatalog.loadTable(TBL); + String metadataFileLocation = + ((HasTableOperations) table).operations().current().metadataFileLocation(); + table.io().deleteFile(metadataFileLocation); + + assertThatThrownBy(() -> restCatalog.loadTable(TBL)) + .isInstanceOf(NotFoundException.class) + .hasMessageContaining("No in-memory file found for location: " + metadataFileLocation); + } + private RESTCatalog catalog(RESTCatalogAdapter adapter) { RESTCatalog catalog = new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); From 99f14e7ebed010c4542cfdc98cd6f62e3922bc07 Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Wed, 7 Jan 2026 17:55:48 -0800 Subject: [PATCH 163/201] Spark 4.1: Initial support for MERGE INTO schema evolution (#14970) --- .../extensions/TestMergeSchemaEvolution.java | 227 ++++++++++++++++++ .../iceberg/spark/source/SparkTable.java | 1 + 2 files changed, 228 insertions(+) create mode 100644 spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeSchemaEvolution.java diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeSchemaEvolution.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeSchemaEvolution.java new file mode 100644 index 000000000000..c77b7e05768d --- /dev/null +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeSchemaEvolution.java @@ -0,0 +1,227 @@ +/* + * 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.extensions; + +import static org.apache.spark.sql.functions.col; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMergeSchemaEvolution extends SparkRowLevelOperationsTestBase { + + @BeforeAll + public static void setupSparkConf() { + spark.conf().set("spark.sql.shuffle.partitions", "4"); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS source"); + } + + @TestTemplate + public void testMergeWithSchemaEvolutionSourceHasMoreColumns() { + assumeThat(branch).as("Schema evolution does not work for branches currently").isNull(); + + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"software\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING, salary INT", + "{ \"id\": 1, \"dep\": \"hr\", \"salary\": 100 }\n" + + "{ \"id\": 3, \"dep\": \"finance\", \"salary\": 300 }"); + + sql( + "MERGE WITH SCHEMA EVOLUTION INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET * " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + commitTarget()); + + // The new 'salary' column should be added to the target table + ImmutableList expectedRows = + ImmutableList.of( + row(1, "hr", 100), // updated with salary + row(2, "software", null), // kept, salary is null + row(3, "finance", 300)); // new row with salary + assertEquals( + "Should have expected rows with new column", + expectedRows, + sql("SELECT id, dep, salary FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithSchemaEvolutionSourceHasFewerColumns() { + assumeThat(branch).as("Schema evolution does not work for branches currently").isNull(); + + createAndInitTable( + "id INT, dep STRING, salary INT", + "{ \"id\": 1, \"dep\": \"hr\", \"salary\": 100 }\n" + + "{ \"id\": 2, \"dep\": \"software\", \"salary\": 200 }"); + + createOrReplaceView( + "source", + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"hr-updated\" }\n" + "{ \"id\": 3, \"dep\": \"finance\" }"); + + sql( + "MERGE WITH SCHEMA EVOLUTION INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET * " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + commitTarget()); + + // Rows should have null for missing salary column from source + ImmutableList expectedRows = + ImmutableList.of( + row(1, "hr-updated", 100), // updated, salary retains value + row(2, "software", 200), // kept + row(3, "finance", null)); // new row, salary is null + assertEquals( + "Should have expected rows", + expectedRows, + sql("SELECT id, dep, salary FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithSchemaEvolutionUsingDataFrameApi() { + assumeThat(branch).as("Schema evolution does not work for branches currently").isNull(); + + createAndInitTable( + "id INT, dep STRING", + "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"software\" }"); + + createOrReplaceView( + "source", + "id INT, dep STRING, salary INT", + "{ \"id\": 1, \"dep\": \"hr-updated\", \"salary\": 100 }\n" + + "{ \"id\": 3, \"dep\": \"finance\", \"salary\": 300 }"); + + spark + .table("source") + .mergeInto(commitTarget(), col(commitTarget() + ".id").equalTo(col("source.id"))) + .whenMatched() + .updateAll() + .whenNotMatched() + .insertAll() + .withSchemaEvolution() + .merge(); + + // The new 'salary' column should be added + ImmutableList expectedRows = + ImmutableList.of( + row(1, "hr-updated", 100), // updated + row(2, "software", null), // kept + row(3, "finance", 300)); // new + assertEquals( + "Should have expected rows with schema evolution via DataFrame API", + expectedRows, + sql("SELECT id, dep, salary FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithSchemaEvolutionNestedStruct() { + assumeThat(branch).as("Schema evolution does not work for branches currently").isNull(); + + createAndInitTable( + "id INT, s STRUCT", + "{ \"id\": 1, \"s\": { \"c1\": 10, \"c2\": \"a\" } }\n" + + "{ \"id\": 2, \"s\": { \"c1\": 20, \"c2\": \"b\" } }"); + + createOrReplaceView( + "source", + "id INT, s STRUCT", + "{ \"id\": 1, \"s\": { \"c1\": 100, \"c2\": \"aa\", \"c3\": 1000 } }\n" + + "{ \"id\": 3, \"s\": { \"c1\": 300, \"c2\": \"cc\", \"c3\": 3000 } }"); + + sql( + "MERGE WITH SCHEMA EVOLUTION INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET * " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + commitTarget()); + + // The nested struct should have the new c3 field + ImmutableList expectedRows = + ImmutableList.of( + row(1, row(100, "aa", 1000)), // updated with nested field + row(2, row(20, "b", null)), // kept, c3 is null + row(3, row(300, "cc", 3000))); // new + assertEquals( + "Should have expected rows with nested struct evolution", + expectedRows, + sql("SELECT id, s FROM %s ORDER BY id", selectTarget())); + } + + @TestTemplate + public void testMergeWithSchemaEvolutionTypeWidening() { + assumeThat(branch).as("Schema evolution does not work for branches currently").isNull(); + + // Target has INT column + createAndInitTable( + "id INT, value INT", "{ \"id\": 1, \"value\": 100 }\n" + "{ \"id\": 2, \"value\": 200 }"); + + // Source has LONG column - should widen INT to LONG + createOrReplaceView( + "source", + "id INT, value LONG", + "{ \"id\": 1, \"value\": 1000000000000 }\n" + "{ \"id\": 3, \"value\": 3000000000000 }"); + + sql( + "MERGE WITH SCHEMA EVOLUTION INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED THEN " + + " UPDATE SET * " + + "WHEN NOT MATCHED THEN " + + " INSERT *", + commitTarget()); + + // The 'value' column should be widened from INT to LONG + ImmutableList expectedRows = + ImmutableList.of( + row(1, 1000000000000L), // updated with long value + row(2, 200L), // kept, value promoted to long + row(3, 3000000000000L)); // new row with long value + assertEquals( + "Should have expected rows with type widening", + expectedRows, + sql("SELECT id, value FROM %s ORDER BY id", selectTarget())); + } + + @Override + protected Map extraTableProperties() { + return Map.of(); + } +} 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 1ee9e9b08074..f3b60b902ab5 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 @@ -106,6 +106,7 @@ public class SparkTable "identifier-fields"); private static final Set CAPABILITIES = ImmutableSet.of( + TableCapability.AUTOMATIC_SCHEMA_EVOLUTION, TableCapability.BATCH_READ, TableCapability.BATCH_WRITE, TableCapability.MICRO_BATCH_READ, From daa3bb257bfc17ae7988dbff0a5379a16c1abcc9 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Thu, 8 Jan 2026 00:56:26 -0500 Subject: [PATCH 164/201] manually update spark 3.4 (#14993) --- site/docs/multi-engine-support.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/docs/multi-engine-support.md b/site/docs/multi-engine-support.md index 6ff49d3ea7d3..d63964614197 100644 --- a/site/docs/multi-engine-support.md +++ b/site/docs/multi-engine-support.md @@ -72,7 +72,7 @@ Each engine version undergoes the following lifecycle stages: | 3.1 | End of Life | 0.12.0 | 1.3.1 | [iceberg-spark-runtime-3.1_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.1_2.12/1.3.1/iceberg-spark-runtime-3.1_2.12-1.3.1.jar) [1] | | 3.2 | End of Life | 0.13.0 | 1.4.3 | [iceberg-spark-runtime-3.2_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.2_2.12/1.4.3/iceberg-spark-runtime-3.2_2.12-1.4.3.jar), [iceberg-spark-runtime-3.2_2.13](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.2_2.13/1.4.3/iceberg-spark-runtime-3.2_2.13-1.4.3.jar) | | 3.3 | End of Life | 0.14.0 | 1.8.1 | [iceberg-spark-runtime-3.3_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.3_2.12/1.8.1/iceberg-spark-runtime-3.3_2.12-1.8.1.jar), [iceberg-spark-runtime-3.3_2.13](https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-spark-runtime-3.3_2.13/1.8.1/iceberg-spark-runtime-3.3_2.13-1.8.1.jar) | -| 3.4 | Deprecated | 1.3.0 | 1.10.0 | [iceberg-spark-runtime-3.4_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.4_2.12/1.10.0/iceberg-spark-runtime-3.4_2.12-1.10.0.jar), [iceberg-spark-runtime-3.4_2.13](https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-spark-runtime-3.4_2.13/1.10.0/iceberg-spark-runtime-3.4_2.13-1.10.0.jar) | +| 3.4 | Deprecated | 1.3.0 | 1.10.1 | [iceberg-spark-runtime-3.4_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.4_2.12/1.10.1/iceberg-spark-runtime-3.4_2.12-1.10.1.jar), [iceberg-spark-runtime-3.4_2.13](https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-spark-runtime-3.4_2.13/1.10.1/iceberg-spark-runtime-3.4_2.13-1.10.1.jar) | | 3.5 | Maintained | 1.4.0 | {{ icebergVersion }} | [iceberg-spark-runtime-3.5_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.5_2.12/{{ icebergVersion }}/iceberg-spark-runtime-3.5_2.12-{{ icebergVersion }}.jar), [iceberg-spark-runtime-3.5_2.13](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.5_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.5_2.13-{{ icebergVersion }}.jar) | | 4.0 | Maintained | 1.10.0 | {{ icebergVersion }} | [iceberg-spark-runtime-4.0_2.13](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-4.0_2.13/{{ icebergVersion }}/iceberg-spark-runtime-4.0_2.13-{{ icebergVersion }}.jar) | From 01e324022f115c1457367872aac9a2d35ef0fdbf Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Thu, 8 Jan 2026 01:20:51 -0500 Subject: [PATCH 165/201] site infra: when running `make serve`, add a tip on using `make serve-dev` instead --- site/dev/serve.sh | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/site/dev/serve.sh b/site/dev/serve.sh index 3e670e72b563..938c8f63917c 100755 --- a/site/dev/serve.sh +++ b/site/dev/serve.sh @@ -19,6 +19,17 @@ source dev/common.sh set -e +echo "" +echo "**********************************************" +echo "** **" +echo "** 💡 TIP: Use 'make serve-dev' instead! **" +echo "** **" +echo "** It's FASTER for local development **" +echo "** (only builds nightly and latest) **" +echo "** **" +echo "**********************************************" +echo "" + ./dev/setup_env.sh ./dev/lint.sh From cde5b9f984344d134fc907b2ff3883499d7e54c3 Mon Sep 17 00:00:00 2001 From: Robin Moffatt Date: Thu, 8 Jan 2026 15:05:54 +0000 Subject: [PATCH 166/201] Kafka Connect: Fix CVE-2025-55163 in grpc-netty-shaded (#14985) --- gcp-bundle/LICENSE | 2 +- gcp-bundle/NOTICE | 2 +- kafka-connect/build.gradle | 1 + kafka-connect/kafka-connect-runtime/hive/LICENSE | 2 +- kafka-connect/kafka-connect-runtime/hive/NOTICE | 2 +- kafka-connect/kafka-connect-runtime/main/LICENSE | 2 +- kafka-connect/kafka-connect-runtime/main/NOTICE | 2 +- 7 files changed, 7 insertions(+), 6 deletions(-) diff --git a/gcp-bundle/LICENSE b/gcp-bundle/LICENSE index de9a4007e68f..bf084064669c 100644 --- a/gcp-bundle/LICENSE +++ b/gcp-bundle/LICENSE @@ -906,7 +906,7 @@ License: Apache 2.0 - https://opensource.org/licenses/Apache-2.0 -------------------------------------------------------------------------------- -Group: io.grpc Name: grpc-netty-shaded Version: 1.71.0 +Group: io.grpc Name: grpc-netty-shaded Version: 1.76.2 Project URL: https://github.com/grpc/grpc-java License: Apache 2.0 - https://opensource.org/licenses/Apache-2.0 diff --git a/gcp-bundle/NOTICE b/gcp-bundle/NOTICE index 5ea9500ec678..465fc754f059 100644 --- a/gcp-bundle/NOTICE +++ b/gcp-bundle/NOTICE @@ -29,7 +29,7 @@ from the source code management (SCM) system project uses. -------------------------------------------------------------------------------- -NOTICE for Group: io.grpc Name: grpc-netty-shaded Version: 1.71.0 +NOTICE for Group: io.grpc Name: grpc-netty-shaded Version: 1.76.2 | The Netty Project | ================= diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 44d095e1c6fa..5f145312695a 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -76,6 +76,7 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { force 'org.apache.hadoop.thirdparty:hadoop-shaded-guava:1.5.0' force 'com.fasterxml.woodstox:woodstox-core:6.7.0' force 'commons-beanutils:commons-beanutils:1.11.0' + force 'io.grpc:grpc-netty-shaded:1.76.2' } } } diff --git a/kafka-connect/kafka-connect-runtime/hive/LICENSE b/kafka-connect/kafka-connect-runtime/hive/LICENSE index 1f18593b0fad..caccc7ff83a7 100644 --- a/kafka-connect/kafka-connect-runtime/hive/LICENSE +++ b/kafka-connect/kafka-connect-runtime/hive/LICENSE @@ -783,7 +783,7 @@ License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 -------------------------------------------------------------------------------- -Group: io.grpc Name: grpc-netty-shaded Version: 1.71.0 +Group: io.grpc Name: grpc-netty-shaded Version: 1.76.2 Project URL (from POM): https://github.com/grpc/grpc-java License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 diff --git a/kafka-connect/kafka-connect-runtime/hive/NOTICE b/kafka-connect/kafka-connect-runtime/hive/NOTICE index bdcaa9a7b64b..4e77d54e2219 100644 --- a/kafka-connect/kafka-connect-runtime/hive/NOTICE +++ b/kafka-connect/kafka-connect-runtime/hive/NOTICE @@ -30,7 +30,7 @@ This binary artifact contains code from the following projects: -------------------------------------------------------------------------------- -Group: io.grpc Name: grpc-netty-shaded Version: 1.71.0 +Group: io.grpc Name: grpc-netty-shaded Version: 1.76.2 Notice: | The Netty Project diff --git a/kafka-connect/kafka-connect-runtime/main/LICENSE b/kafka-connect/kafka-connect-runtime/main/LICENSE index c577e143819d..1b78a369c2fb 100644 --- a/kafka-connect/kafka-connect-runtime/main/LICENSE +++ b/kafka-connect/kafka-connect-runtime/main/LICENSE @@ -722,7 +722,7 @@ License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 -------------------------------------------------------------------------------- -Group: io.grpc Name: grpc-netty-shaded Version: 1.71.0 +Group: io.grpc Name: grpc-netty-shaded Version: 1.76.2 Project URL (from POM): https://github.com/grpc/grpc-java License (from POM): Apache 2.0 - https://opensource.org/licenses/Apache-2.0 diff --git a/kafka-connect/kafka-connect-runtime/main/NOTICE b/kafka-connect/kafka-connect-runtime/main/NOTICE index f2be4788082c..257600fa103c 100644 --- a/kafka-connect/kafka-connect-runtime/main/NOTICE +++ b/kafka-connect/kafka-connect-runtime/main/NOTICE @@ -30,7 +30,7 @@ This binary artifact contains code from the following projects: -------------------------------------------------------------------------------- -Group: io.grpc Name: grpc-netty-shaded Version: 1.71.0 +Group: io.grpc Name: grpc-netty-shaded Version: 1.76.2 Notice: | The Netty Project From 615b5a097cf27a65fec2a9f613150ab83ea8bc7a Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Thu, 8 Jan 2026 09:45:02 -0800 Subject: [PATCH 167/201] Core: Unlink table metadata's last-updated timestamp from snapshot timestamp (#14504) --- core/src/main/java/org/apache/iceberg/TableMetadata.java | 9 ++++----- .../iceberg/flink/source/TestFlinkMetaDataTable.java | 7 ++++--- .../iceberg/flink/source/TestFlinkMetaDataTable.java | 7 ++++--- .../iceberg/flink/source/TestFlinkMetaDataTable.java | 7 ++++--- .../iceberg/spark/extensions/TestMetadataTables.java | 5 ++--- .../spark/source/TestIcebergSourceTablesBase.java | 5 +++++ .../iceberg/spark/extensions/TestMetadataTables.java | 5 ++--- .../spark/source/TestIcebergSourceTablesBase.java | 5 +++++ .../iceberg/spark/extensions/TestMetadataTables.java | 5 ++--- .../spark/source/TestIcebergSourceTablesBase.java | 5 +++++ 10 files changed, 37 insertions(+), 23 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 7dac5d401a80..5aa76d5680fa 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -1258,7 +1258,6 @@ public Builder addSnapshot(Snapshot snapshot) { snapshot.sequenceNumber(), lastSequenceNumber); - this.lastUpdatedMillis = snapshot.timestampMillis(); this.lastSequenceNumber = snapshot.sequenceNumber(); snapshots.add(snapshot); snapshotsById.put(snapshot.snapshotId(), snapshot); @@ -1316,9 +1315,6 @@ public Builder setRef(String name, SnapshotRef ref) { Snapshot snapshot = snapshotsById.get(snapshotId); ValidationException.check( snapshot != null, "Cannot set %s to unknown snapshot: %s", name, snapshotId); - if (isAddedSnapshot(snapshotId)) { - this.lastUpdatedMillis = snapshot.timestampMillis(); - } if (SnapshotRef.MAIN_BRANCH.equals(name)) { this.currentSnapshotId = ref.snapshotId(); @@ -1326,7 +1322,10 @@ public Builder setRef(String name, SnapshotRef ref) { this.lastUpdatedMillis = System.currentTimeMillis(); } - snapshotLog.add(new SnapshotLogEntry(lastUpdatedMillis, ref.snapshotId())); + // rollback to an existing snapshot will use current timestamp as the time of the change + long timeOfChange = + isAddedSnapshot(snapshotId) ? snapshot.timestampMillis() : this.lastUpdatedMillis; + snapshotLog.add(new SnapshotLogEntry(timeOfChange, ref.snapshotId())); } refs.put(name, ref); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java index 8352924d042a..1903cb125332 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java @@ -625,8 +625,9 @@ public void testMetadataLogEntries() { assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(parentSnapshot.snapshotId()); metadataLog = metadataLogs.get(2); - assertThat(metadataLog.getField("timestamp")) - .isEqualTo(Instant.ofEpochMilli(currentSnapshot.timestampMillis())); + assertThat((Instant) metadataLog.getFieldAs("timestamp")) + .isAfterOrEqualTo(Instant.ofEpochMilli(currentSnapshot.timestampMillis())) + .isEqualTo(Instant.ofEpochMilli(tableMetadata.lastUpdatedMillis())); assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(currentSnapshot.snapshotId()); assertThat(metadataLog.getField("latest_schema_id")).isEqualTo(currentSnapshot.schemaId()); @@ -640,7 +641,7 @@ public void testMetadataLogEntries() { TABLE_NAME, currentSnapshotId); assertThat(metadataLogWithFilters).hasSize(1); metadataLog = metadataLogWithFilters.get(0); - assertThat(Instant.ofEpochMilli(tableMetadata.currentSnapshot().timestampMillis())) + assertThat(Instant.ofEpochMilli(tableMetadata.lastUpdatedMillis())) .isEqualTo(metadataLog.getField("timestamp")); assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java index 8352924d042a..1903cb125332 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java @@ -625,8 +625,9 @@ public void testMetadataLogEntries() { assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(parentSnapshot.snapshotId()); metadataLog = metadataLogs.get(2); - assertThat(metadataLog.getField("timestamp")) - .isEqualTo(Instant.ofEpochMilli(currentSnapshot.timestampMillis())); + assertThat((Instant) metadataLog.getFieldAs("timestamp")) + .isAfterOrEqualTo(Instant.ofEpochMilli(currentSnapshot.timestampMillis())) + .isEqualTo(Instant.ofEpochMilli(tableMetadata.lastUpdatedMillis())); assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(currentSnapshot.snapshotId()); assertThat(metadataLog.getField("latest_schema_id")).isEqualTo(currentSnapshot.schemaId()); @@ -640,7 +641,7 @@ public void testMetadataLogEntries() { TABLE_NAME, currentSnapshotId); assertThat(metadataLogWithFilters).hasSize(1); metadataLog = metadataLogWithFilters.get(0); - assertThat(Instant.ofEpochMilli(tableMetadata.currentSnapshot().timestampMillis())) + assertThat(Instant.ofEpochMilli(tableMetadata.lastUpdatedMillis())) .isEqualTo(metadataLog.getField("timestamp")); assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java index 8352924d042a..1903cb125332 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java @@ -625,8 +625,9 @@ public void testMetadataLogEntries() { assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(parentSnapshot.snapshotId()); metadataLog = metadataLogs.get(2); - assertThat(metadataLog.getField("timestamp")) - .isEqualTo(Instant.ofEpochMilli(currentSnapshot.timestampMillis())); + assertThat((Instant) metadataLog.getFieldAs("timestamp")) + .isAfterOrEqualTo(Instant.ofEpochMilli(currentSnapshot.timestampMillis())) + .isEqualTo(Instant.ofEpochMilli(tableMetadata.lastUpdatedMillis())); assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(currentSnapshot.snapshotId()); assertThat(metadataLog.getField("latest_schema_id")).isEqualTo(currentSnapshot.schemaId()); @@ -640,7 +641,7 @@ public void testMetadataLogEntries() { TABLE_NAME, currentSnapshotId); assertThat(metadataLogWithFilters).hasSize(1); metadataLog = metadataLogWithFilters.get(0); - assertThat(Instant.ofEpochMilli(tableMetadata.currentSnapshot().timestampMillis())) + assertThat(Instant.ofEpochMilli(tableMetadata.lastUpdatedMillis())) .isEqualTo(metadataLog.getField("timestamp")); assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java index 82a065073d94..5153a1fcad5b 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java @@ -578,7 +578,7 @@ public void testMetadataLogEntries() throws Exception { parentSnapshot.schemaId(), parentSnapshot.sequenceNumber()), row( - DateTimeUtils.toJavaTimestamp(currentSnapshot.timestampMillis() * 1000), + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), tableMetadata.metadataFileLocation(), currentSnapshot.snapshotId(), currentSnapshot.schemaId(), @@ -597,8 +597,7 @@ public void testMetadataLogEntries() throws Exception { "Result should match the latest snapshot entry", ImmutableList.of( row( - DateTimeUtils.toJavaTimestamp( - tableMetadata.currentSnapshot().timestampMillis() * 1000), + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), tableMetadata.metadataFileLocation(), tableMetadata.currentSnapshot().snapshotId(), tableMetadata.currentSnapshot().schemaId(), diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 09219963f7a3..53c9ac6b2257 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -46,6 +46,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.Files; +import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -820,6 +821,10 @@ public void testHistoryTable() { // rollback the table state to the first snapshot table.manageSnapshots().rollbackTo(firstSnapshotId).commit(); long rollbackTimestamp = Iterables.getLast(table.history()).timestampMillis(); + assertThat(rollbackTimestamp) + .as("Rollback history timestamp should be greater than first snapshot timestamp") + .isEqualTo(((HasTableOperations) table).operations().current().lastUpdatedMillis()) + .isGreaterThan(firstSnapshotTimestamp); inputDf .select("id", "data") diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java index ac528d1c470e..be114782cbf4 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java @@ -578,7 +578,7 @@ public void testMetadataLogEntries() throws Exception { parentSnapshot.schemaId(), parentSnapshot.sequenceNumber()), row( - DateTimeUtils.toJavaTimestamp(currentSnapshot.timestampMillis() * 1000), + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), tableMetadata.metadataFileLocation(), currentSnapshot.snapshotId(), currentSnapshot.schemaId(), @@ -597,8 +597,7 @@ public void testMetadataLogEntries() throws Exception { "Result should match the latest snapshot entry", ImmutableList.of( row( - DateTimeUtils.toJavaTimestamp( - tableMetadata.currentSnapshot().timestampMillis() * 1000), + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), tableMetadata.metadataFileLocation(), tableMetadata.currentSnapshot().snapshotId(), tableMetadata.currentSnapshot().schemaId(), diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 63994726cafb..67647925e59d 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -46,6 +46,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.Files; +import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -821,6 +822,10 @@ public void testHistoryTable() { // rollback the table state to the first snapshot table.manageSnapshots().rollbackTo(firstSnapshotId).commit(); long rollbackTimestamp = Iterables.getLast(table.history()).timestampMillis(); + assertThat(rollbackTimestamp) + .as("Rollback history timestamp should be greater than first snapshot timestamp") + .isEqualTo(((HasTableOperations) table).operations().current().lastUpdatedMillis()) + .isGreaterThan(firstSnapshotTimestamp); inputDf .select("id", "data") diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java index ac528d1c470e..be114782cbf4 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java @@ -578,7 +578,7 @@ public void testMetadataLogEntries() throws Exception { parentSnapshot.schemaId(), parentSnapshot.sequenceNumber()), row( - DateTimeUtils.toJavaTimestamp(currentSnapshot.timestampMillis() * 1000), + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), tableMetadata.metadataFileLocation(), currentSnapshot.snapshotId(), currentSnapshot.schemaId(), @@ -597,8 +597,7 @@ public void testMetadataLogEntries() throws Exception { "Result should match the latest snapshot entry", ImmutableList.of( row( - DateTimeUtils.toJavaTimestamp( - tableMetadata.currentSnapshot().timestampMillis() * 1000), + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), tableMetadata.metadataFileLocation(), tableMetadata.currentSnapshot().snapshotId(), tableMetadata.currentSnapshot().schemaId(), diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java index 63994726cafb..67647925e59d 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSourceTablesBase.java @@ -46,6 +46,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.Files; +import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -821,6 +822,10 @@ public void testHistoryTable() { // rollback the table state to the first snapshot table.manageSnapshots().rollbackTo(firstSnapshotId).commit(); long rollbackTimestamp = Iterables.getLast(table.history()).timestampMillis(); + assertThat(rollbackTimestamp) + .as("Rollback history timestamp should be greater than first snapshot timestamp") + .isEqualTo(((HasTableOperations) table).operations().current().lastUpdatedMillis()) + .isGreaterThan(firstSnapshotTimestamp); inputDf .select("id", "data") From 0094ccc0aeec33638f34ff5c4074b4cfe4bdf58c Mon Sep 17 00:00:00 2001 From: pvary Date: Thu, 8 Jan 2026 22:52:23 +0100 Subject: [PATCH 168/201] Use SnapshotRef.MAIN_BRANCH instead of the 'main' string (#14999) --- .../org/apache/iceberg/TestDeleteFiles.java | 5 +- .../org/apache/iceberg/TestMergeAppend.java | 5 +- .../org/apache/iceberg/TestOverwrite.java | 5 +- .../iceberg/TestOverwriteWithValidation.java | 5 +- .../apache/iceberg/TestRemoveSnapshots.java | 4 +- .../apache/iceberg/TestReplacePartitions.java | 3 +- .../org/apache/iceberg/TestRewriteFiles.java | 3 +- .../java/org/apache/iceberg/TestRowDelta.java | 7 +- .../iceberg/TestRowLineageAssignment.java | 4 +- .../org/apache/iceberg/TestTableMetadata.java | 16 +- .../org/apache/iceberg/TestTransaction.java | 12 +- ...RecordSerializerDeserializerBenchmark.java | 3 +- .../flink/sink/dynamic/WriteTarget.java | 3 +- .../sink/TestFlinkIcebergSinkBranch.java | 8 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 2 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 18 +- .../flink/sink/TestIcebergFilesCommitter.java | 7 +- .../flink/sink/TestIcebergSinkBranch.java | 4 +- .../iceberg/flink/sink/TestIcebergSinkV2.java | 2 +- .../sink/dynamic/TestDynamicIcebergSink.java | 248 +++++++++++++----- .../dynamic/TestDynamicIcebergSinkPerf.java | 5 +- .../sink/dynamic/TestHashKeyGenerator.java | 3 +- .../sink/dynamic/TestTableMetadataCache.java | 7 +- .../flink/sink/dynamic/TestTableUpdater.java | 44 +++- .../flink/source/TestFlinkMetaDataTable.java | 6 +- ...RecordSerializerDeserializerBenchmark.java | 3 +- .../flink/sink/dynamic/WriteTarget.java | 3 +- .../sink/TestFlinkIcebergSinkBranch.java | 8 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 2 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 18 +- .../flink/sink/TestIcebergFilesCommitter.java | 7 +- .../flink/sink/TestIcebergSinkBranch.java | 4 +- .../iceberg/flink/sink/TestIcebergSinkV2.java | 2 +- .../sink/dynamic/TestDynamicIcebergSink.java | 248 +++++++++++++----- .../dynamic/TestDynamicIcebergSinkPerf.java | 5 +- .../sink/dynamic/TestHashKeyGenerator.java | 3 +- .../sink/dynamic/TestTableMetadataCache.java | 7 +- .../flink/sink/dynamic/TestTableUpdater.java | 44 +++- .../flink/source/TestFlinkMetaDataTable.java | 6 +- ...RecordSerializerDeserializerBenchmark.java | 3 +- .../flink/sink/dynamic/WriteTarget.java | 3 +- .../sink/TestFlinkIcebergSinkBranch.java | 8 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 2 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 18 +- .../flink/sink/TestIcebergFilesCommitter.java | 7 +- .../flink/sink/TestIcebergSinkBranch.java | 4 +- .../iceberg/flink/sink/TestIcebergSinkV2.java | 2 +- .../sink/dynamic/TestDynamicIcebergSink.java | 248 +++++++++++++----- .../dynamic/TestDynamicIcebergSinkPerf.java | 5 +- .../sink/dynamic/TestHashKeyGenerator.java | 3 +- .../sink/dynamic/TestTableMetadataCache.java | 7 +- .../flink/sink/dynamic/TestTableUpdater.java | 44 +++- .../flink/source/TestFlinkMetaDataTable.java | 6 +- .../iceberg/nessie/BaseTestIceberg.java | 3 +- .../iceberg/nessie/TestBranchVisibility.java | 9 +- .../nessie/TestCustomNessieClient.java | 3 +- .../iceberg/nessie/TestNessieCatalog.java | 3 +- .../nessie/TestNessieIcebergClient.java | 14 +- .../iceberg/nessie/TestNessieTable.java | 11 +- .../iceberg/nessie/TestNessieViewCatalog.java | 3 +- .../spark/extensions/TestMetadataTables.java | 7 +- .../spark/source/TestSparkDataWrite.java | 2 +- .../spark/extensions/TestMetadataTables.java | 7 +- .../spark/source/TestSparkDataWrite.java | 2 +- .../spark/extensions/TestMetadataTables.java | 7 +- .../spark/source/TestSparkDataWrite.java | 2 +- .../spark/extensions/TestMetadataTables.java | 7 +- .../spark/source/TestSparkDataWrite.java | 2 +- 68 files changed, 880 insertions(+), 351 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java index cad294f97a28..ea0988155b1d 100644 --- a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java @@ -84,7 +84,10 @@ public class TestDeleteFiles extends TestBase { @Parameters(name = "formatVersion = {0}, branch = {1}") protected static List parameters() { return TestHelpers.ALL_VERSIONS.stream() - .flatMap(v -> Stream.of(new Object[] {v, "main"}, new Object[] {v, "testBranch"})) + .flatMap( + v -> + Stream.of( + new Object[] {v, SnapshotRef.MAIN_BRANCH}, new Object[] {v, "testBranch"})) .collect(Collectors.toList()); } diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java index a2c5ba0adbf2..0759b0f13ad7 100644 --- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -52,7 +52,10 @@ public class TestMergeAppend extends TestBase { @Parameters(name = "formatVersion = {0}, branch = {1}") protected static List parameters() { return TestHelpers.ALL_VERSIONS.stream() - .flatMap(v -> Stream.of(new Object[] {v, "main"}, new Object[] {v, "testBranch"})) + .flatMap( + v -> + Stream.of( + new Object[] {v, SnapshotRef.MAIN_BRANCH}, new Object[] {v, "testBranch"})) .collect(Collectors.toList()); } diff --git a/core/src/test/java/org/apache/iceberg/TestOverwrite.java b/core/src/test/java/org/apache/iceberg/TestOverwrite.java index c853cf69ea08..efd4935a4b9f 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwrite.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwrite.java @@ -111,7 +111,10 @@ public class TestOverwrite extends TestBase { @Parameters(name = "formatVersion = {0}, branch = {1}") protected static List parameters() { return TestHelpers.ALL_VERSIONS.stream() - .flatMap(v -> Stream.of(new Object[] {v, "main"}, new Object[] {v, "testBranch"})) + .flatMap( + v -> + Stream.of( + new Object[] {v, SnapshotRef.MAIN_BRANCH}, new Object[] {v, "testBranch"})) .collect(Collectors.toList()); } diff --git a/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java b/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java index f984002f4601..f3070c0e2ccc 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java @@ -186,7 +186,10 @@ public class TestOverwriteWithValidation extends TestBase { @Parameters(name = "formatVersion = {0}, branch = {1}") protected static List parameters() { return TestHelpers.ALL_VERSIONS.stream() - .flatMap(v -> Stream.of(new Object[] {v, "main"}, new Object[] {v, "testBranch"})) + .flatMap( + v -> + Stream.of( + new Object[] {v, SnapshotRef.MAIN_BRANCH}, new Object[] {v, "testBranch"})) .collect(Collectors.toList()); } diff --git a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java index c250ff82e4f7..fa2ccecca07e 100644 --- a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java +++ b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java @@ -1388,7 +1388,7 @@ public void testUnreferencedSnapshotParentOfTag() { table .manageSnapshots() .createTag("tag", table.currentSnapshot().snapshotId()) - .replaceBranch("main", initialSnapshotId) + .replaceBranch(SnapshotRef.MAIN_BRANCH, initialSnapshotId) .commit(); removeSnapshots(table) @@ -1426,7 +1426,7 @@ public void testSnapshotParentOfBranchNotUnreferenced() { .manageSnapshots() .createBranch("branch", table.currentSnapshot().snapshotId()) .setMaxSnapshotAgeMs("branch", Long.MAX_VALUE) - .replaceBranch("main", initialSnapshotId) + .replaceBranch(SnapshotRef.MAIN_BRANCH, initialSnapshotId) .commit(); removeSnapshots(table) diff --git a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java index abc27cddd610..d246241287b5 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java @@ -125,7 +125,8 @@ public class TestReplacePartitions extends TestBase { @Parameters(name = "formatVersion = {0}, branch = {1}") protected static List parameters() { return TestHelpers.ALL_VERSIONS.stream() - .flatMap(v -> Stream.of(new Object[] {v, "main"}, new Object[] {v, "branch"})) + .flatMap( + v -> Stream.of(new Object[] {v, SnapshotRef.MAIN_BRANCH}, new Object[] {v, "branch"})) .collect(Collectors.toList()); } diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java index 64b2b4d52c46..16c4a0c20ec6 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java @@ -48,7 +48,8 @@ public class TestRewriteFiles extends TestBase { @Parameters(name = "formatVersion = {0}, branch = {1}") protected static List parameters() { return TestHelpers.ALL_VERSIONS.stream() - .flatMap(v -> Stream.of(new Object[] {v, "main"}, new Object[] {v, "branch"})) + .flatMap( + v -> Stream.of(new Object[] {v, SnapshotRef.MAIN_BRANCH}, new Object[] {v, "branch"})) .collect(Collectors.toList()); } diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 397e1ac56dc4..4eff3a400f72 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -60,7 +60,10 @@ public class TestRowDelta extends TestBase { @Parameters(name = "formatVersion = {0}, branch = {1}") protected static List parameters() { return TestHelpers.V2_AND_ABOVE.stream() - .flatMap(v -> Stream.of(new Object[] {v, "main"}, new Object[] {v, "testBranch"})) + .flatMap( + v -> + Stream.of( + new Object[] {v, SnapshotRef.MAIN_BRANCH}, new Object[] {v, "testBranch"})) .collect(Collectors.toList()); } @@ -1761,7 +1764,7 @@ public void testConcurrentDeletesRewriteSameRemoveRows() { public void testConcurrentManifestRewriteWithRemoveRowsRemoval() throws IOException { assumeThat(formatVersion).isEqualTo(2); // Manifest rewrite isn't supported on branches currently - assumeThat(branch).isEqualTo("main"); + assumeThat(branch).isEqualTo(SnapshotRef.MAIN_BRANCH); DataFile dataFile = newDataFile("data_bucket=0"); DeleteFile deleteFile = newDeleteFile(dataFile.specId(), "data_bucket=0"); diff --git a/core/src/test/java/org/apache/iceberg/TestRowLineageAssignment.java b/core/src/test/java/org/apache/iceberg/TestRowLineageAssignment.java index 5f027ccc88ca..870622bc983d 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowLineageAssignment.java +++ b/core/src/test/java/org/apache/iceberg/TestRowLineageAssignment.java @@ -244,7 +244,7 @@ public void testFastForwardPreservesRowIds() { long branchSnapshot = table.snapshot("branch").snapshotId(); // fast-forward main to the branch - table.manageSnapshots().fastForwardBranch("main", "branch").commit(); + table.manageSnapshots().fastForwardBranch(SnapshotRef.MAIN_BRANCH, "branch").commit(); // branch data manifests: [added(FILE_C)], [added(FILE_B)], [added(FILE_A)] assertThat(table.operations().current().nextRowId()) @@ -252,7 +252,7 @@ public void testFastForwardPreservesRowIds() { assertThat(table.currentSnapshot().snapshotId()).isEqualTo(branchSnapshot); // validate that the branches have the same first_row_id assignments - for (String branch : List.of("main", "branch")) { + for (String branch : List.of(SnapshotRef.MAIN_BRANCH, "branch")) { checkManifestListAssignment( table.io().newInputFile(table.snapshot(branch).manifestListLocation()), startingNextRowId + FILE_B.recordCount(), diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index ed7b630f6043..bddb87859377 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -153,9 +153,12 @@ public void testJsonConversion() throws Exception { Map refs = ImmutableMap.of( - "main", SnapshotRef.branchBuilder(currentSnapshotId).build(), - "previous", SnapshotRef.tagBuilder(previousSnapshotId).build(), - "test", SnapshotRef.branchBuilder(previousSnapshotId).build()); + SnapshotRef.MAIN_BRANCH, + SnapshotRef.branchBuilder(currentSnapshotId).build(), + "previous", + SnapshotRef.tagBuilder(previousSnapshotId).build(), + "test", + SnapshotRef.branchBuilder(previousSnapshotId).build()); List statisticsFiles = ImmutableList.of( @@ -402,7 +405,8 @@ public void testInvalidMainBranch() throws IOException { Schema schema = new Schema(6, Types.NestedField.required(10, "x", Types.StringType.get())); Map refs = - ImmutableMap.of("main", SnapshotRef.branchBuilder(previousSnapshotId).build()); + ImmutableMap.of( + SnapshotRef.MAIN_BRANCH, SnapshotRef.branchBuilder(previousSnapshotId).build()); assertThatThrownBy( () -> @@ -450,7 +454,7 @@ public void testMainWithoutCurrent() throws IOException { Schema schema = new Schema(6, Types.NestedField.required(10, "x", Types.StringType.get())); Map refs = - ImmutableMap.of("main", SnapshotRef.branchBuilder(snapshotId).build()); + ImmutableMap.of(SnapshotRef.MAIN_BRANCH, SnapshotRef.branchBuilder(snapshotId).build()); assertThatThrownBy( () -> @@ -492,7 +496,7 @@ public void testBranchSnapshotMissing() { Schema schema = new Schema(6, Types.NestedField.required(10, "x", Types.StringType.get())); Map refs = - ImmutableMap.of("main", SnapshotRef.branchBuilder(snapshotId).build()); + ImmutableMap.of(SnapshotRef.MAIN_BRANCH, SnapshotRef.branchBuilder(snapshotId).build()); assertThatThrownBy( () -> diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index 7715c045bd9e..61e0b9ae5a32 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -765,7 +765,7 @@ public void testCommitProperties() { @TestTemplate public void testRowDeltaWithConcurrentManifestRewrite() throws IOException { assumeThat(formatVersion).isEqualTo(2); - String branch = "main"; + String branch = SnapshotRef.MAIN_BRANCH; RowDelta rowDelta = table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES); Snapshot first = commit(table, rowDelta, branch); @@ -829,7 +829,7 @@ public void testRowDeltaWithConcurrentManifestRewrite() throws IOException { @TestTemplate public void testOverwriteWithConcurrentManifestRewrite() throws IOException { assumeThat(formatVersion).isGreaterThanOrEqualTo(2); - String branch = "main"; + String branch = SnapshotRef.MAIN_BRANCH; OverwriteFiles overwrite = table.newOverwrite().addFile(FILE_A).addFile(FILE_A2); Snapshot first = commit(table, overwrite, branch); @@ -917,8 +917,12 @@ public void testExtendBaseTransaction() { txn.commitTransaction(); assertThat(version()).isEqualTo(2); - assertThat(readMetadata().refs()).hasSize(2).containsKey("main").containsKey("branch"); - assertThat(readMetadata().ref("main").snapshotId()).isEqualTo(mainSnapshot.snapshotId()); + assertThat(readMetadata().refs()) + .hasSize(2) + .containsKey(SnapshotRef.MAIN_BRANCH) + .containsKey("branch"); + assertThat(readMetadata().ref(SnapshotRef.MAIN_BRANCH).snapshotId()) + .isEqualTo(mainSnapshot.snapshotId()); assertThat(readMetadata().snapshot(mainSnapshot.snapshotId()).allManifests(table.io())) .hasSize(1); assertThat(readMetadata().ref("branch").snapshotId()).isEqualTo(branchSnapshot.snapshotId()); diff --git a/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java b/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java index d7c3a7b32bc8..6e5d04499e75 100644 --- a/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java +++ b/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java @@ -32,6 +32,7 @@ import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.CatalogLoader; @@ -94,7 +95,7 @@ public void setupBenchmark() throws IOException { r -> new DynamicRecordInternal( "t", - "main", + SnapshotRef.MAIN_BRANCH, SCHEMA, RowDataConverter.convert(SCHEMA, r), PartitionSpec.unpartitioned(), diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java index a01b82539c5a..beb244b72eab 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java @@ -24,6 +24,7 @@ import java.util.Set; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -44,7 +45,7 @@ class WriteTarget implements Serializable { boolean upsertMode, Set equalityFields) { this.tableName = tableName; - this.branch = branch != null ? branch : "main"; + this.branch = branch != null ? branch : SnapshotRef.MAIN_BRANCH; this.schemaId = schemaId; this.specId = specId; this.upsertMode = upsertMode; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index fefd88cf57fe..32f8c5a7998d 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -69,15 +69,15 @@ public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { public static Object[][] parameters() { return new Object[][] { // Remove after the deprecation of TableSchema - BEGIN - {"1", "main", true}, + {"1", SnapshotRef.MAIN_BRANCH, true}, {"1", "testBranch", true}, - {"2", "main", true}, + {"2", SnapshotRef.MAIN_BRANCH, true}, {"2", "testBranch", true}, // Remove after the deprecation of TableSchema - END - {"1", "main", false}, + {"1", SnapshotRef.MAIN_BRANCH, false}, {"1", "testBranch", false}, - {"2", "main", false}, + {"2", SnapshotRef.MAIN_BRANCH, false}, {"2", "testBranch", false}, }; } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index ffd40b6cdc95..9ef7c707821e 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -242,7 +242,7 @@ public void testDeleteStats() throws Exception { false, elementsPerCheckpoint, expectedRecords, - "main"); + SnapshotRef.MAIN_BRANCH); DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); String fromStat = diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index 8ce3e1886f40..9117534ed63b 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -55,11 +55,25 @@ public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { public static Object[][] parameters() { return new Object[][] { // Remove after the deprecation of TableSchema - BEGIN - {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, true, "main"}, + { + FileFormat.AVRO, + 1, + false, + TableProperties.WRITE_DISTRIBUTION_MODE_NONE, + true, + SnapshotRef.MAIN_BRANCH + }, {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, true, "testBranch"}, // Remove after the deprecation of TableSchema - END - {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, false, "main"}, + { + FileFormat.AVRO, + 1, + false, + TableProperties.WRITE_DISTRIBUTION_MODE_NONE, + false, + SnapshotRef.MAIN_BRANCH + }, { FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, false, "testBranch" }, diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index abe17e710fde..d478f91498c8 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -63,6 +63,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.StructLike; import org.apache.iceberg.TestBase; import org.apache.iceberg.flink.SimpleDataUtil; @@ -95,11 +96,11 @@ public class TestIcebergFilesCommitter extends TestBase { @Parameters(name = "formatVersion = {0}, fileFormat = {1}, branch = {2}") protected static List parameters() { return Arrays.asList( - new Object[] {1, FileFormat.AVRO, "main"}, + new Object[] {1, FileFormat.AVRO, SnapshotRef.MAIN_BRANCH}, new Object[] {2, FileFormat.AVRO, "test-branch"}, - new Object[] {1, FileFormat.PARQUET, "main"}, + new Object[] {1, FileFormat.PARQUET, SnapshotRef.MAIN_BRANCH}, new Object[] {2, FileFormat.PARQUET, "test-branch"}, - new Object[] {1, FileFormat.ORC, "main"}, + new Object[] {1, FileFormat.ORC, SnapshotRef.MAIN_BRANCH}, new Object[] {2, FileFormat.ORC, "test-branch"}); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java index ee5560712657..cbacd15ae8f5 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java @@ -57,11 +57,11 @@ public class TestIcebergSinkBranch extends TestFlinkIcebergSinkBase { public static Object[][] parameters() { return new Object[][] { // Remove after the deprecation of TableSchema - BEGIN - {"main", true}, + {SnapshotRef.MAIN_BRANCH, true}, {"testBranch", true}, // Remove after the deprecation of TableSchema - END - {"main", false}, + {SnapshotRef.MAIN_BRANCH, false}, {"testBranch", false}, }; } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java index f873dcd99c06..590b3f6c984c 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -225,7 +225,7 @@ public void testDeleteStats() throws Exception { false, elementsPerCheckpoint, expectedRecords, - "main"); + SnapshotRef.MAIN_BRANCH); DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); String fromStat = diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java index 2711df72cab1..0c07bc946189 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java @@ -58,6 +58,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Catalog; @@ -217,9 +218,15 @@ void testWrite() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows); } @@ -230,11 +237,11 @@ void testWritePartitioned() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec)); + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec)); runTest(rows); } @@ -257,11 +264,11 @@ void testWritePartitionedAdjustSchemaIdsInSpec() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(schema, "t1", "main", spec), - new DynamicIcebergDataImpl(schema, "t1", "main", spec), - new DynamicIcebergDataImpl(schema, "t1", "main", spec), - new DynamicIcebergDataImpl(schema2, "t1", "main", spec2), - new DynamicIcebergDataImpl(schema2, "t1", "main", spec2)); + new DynamicIcebergDataImpl(schema, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(schema, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(schema, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(schema2, "t1", SnapshotRef.MAIN_BRANCH, spec2), + new DynamicIcebergDataImpl(schema2, "t1", SnapshotRef.MAIN_BRANCH, spec2)); runTest(rows); } @@ -291,15 +298,35 @@ void testSchemaEvolutionFieldOrderChanges() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + schema, + expectedSchema, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + schema, + expectedSchema, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + schema, + expectedSchema, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - schema2, expectedSchema2, "t1", "main", PartitionSpec.unpartitioned()), + schema2, + expectedSchema2, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - schema2, expectedSchema2, "t1", "main", PartitionSpec.unpartitioned())); + schema2, + expectedSchema2, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); for (DynamicIcebergDataImpl row : rows) { if (row.schemaExpected == expectedSchema) { @@ -316,9 +343,15 @@ void testMultipleTables() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows); } @@ -329,8 +362,8 @@ void testMultipleTablesPartitioned() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t2", "main", spec)); + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t2", SnapshotRef.MAIN_BRANCH, spec)); runTest(rows); } @@ -340,9 +373,15 @@ void testSchemaEvolutionAddField() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA2, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows, this.env, 1); } @@ -352,9 +391,15 @@ void testRowEvolutionNullMissingOptionalField() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA2, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows, this.env, 1); } @@ -379,7 +424,7 @@ void testRowEvolutionMakeMissingRequiredFieldOptional() throws Exception { writeSchemaWithoutRequiredField, existingSchemaWithRequiredField, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned())); runTest(rows, this.env, 1); @@ -393,9 +438,10 @@ void testSchemaEvolutionNonBackwardsCompatible() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(initialSchema, "t1", "main", PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - erroringSchema, "t1", "main", PartitionSpec.unpartitioned())); + initialSchema, "t1", SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + erroringSchema, "t1", SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned())); try { runTest(rows, StreamExecutionEnvironment.getExecutionEnvironment(), 1); @@ -416,18 +462,28 @@ void testPartitionSpecEvolution() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec1), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec2), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec1), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec2), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec1), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2)); + SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec2)); runTest(rows); } @@ -439,7 +495,10 @@ void testMultipleBranches() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", "branch1", PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows); } @@ -449,23 +508,50 @@ void testWriteMultipleTablesWithSchemaChanges() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t2", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA2, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t2", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA2, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows); } @@ -478,7 +564,7 @@ void testUpsert() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -487,7 +573,7 @@ void testUpsert() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -495,7 +581,7 @@ void testUpsert() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -503,7 +589,7 @@ void testUpsert() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -547,7 +633,7 @@ void testUpsertV3() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -556,7 +642,7 @@ void testUpsertV3() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -564,7 +650,7 @@ void testUpsertV3() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -572,7 +658,7 @@ void testUpsertV3() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -625,7 +711,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -634,7 +720,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -642,7 +728,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -650,7 +736,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -662,7 +748,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t2", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -671,7 +757,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t2", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -679,7 +765,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t2", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -687,7 +773,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t2", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -745,9 +831,15 @@ void testCommitFailedBeforeOrAfterCommit() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); final CommitHook commitHook = new FailBeforeAndAfterCommit(); assertThat(FailBeforeAndAfterCommit.failedBeforeCommit).isFalse(); @@ -769,9 +861,15 @@ void testCommitConcurrency() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); TableIdentifier tableIdentifier = TableIdentifier.of("default", "t1"); Catalog catalog = CATALOG_EXTENSION.catalog(); @@ -789,9 +887,15 @@ void testCommitsOnceWhenConcurrentDuplicateCommit(boolean overwriteMode) throws List records = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, tableId.name(), "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + tableId.name(), + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, tableId.name(), "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + tableId.name(), + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); CommitHook duplicateCommit = new DuplicateCommitHook( @@ -842,9 +946,11 @@ void testOptInDropUnusedColumns() throws Exception { List rows = Lists.newArrayList( // Drop columns - new DynamicIcebergDataImpl(schema2, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + schema2, "t1", SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned()), // Re-add columns - new DynamicIcebergDataImpl(schema1, "t1", "main", PartitionSpec.unpartitioned())); + new DynamicIcebergDataImpl( + schema1, "t1", SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned())); DataStream dataStream = env.fromData(rows, TypeInformation.of(new TypeHint<>() {})); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java index ae5b2f67120b..b9aa56ab2e61 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java @@ -39,6 +39,7 @@ 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.catalog.TableIdentifier; import org.apache.iceberg.data.RandomGenericData; @@ -127,7 +128,7 @@ void before() { PartitionSpec.unpartitioned(), ImmutableMap.of(MAX_CONTINUOUS_EMPTY_COMMITS, "100000")); - table.manageSnapshots().createBranch("main").commit(); + table.manageSnapshots().createBranch(SnapshotRef.MAIN_BRANCH).commit(); } List records = RandomGenericData.generate(SCHEMA, SAMPLE_SIZE, 1L); @@ -136,7 +137,7 @@ void before() { rows.add( new DynamicRecord( IDENTIFIERS[i % TABLE_NUM], - "main", + SnapshotRef.MAIN_BRANCH, SCHEMA, RowDataConverter.convert(SCHEMA, records.get(i)), PartitionSpec.unpartitioned(), diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java index 04246bf03996..9fe854643f7f 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java @@ -32,6 +32,7 @@ import org.apache.iceberg.DistributionMode; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; @@ -44,7 +45,7 @@ class TestHashKeyGenerator { Types.NestedField.required(1, "id", Types.IntegerType.get()), Types.NestedField.required(2, "data", Types.StringType.get())); - private static final String BRANCH = "main"; + private static final String BRANCH = SnapshotRef.MAIN_BRANCH; private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("default", "table"); @Test diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java index 72b420c3452f..d696059902f0 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java @@ -26,6 +26,7 @@ import org.apache.commons.lang3.SerializationUtils; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; @@ -86,7 +87,11 @@ void testCacheInvalidationAfterSchemaChange() { catalog.dropTable(tableIdentifier); catalog.createTable(tableIdentifier, SCHEMA2); tableUpdater.update( - tableIdentifier, "main", SCHEMA2, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA2, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); Schema schema2 = cache.schema(tableIdentifier, SCHEMA2, false).resolvedTableSchema(); assertThat(schema2.sameSchema(SCHEMA2)).isTrue(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java index a49624a65ebf..c0b376d30e60 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java @@ -25,6 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; @@ -66,7 +67,11 @@ void testTableCreation(@TempDir Path tempDir) { catalog1.createTable(identifier, schema, spec, locationOverride, tableProperties); tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), tableCreator); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + tableCreator); assertThat(catalog.tableExists(tableIdentifier)).isTrue(); assertThat(catalog.loadTable(tableIdentifier).properties().get("key")).isEqualTo("value"); assertThat(catalog.loadTable(tableIdentifier).location()).isEqualTo(locationOverride); @@ -89,7 +94,11 @@ void testTableAlreadyExists() { // Make sure that the cache is invalidated and the table refreshed without an error Tuple2 result = tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); assertThat(result.f0.resolvedTableSchema().sameSchema(SCHEMA)).isTrue(); assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); assertThat(result.f1).isEqualTo(PartitionSpec.unpartitioned()); @@ -121,7 +130,8 @@ void testSpecCreation() { TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).bucket("data", 10).build(); - tableUpdater.update(tableIdentifier, "main", SCHEMA, spec, TableCreator.DEFAULT); + tableUpdater.update( + tableIdentifier, SnapshotRef.MAIN_BRANCH, SCHEMA, spec, TableCreator.DEFAULT); Table table = catalog.loadTable(tableIdentifier); assertThat(table).isNotNull(); @@ -141,7 +151,7 @@ void testInvalidateOldCacheEntryOnUpdate() { tableUpdater .update( tableIdentifier, - "main", + SnapshotRef.MAIN_BRANCH, SCHEMA2, PartitionSpec.unpartitioned(), TableCreator.DEFAULT) @@ -163,7 +173,11 @@ void testLastResultInvalidation() { // Initialize cache tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); // Update table behind the scenes catalog.dropTable(tableIdentifier); @@ -177,7 +191,7 @@ void testLastResultInvalidation() { tableUpdater .update( tableIdentifier, - "main", + SnapshotRef.MAIN_BRANCH, SCHEMA2, PartitionSpec.unpartitioned(), TableCreator.DEFAULT) @@ -203,7 +217,11 @@ void testDropUnusedColumns() { Tuple2 result = tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); Schema tableSchema = catalog.loadTable(tableIdentifier).schema(); @@ -226,7 +244,11 @@ void testNamespaceAndTableCreation() { Tuple2 result = tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); assertThat(namespaceCatalog.namespaceExists(Namespace.of("new_namespace"))).isTrue(); @@ -251,7 +273,11 @@ void testTableCreationWithExistingNamespace() { Tuple2 result = tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); assertThat(namespaceCatalog.namespaceExists(namespace)).isTrue(); assertThat(catalog.tableExists(tableIdentifier)).isTrue(); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java index 1903cb125332..6ae7bfb53a2e 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java @@ -50,6 +50,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.avro.Avro; @@ -696,7 +697,7 @@ public void testSnapshotReferencesMetatable() { // Check branch entries in refs table List mainBranch = sql("SELECT * FROM %s$refs WHERE name='main' AND type='BRANCH'", TABLE_NAME); - assertThat((String) mainBranch.get(0).getFieldAs("name")).isEqualTo("main"); + assertThat((String) mainBranch.get(0).getFieldAs("name")).isEqualTo(SnapshotRef.MAIN_BRANCH); assertThat((String) mainBranch.get(0).getFieldAs("type")).isEqualTo("BRANCH"); assertThat((Long) mainBranch.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); List testBranch = @@ -733,7 +734,8 @@ public void testSnapshotReferencesMetatable() { assertThat((String) testTagProjection.get(0).getFieldAs("min_snapshots_to_keep")).isNull(); List mainBranchProjection = sql("SELECT name, type FROM %s$refs WHERE name='main' AND type = 'BRANCH'", TABLE_NAME); - assertThat((String) mainBranchProjection.get(0).getFieldAs("name")).isEqualTo("main"); + assertThat((String) mainBranchProjection.get(0).getFieldAs("name")) + .isEqualTo(SnapshotRef.MAIN_BRANCH); assertThat((String) mainBranchProjection.get(0).getFieldAs("type")).isEqualTo("BRANCH"); List testBranchProjection = sql( diff --git a/flink/v2.0/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java b/flink/v2.0/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java index d7c3a7b32bc8..6e5d04499e75 100644 --- a/flink/v2.0/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java +++ b/flink/v2.0/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java @@ -32,6 +32,7 @@ import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.CatalogLoader; @@ -94,7 +95,7 @@ public void setupBenchmark() throws IOException { r -> new DynamicRecordInternal( "t", - "main", + SnapshotRef.MAIN_BRANCH, SCHEMA, RowDataConverter.convert(SCHEMA, r), PartitionSpec.unpartitioned(), diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java index a01b82539c5a..beb244b72eab 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java @@ -24,6 +24,7 @@ import java.util.Set; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -44,7 +45,7 @@ class WriteTarget implements Serializable { boolean upsertMode, Set equalityFields) { this.tableName = tableName; - this.branch = branch != null ? branch : "main"; + this.branch = branch != null ? branch : SnapshotRef.MAIN_BRANCH; this.schemaId = schemaId; this.specId = specId; this.upsertMode = upsertMode; diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index a77ddead3003..0497ba6c680f 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -69,15 +69,15 @@ public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { public static Object[][] parameters() { return new Object[][] { // Remove after the deprecation of TableSchema - BEGIN - {"1", "main", true}, + {"1", SnapshotRef.MAIN_BRANCH, true}, {"1", "testBranch", true}, - {"2", "main", true}, + {"2", SnapshotRef.MAIN_BRANCH, true}, {"2", "testBranch", true}, // Remove after the deprecation of TableSchema - END - {"1", "main", false}, + {"1", SnapshotRef.MAIN_BRANCH, false}, {"1", "testBranch", false}, - {"2", "main", false}, + {"2", SnapshotRef.MAIN_BRANCH, false}, {"2", "testBranch", false}, }; } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index ffd40b6cdc95..9ef7c707821e 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -242,7 +242,7 @@ public void testDeleteStats() throws Exception { false, elementsPerCheckpoint, expectedRecords, - "main"); + SnapshotRef.MAIN_BRANCH); DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); String fromStat = diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index 8ce3e1886f40..9117534ed63b 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -55,11 +55,25 @@ public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { public static Object[][] parameters() { return new Object[][] { // Remove after the deprecation of TableSchema - BEGIN - {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, true, "main"}, + { + FileFormat.AVRO, + 1, + false, + TableProperties.WRITE_DISTRIBUTION_MODE_NONE, + true, + SnapshotRef.MAIN_BRANCH + }, {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, true, "testBranch"}, // Remove after the deprecation of TableSchema - END - {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, false, "main"}, + { + FileFormat.AVRO, + 1, + false, + TableProperties.WRITE_DISTRIBUTION_MODE_NONE, + false, + SnapshotRef.MAIN_BRANCH + }, { FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, false, "testBranch" }, diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 68621018be57..abe77e795b09 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -63,6 +63,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.StructLike; import org.apache.iceberg.TestBase; import org.apache.iceberg.flink.SimpleDataUtil; @@ -95,11 +96,11 @@ public class TestIcebergFilesCommitter extends TestBase { @Parameters(name = "formatVersion = {0}, fileFormat = {1}, branch = {2}") protected static List parameters() { return Arrays.asList( - new Object[] {1, FileFormat.AVRO, "main"}, + new Object[] {1, FileFormat.AVRO, SnapshotRef.MAIN_BRANCH}, new Object[] {2, FileFormat.AVRO, "test-branch"}, - new Object[] {1, FileFormat.PARQUET, "main"}, + new Object[] {1, FileFormat.PARQUET, SnapshotRef.MAIN_BRANCH}, new Object[] {2, FileFormat.PARQUET, "test-branch"}, - new Object[] {1, FileFormat.ORC, "main"}, + new Object[] {1, FileFormat.ORC, SnapshotRef.MAIN_BRANCH}, new Object[] {2, FileFormat.ORC, "test-branch"}); } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java index ddcb57f6ca33..0789be5b03cd 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java @@ -57,11 +57,11 @@ public class TestIcebergSinkBranch extends TestFlinkIcebergSinkBase { public static Object[][] parameters() { return new Object[][] { // Remove after the deprecation of TableSchema - BEGIN - {"main", true}, + {SnapshotRef.MAIN_BRANCH, true}, {"testBranch", true}, // Remove after the deprecation of TableSchema - END - {"main", false}, + {SnapshotRef.MAIN_BRANCH, false}, {"testBranch", false}, }; } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java index f873dcd99c06..590b3f6c984c 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -225,7 +225,7 @@ public void testDeleteStats() throws Exception { false, elementsPerCheckpoint, expectedRecords, - "main"); + SnapshotRef.MAIN_BRANCH); DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); String fromStat = diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java index 2711df72cab1..0c07bc946189 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java @@ -58,6 +58,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Catalog; @@ -217,9 +218,15 @@ void testWrite() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows); } @@ -230,11 +237,11 @@ void testWritePartitioned() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec)); + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec)); runTest(rows); } @@ -257,11 +264,11 @@ void testWritePartitionedAdjustSchemaIdsInSpec() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(schema, "t1", "main", spec), - new DynamicIcebergDataImpl(schema, "t1", "main", spec), - new DynamicIcebergDataImpl(schema, "t1", "main", spec), - new DynamicIcebergDataImpl(schema2, "t1", "main", spec2), - new DynamicIcebergDataImpl(schema2, "t1", "main", spec2)); + new DynamicIcebergDataImpl(schema, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(schema, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(schema, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(schema2, "t1", SnapshotRef.MAIN_BRANCH, spec2), + new DynamicIcebergDataImpl(schema2, "t1", SnapshotRef.MAIN_BRANCH, spec2)); runTest(rows); } @@ -291,15 +298,35 @@ void testSchemaEvolutionFieldOrderChanges() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + schema, + expectedSchema, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + schema, + expectedSchema, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + schema, + expectedSchema, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - schema2, expectedSchema2, "t1", "main", PartitionSpec.unpartitioned()), + schema2, + expectedSchema2, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - schema2, expectedSchema2, "t1", "main", PartitionSpec.unpartitioned())); + schema2, + expectedSchema2, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); for (DynamicIcebergDataImpl row : rows) { if (row.schemaExpected == expectedSchema) { @@ -316,9 +343,15 @@ void testMultipleTables() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows); } @@ -329,8 +362,8 @@ void testMultipleTablesPartitioned() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t2", "main", spec)); + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t2", SnapshotRef.MAIN_BRANCH, spec)); runTest(rows); } @@ -340,9 +373,15 @@ void testSchemaEvolutionAddField() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA2, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows, this.env, 1); } @@ -352,9 +391,15 @@ void testRowEvolutionNullMissingOptionalField() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA2, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows, this.env, 1); } @@ -379,7 +424,7 @@ void testRowEvolutionMakeMissingRequiredFieldOptional() throws Exception { writeSchemaWithoutRequiredField, existingSchemaWithRequiredField, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned())); runTest(rows, this.env, 1); @@ -393,9 +438,10 @@ void testSchemaEvolutionNonBackwardsCompatible() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(initialSchema, "t1", "main", PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - erroringSchema, "t1", "main", PartitionSpec.unpartitioned())); + initialSchema, "t1", SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + erroringSchema, "t1", SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned())); try { runTest(rows, StreamExecutionEnvironment.getExecutionEnvironment(), 1); @@ -416,18 +462,28 @@ void testPartitionSpecEvolution() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec1), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec2), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec1), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec2), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec1), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2)); + SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec2)); runTest(rows); } @@ -439,7 +495,10 @@ void testMultipleBranches() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", "branch1", PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows); } @@ -449,23 +508,50 @@ void testWriteMultipleTablesWithSchemaChanges() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t2", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA2, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t2", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA2, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows); } @@ -478,7 +564,7 @@ void testUpsert() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -487,7 +573,7 @@ void testUpsert() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -495,7 +581,7 @@ void testUpsert() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -503,7 +589,7 @@ void testUpsert() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -547,7 +633,7 @@ void testUpsertV3() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -556,7 +642,7 @@ void testUpsertV3() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -564,7 +650,7 @@ void testUpsertV3() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -572,7 +658,7 @@ void testUpsertV3() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -625,7 +711,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -634,7 +720,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -642,7 +728,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -650,7 +736,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -662,7 +748,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t2", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -671,7 +757,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t2", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -679,7 +765,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t2", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -687,7 +773,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t2", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -745,9 +831,15 @@ void testCommitFailedBeforeOrAfterCommit() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); final CommitHook commitHook = new FailBeforeAndAfterCommit(); assertThat(FailBeforeAndAfterCommit.failedBeforeCommit).isFalse(); @@ -769,9 +861,15 @@ void testCommitConcurrency() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); TableIdentifier tableIdentifier = TableIdentifier.of("default", "t1"); Catalog catalog = CATALOG_EXTENSION.catalog(); @@ -789,9 +887,15 @@ void testCommitsOnceWhenConcurrentDuplicateCommit(boolean overwriteMode) throws List records = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, tableId.name(), "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + tableId.name(), + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, tableId.name(), "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + tableId.name(), + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); CommitHook duplicateCommit = new DuplicateCommitHook( @@ -842,9 +946,11 @@ void testOptInDropUnusedColumns() throws Exception { List rows = Lists.newArrayList( // Drop columns - new DynamicIcebergDataImpl(schema2, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + schema2, "t1", SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned()), // Re-add columns - new DynamicIcebergDataImpl(schema1, "t1", "main", PartitionSpec.unpartitioned())); + new DynamicIcebergDataImpl( + schema1, "t1", SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned())); DataStream dataStream = env.fromData(rows, TypeInformation.of(new TypeHint<>() {})); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java index ae5b2f67120b..b9aa56ab2e61 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java @@ -39,6 +39,7 @@ 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.catalog.TableIdentifier; import org.apache.iceberg.data.RandomGenericData; @@ -127,7 +128,7 @@ void before() { PartitionSpec.unpartitioned(), ImmutableMap.of(MAX_CONTINUOUS_EMPTY_COMMITS, "100000")); - table.manageSnapshots().createBranch("main").commit(); + table.manageSnapshots().createBranch(SnapshotRef.MAIN_BRANCH).commit(); } List records = RandomGenericData.generate(SCHEMA, SAMPLE_SIZE, 1L); @@ -136,7 +137,7 @@ void before() { rows.add( new DynamicRecord( IDENTIFIERS[i % TABLE_NUM], - "main", + SnapshotRef.MAIN_BRANCH, SCHEMA, RowDataConverter.convert(SCHEMA, records.get(i)), PartitionSpec.unpartitioned(), diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java index 04246bf03996..9fe854643f7f 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java @@ -32,6 +32,7 @@ import org.apache.iceberg.DistributionMode; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; @@ -44,7 +45,7 @@ class TestHashKeyGenerator { Types.NestedField.required(1, "id", Types.IntegerType.get()), Types.NestedField.required(2, "data", Types.StringType.get())); - private static final String BRANCH = "main"; + private static final String BRANCH = SnapshotRef.MAIN_BRANCH; private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("default", "table"); @Test diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java index 72b420c3452f..d696059902f0 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java @@ -26,6 +26,7 @@ import org.apache.commons.lang3.SerializationUtils; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; @@ -86,7 +87,11 @@ void testCacheInvalidationAfterSchemaChange() { catalog.dropTable(tableIdentifier); catalog.createTable(tableIdentifier, SCHEMA2); tableUpdater.update( - tableIdentifier, "main", SCHEMA2, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA2, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); Schema schema2 = cache.schema(tableIdentifier, SCHEMA2, false).resolvedTableSchema(); assertThat(schema2.sameSchema(SCHEMA2)).isTrue(); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java index a49624a65ebf..c0b376d30e60 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java @@ -25,6 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; @@ -66,7 +67,11 @@ void testTableCreation(@TempDir Path tempDir) { catalog1.createTable(identifier, schema, spec, locationOverride, tableProperties); tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), tableCreator); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + tableCreator); assertThat(catalog.tableExists(tableIdentifier)).isTrue(); assertThat(catalog.loadTable(tableIdentifier).properties().get("key")).isEqualTo("value"); assertThat(catalog.loadTable(tableIdentifier).location()).isEqualTo(locationOverride); @@ -89,7 +94,11 @@ void testTableAlreadyExists() { // Make sure that the cache is invalidated and the table refreshed without an error Tuple2 result = tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); assertThat(result.f0.resolvedTableSchema().sameSchema(SCHEMA)).isTrue(); assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); assertThat(result.f1).isEqualTo(PartitionSpec.unpartitioned()); @@ -121,7 +130,8 @@ void testSpecCreation() { TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).bucket("data", 10).build(); - tableUpdater.update(tableIdentifier, "main", SCHEMA, spec, TableCreator.DEFAULT); + tableUpdater.update( + tableIdentifier, SnapshotRef.MAIN_BRANCH, SCHEMA, spec, TableCreator.DEFAULT); Table table = catalog.loadTable(tableIdentifier); assertThat(table).isNotNull(); @@ -141,7 +151,7 @@ void testInvalidateOldCacheEntryOnUpdate() { tableUpdater .update( tableIdentifier, - "main", + SnapshotRef.MAIN_BRANCH, SCHEMA2, PartitionSpec.unpartitioned(), TableCreator.DEFAULT) @@ -163,7 +173,11 @@ void testLastResultInvalidation() { // Initialize cache tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); // Update table behind the scenes catalog.dropTable(tableIdentifier); @@ -177,7 +191,7 @@ void testLastResultInvalidation() { tableUpdater .update( tableIdentifier, - "main", + SnapshotRef.MAIN_BRANCH, SCHEMA2, PartitionSpec.unpartitioned(), TableCreator.DEFAULT) @@ -203,7 +217,11 @@ void testDropUnusedColumns() { Tuple2 result = tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); Schema tableSchema = catalog.loadTable(tableIdentifier).schema(); @@ -226,7 +244,11 @@ void testNamespaceAndTableCreation() { Tuple2 result = tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); assertThat(namespaceCatalog.namespaceExists(Namespace.of("new_namespace"))).isTrue(); @@ -251,7 +273,11 @@ void testTableCreationWithExistingNamespace() { Tuple2 result = tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); assertThat(namespaceCatalog.namespaceExists(namespace)).isTrue(); assertThat(catalog.tableExists(tableIdentifier)).isTrue(); diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java index 1903cb125332..6ae7bfb53a2e 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java @@ -50,6 +50,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.avro.Avro; @@ -696,7 +697,7 @@ public void testSnapshotReferencesMetatable() { // Check branch entries in refs table List mainBranch = sql("SELECT * FROM %s$refs WHERE name='main' AND type='BRANCH'", TABLE_NAME); - assertThat((String) mainBranch.get(0).getFieldAs("name")).isEqualTo("main"); + assertThat((String) mainBranch.get(0).getFieldAs("name")).isEqualTo(SnapshotRef.MAIN_BRANCH); assertThat((String) mainBranch.get(0).getFieldAs("type")).isEqualTo("BRANCH"); assertThat((Long) mainBranch.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); List testBranch = @@ -733,7 +734,8 @@ public void testSnapshotReferencesMetatable() { assertThat((String) testTagProjection.get(0).getFieldAs("min_snapshots_to_keep")).isNull(); List mainBranchProjection = sql("SELECT name, type FROM %s$refs WHERE name='main' AND type = 'BRANCH'", TABLE_NAME); - assertThat((String) mainBranchProjection.get(0).getFieldAs("name")).isEqualTo("main"); + assertThat((String) mainBranchProjection.get(0).getFieldAs("name")) + .isEqualTo(SnapshotRef.MAIN_BRANCH); assertThat((String) mainBranchProjection.get(0).getFieldAs("type")).isEqualTo("BRANCH"); List testBranchProjection = sql( diff --git a/flink/v2.1/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java b/flink/v2.1/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java index d7c3a7b32bc8..6e5d04499e75 100644 --- a/flink/v2.1/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java +++ b/flink/v2.1/flink/src/jmh/java/org/apache/iceberg/flink/sink/dynamic/DynamicRecordSerializerDeserializerBenchmark.java @@ -32,6 +32,7 @@ import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.CatalogLoader; @@ -94,7 +95,7 @@ public void setupBenchmark() throws IOException { r -> new DynamicRecordInternal( "t", - "main", + SnapshotRef.MAIN_BRANCH, SCHEMA, RowDataConverter.convert(SCHEMA, r), PartitionSpec.unpartitioned(), diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java index a01b82539c5a..beb244b72eab 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java @@ -24,6 +24,7 @@ import java.util.Set; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -44,7 +45,7 @@ class WriteTarget implements Serializable { boolean upsertMode, Set equalityFields) { this.tableName = tableName; - this.branch = branch != null ? branch : "main"; + this.branch = branch != null ? branch : SnapshotRef.MAIN_BRANCH; this.schemaId = schemaId; this.specId = specId; this.upsertMode = upsertMode; diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index a77ddead3003..0497ba6c680f 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -69,15 +69,15 @@ public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { public static Object[][] parameters() { return new Object[][] { // Remove after the deprecation of TableSchema - BEGIN - {"1", "main", true}, + {"1", SnapshotRef.MAIN_BRANCH, true}, {"1", "testBranch", true}, - {"2", "main", true}, + {"2", SnapshotRef.MAIN_BRANCH, true}, {"2", "testBranch", true}, // Remove after the deprecation of TableSchema - END - {"1", "main", false}, + {"1", SnapshotRef.MAIN_BRANCH, false}, {"1", "testBranch", false}, - {"2", "main", false}, + {"2", SnapshotRef.MAIN_BRANCH, false}, {"2", "testBranch", false}, }; } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index ffd40b6cdc95..9ef7c707821e 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -242,7 +242,7 @@ public void testDeleteStats() throws Exception { false, elementsPerCheckpoint, expectedRecords, - "main"); + SnapshotRef.MAIN_BRANCH); DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); String fromStat = diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index 8ce3e1886f40..9117534ed63b 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -55,11 +55,25 @@ public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { public static Object[][] parameters() { return new Object[][] { // Remove after the deprecation of TableSchema - BEGIN - {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, true, "main"}, + { + FileFormat.AVRO, + 1, + false, + TableProperties.WRITE_DISTRIBUTION_MODE_NONE, + true, + SnapshotRef.MAIN_BRANCH + }, {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, true, "testBranch"}, // Remove after the deprecation of TableSchema - END - {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, false, "main"}, + { + FileFormat.AVRO, + 1, + false, + TableProperties.WRITE_DISTRIBUTION_MODE_NONE, + false, + SnapshotRef.MAIN_BRANCH + }, { FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE, false, "testBranch" }, diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 68621018be57..abe77e795b09 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -63,6 +63,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.StructLike; import org.apache.iceberg.TestBase; import org.apache.iceberg.flink.SimpleDataUtil; @@ -95,11 +96,11 @@ public class TestIcebergFilesCommitter extends TestBase { @Parameters(name = "formatVersion = {0}, fileFormat = {1}, branch = {2}") protected static List parameters() { return Arrays.asList( - new Object[] {1, FileFormat.AVRO, "main"}, + new Object[] {1, FileFormat.AVRO, SnapshotRef.MAIN_BRANCH}, new Object[] {2, FileFormat.AVRO, "test-branch"}, - new Object[] {1, FileFormat.PARQUET, "main"}, + new Object[] {1, FileFormat.PARQUET, SnapshotRef.MAIN_BRANCH}, new Object[] {2, FileFormat.PARQUET, "test-branch"}, - new Object[] {1, FileFormat.ORC, "main"}, + new Object[] {1, FileFormat.ORC, SnapshotRef.MAIN_BRANCH}, new Object[] {2, FileFormat.ORC, "test-branch"}); } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java index ddcb57f6ca33..0789be5b03cd 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java @@ -57,11 +57,11 @@ public class TestIcebergSinkBranch extends TestFlinkIcebergSinkBase { public static Object[][] parameters() { return new Object[][] { // Remove after the deprecation of TableSchema - BEGIN - {"main", true}, + {SnapshotRef.MAIN_BRANCH, true}, {"testBranch", true}, // Remove after the deprecation of TableSchema - END - {"main", false}, + {SnapshotRef.MAIN_BRANCH, false}, {"testBranch", false}, }; } diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java index f873dcd99c06..590b3f6c984c 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java @@ -225,7 +225,7 @@ public void testDeleteStats() throws Exception { false, elementsPerCheckpoint, expectedRecords, - "main"); + SnapshotRef.MAIN_BRANCH); DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); String fromStat = diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java index 2711df72cab1..0c07bc946189 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java @@ -58,6 +58,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Catalog; @@ -217,9 +218,15 @@ void testWrite() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows); } @@ -230,11 +237,11 @@ void testWritePartitioned() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec)); + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec)); runTest(rows); } @@ -257,11 +264,11 @@ void testWritePartitionedAdjustSchemaIdsInSpec() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(schema, "t1", "main", spec), - new DynamicIcebergDataImpl(schema, "t1", "main", spec), - new DynamicIcebergDataImpl(schema, "t1", "main", spec), - new DynamicIcebergDataImpl(schema2, "t1", "main", spec2), - new DynamicIcebergDataImpl(schema2, "t1", "main", spec2)); + new DynamicIcebergDataImpl(schema, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(schema, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(schema, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(schema2, "t1", SnapshotRef.MAIN_BRANCH, spec2), + new DynamicIcebergDataImpl(schema2, "t1", SnapshotRef.MAIN_BRANCH, spec2)); runTest(rows); } @@ -291,15 +298,35 @@ void testSchemaEvolutionFieldOrderChanges() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + schema, + expectedSchema, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + schema, + expectedSchema, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - schema, expectedSchema, "t1", "main", PartitionSpec.unpartitioned()), + schema, + expectedSchema, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - schema2, expectedSchema2, "t1", "main", PartitionSpec.unpartitioned()), + schema2, + expectedSchema2, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - schema2, expectedSchema2, "t1", "main", PartitionSpec.unpartitioned())); + schema2, + expectedSchema2, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); for (DynamicIcebergDataImpl row : rows) { if (row.schemaExpected == expectedSchema) { @@ -316,9 +343,15 @@ void testMultipleTables() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows); } @@ -329,8 +362,8 @@ void testMultipleTablesPartitioned() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t2", "main", spec)); + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t2", SnapshotRef.MAIN_BRANCH, spec)); runTest(rows); } @@ -340,9 +373,15 @@ void testSchemaEvolutionAddField() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA2, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows, this.env, 1); } @@ -352,9 +391,15 @@ void testRowEvolutionNullMissingOptionalField() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA2, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows, this.env, 1); } @@ -379,7 +424,7 @@ void testRowEvolutionMakeMissingRequiredFieldOptional() throws Exception { writeSchemaWithoutRequiredField, existingSchemaWithRequiredField, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned())); runTest(rows, this.env, 1); @@ -393,9 +438,10 @@ void testSchemaEvolutionNonBackwardsCompatible() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(initialSchema, "t1", "main", PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - erroringSchema, "t1", "main", PartitionSpec.unpartitioned())); + initialSchema, "t1", SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + erroringSchema, "t1", SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned())); try { runTest(rows, StreamExecutionEnvironment.getExecutionEnvironment(), 1); @@ -416,18 +462,28 @@ void testPartitionSpecEvolution() throws Exception { List rows = Lists.newArrayList( - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec1), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec2), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec1), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec2), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec1), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec1), - new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, "t1", "main", spec2)); + SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec2)); runTest(rows); } @@ -439,7 +495,10 @@ void testMultipleBranches() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", "branch1", PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows); } @@ -449,23 +508,50 @@ void testWriteMultipleTablesWithSchemaChanges() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t2", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA2, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA2, "t2", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA2, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); runTest(rows); } @@ -478,7 +564,7 @@ void testUpsert() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -487,7 +573,7 @@ void testUpsert() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -495,7 +581,7 @@ void testUpsert() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -503,7 +589,7 @@ void testUpsert() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -547,7 +633,7 @@ void testUpsertV3() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -556,7 +642,7 @@ void testUpsertV3() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -564,7 +650,7 @@ void testUpsertV3() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -572,7 +658,7 @@ void testUpsertV3() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -625,7 +711,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -634,7 +720,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -642,7 +728,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -650,7 +736,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t1", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -662,7 +748,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t2", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -671,7 +757,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t2", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -679,7 +765,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t2", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -687,7 +773,7 @@ void testMultiFormatVersion() throws Exception { new DynamicIcebergDataImpl( SimpleDataUtil.SCHEMA, "t2", - "main", + SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned(), true, Sets.newHashSet("id"), @@ -745,9 +831,15 @@ void testCommitFailedBeforeOrAfterCommit() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); final CommitHook commitHook = new FailBeforeAndAfterCommit(); assertThat(FailBeforeAndAfterCommit.failedBeforeCommit).isFalse(); @@ -769,9 +861,15 @@ void testCommitConcurrency() throws Exception { List rows = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t1", "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + "t1", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, "t2", "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + "t2", + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); TableIdentifier tableIdentifier = TableIdentifier.of("default", "t1"); Catalog catalog = CATALOG_EXTENSION.catalog(); @@ -789,9 +887,15 @@ void testCommitsOnceWhenConcurrentDuplicateCommit(boolean overwriteMode) throws List records = Lists.newArrayList( new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, tableId.name(), "main", PartitionSpec.unpartitioned()), + SimpleDataUtil.SCHEMA, + tableId.name(), + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned()), new DynamicIcebergDataImpl( - SimpleDataUtil.SCHEMA, tableId.name(), "main", PartitionSpec.unpartitioned())); + SimpleDataUtil.SCHEMA, + tableId.name(), + SnapshotRef.MAIN_BRANCH, + PartitionSpec.unpartitioned())); CommitHook duplicateCommit = new DuplicateCommitHook( @@ -842,9 +946,11 @@ void testOptInDropUnusedColumns() throws Exception { List rows = Lists.newArrayList( // Drop columns - new DynamicIcebergDataImpl(schema2, "t1", "main", PartitionSpec.unpartitioned()), + new DynamicIcebergDataImpl( + schema2, "t1", SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned()), // Re-add columns - new DynamicIcebergDataImpl(schema1, "t1", "main", PartitionSpec.unpartitioned())); + new DynamicIcebergDataImpl( + schema1, "t1", SnapshotRef.MAIN_BRANCH, PartitionSpec.unpartitioned())); DataStream dataStream = env.fromData(rows, TypeInformation.of(new TypeHint<>() {})); diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java index ae5b2f67120b..b9aa56ab2e61 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java @@ -39,6 +39,7 @@ 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.catalog.TableIdentifier; import org.apache.iceberg.data.RandomGenericData; @@ -127,7 +128,7 @@ void before() { PartitionSpec.unpartitioned(), ImmutableMap.of(MAX_CONTINUOUS_EMPTY_COMMITS, "100000")); - table.manageSnapshots().createBranch("main").commit(); + table.manageSnapshots().createBranch(SnapshotRef.MAIN_BRANCH).commit(); } List records = RandomGenericData.generate(SCHEMA, SAMPLE_SIZE, 1L); @@ -136,7 +137,7 @@ void before() { rows.add( new DynamicRecord( IDENTIFIERS[i % TABLE_NUM], - "main", + SnapshotRef.MAIN_BRANCH, SCHEMA, RowDataConverter.convert(SCHEMA, records.get(i)), PartitionSpec.unpartitioned(), diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java index 04246bf03996..9fe854643f7f 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestHashKeyGenerator.java @@ -32,6 +32,7 @@ import org.apache.iceberg.DistributionMode; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; @@ -44,7 +45,7 @@ class TestHashKeyGenerator { Types.NestedField.required(1, "id", Types.IntegerType.get()), Types.NestedField.required(2, "data", Types.StringType.get())); - private static final String BRANCH = "main"; + private static final String BRANCH = SnapshotRef.MAIN_BRANCH; private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of("default", "table"); @Test diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java index 72b420c3452f..d696059902f0 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableMetadataCache.java @@ -26,6 +26,7 @@ import org.apache.commons.lang3.SerializationUtils; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; @@ -86,7 +87,11 @@ void testCacheInvalidationAfterSchemaChange() { catalog.dropTable(tableIdentifier); catalog.createTable(tableIdentifier, SCHEMA2); tableUpdater.update( - tableIdentifier, "main", SCHEMA2, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA2, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); Schema schema2 = cache.schema(tableIdentifier, SCHEMA2, false).resolvedTableSchema(); assertThat(schema2.sameSchema(SCHEMA2)).isTrue(); diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java index a49624a65ebf..c0b376d30e60 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestTableUpdater.java @@ -25,6 +25,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; @@ -66,7 +67,11 @@ void testTableCreation(@TempDir Path tempDir) { catalog1.createTable(identifier, schema, spec, locationOverride, tableProperties); tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), tableCreator); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + tableCreator); assertThat(catalog.tableExists(tableIdentifier)).isTrue(); assertThat(catalog.loadTable(tableIdentifier).properties().get("key")).isEqualTo("value"); assertThat(catalog.loadTable(tableIdentifier).location()).isEqualTo(locationOverride); @@ -89,7 +94,11 @@ void testTableAlreadyExists() { // Make sure that the cache is invalidated and the table refreshed without an error Tuple2 result = tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); assertThat(result.f0.resolvedTableSchema().sameSchema(SCHEMA)).isTrue(); assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); assertThat(result.f1).isEqualTo(PartitionSpec.unpartitioned()); @@ -121,7 +130,8 @@ void testSpecCreation() { TableUpdater tableUpdater = new TableUpdater(cache, catalog, false); PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).bucket("data", 10).build(); - tableUpdater.update(tableIdentifier, "main", SCHEMA, spec, TableCreator.DEFAULT); + tableUpdater.update( + tableIdentifier, SnapshotRef.MAIN_BRANCH, SCHEMA, spec, TableCreator.DEFAULT); Table table = catalog.loadTable(tableIdentifier); assertThat(table).isNotNull(); @@ -141,7 +151,7 @@ void testInvalidateOldCacheEntryOnUpdate() { tableUpdater .update( tableIdentifier, - "main", + SnapshotRef.MAIN_BRANCH, SCHEMA2, PartitionSpec.unpartitioned(), TableCreator.DEFAULT) @@ -163,7 +173,11 @@ void testLastResultInvalidation() { // Initialize cache tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); // Update table behind the scenes catalog.dropTable(tableIdentifier); @@ -177,7 +191,7 @@ void testLastResultInvalidation() { tableUpdater .update( tableIdentifier, - "main", + SnapshotRef.MAIN_BRANCH, SCHEMA2, PartitionSpec.unpartitioned(), TableCreator.DEFAULT) @@ -203,7 +217,11 @@ void testDropUnusedColumns() { Tuple2 result = tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); assertThat(result.f0.compareResult()).isEqualTo(CompareSchemasVisitor.Result.SAME); Schema tableSchema = catalog.loadTable(tableIdentifier).schema(); @@ -226,7 +244,11 @@ void testNamespaceAndTableCreation() { Tuple2 result = tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); assertThat(namespaceCatalog.namespaceExists(Namespace.of("new_namespace"))).isTrue(); @@ -251,7 +273,11 @@ void testTableCreationWithExistingNamespace() { Tuple2 result = tableUpdater.update( - tableIdentifier, "main", SCHEMA, PartitionSpec.unpartitioned(), TableCreator.DEFAULT); + tableIdentifier, + SnapshotRef.MAIN_BRANCH, + SCHEMA, + PartitionSpec.unpartitioned(), + TableCreator.DEFAULT); assertThat(namespaceCatalog.namespaceExists(namespace)).isTrue(); assertThat(catalog.tableExists(tableIdentifier)).isTrue(); diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java index 1903cb125332..6ae7bfb53a2e 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java @@ -50,6 +50,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.avro.Avro; @@ -696,7 +697,7 @@ public void testSnapshotReferencesMetatable() { // Check branch entries in refs table List mainBranch = sql("SELECT * FROM %s$refs WHERE name='main' AND type='BRANCH'", TABLE_NAME); - assertThat((String) mainBranch.get(0).getFieldAs("name")).isEqualTo("main"); + assertThat((String) mainBranch.get(0).getFieldAs("name")).isEqualTo(SnapshotRef.MAIN_BRANCH); assertThat((String) mainBranch.get(0).getFieldAs("type")).isEqualTo("BRANCH"); assertThat((Long) mainBranch.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); List testBranch = @@ -733,7 +734,8 @@ public void testSnapshotReferencesMetatable() { assertThat((String) testTagProjection.get(0).getFieldAs("min_snapshots_to_keep")).isNull(); List mainBranchProjection = sql("SELECT name, type FROM %s$refs WHERE name='main' AND type = 'BRANCH'", TABLE_NAME); - assertThat((String) mainBranchProjection.get(0).getFieldAs("name")).isEqualTo("main"); + assertThat((String) mainBranchProjection.get(0).getFieldAs("name")) + .isEqualTo(SnapshotRef.MAIN_BRANCH); assertThat((String) mainBranchProjection.get(0).getFieldAs("type")).isEqualTo("BRANCH"); List testBranchProjection = sql( diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java index 40e33759791f..dd33336cf800 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java @@ -39,6 +39,7 @@ import org.apache.iceberg.DataFiles; import org.apache.iceberg.Files; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.TableOperations; @@ -250,7 +251,7 @@ void createBranch(String name) throws NessieNotFoundException, NessieConflictExc void createBranch(String name, String hash) throws NessieNotFoundException, NessieConflictException { - createBranch(name, hash, "main"); + createBranch(name, hash, SnapshotRef.MAIN_BRANCH); } void createBranch(String name, String hash, String sourceRef) diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestBranchVisibility.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestBranchVisibility.java index 4f1b87022f47..4d2cce4fa3e0 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestBranchVisibility.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestBranchVisibility.java @@ -30,6 +30,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.Transaction; @@ -64,7 +65,7 @@ public class TestBranchVisibility extends BaseTestIceberg { private int schemaCounter = 1; public TestBranchVisibility() { - super("main"); + super(SnapshotRef.MAIN_BRANCH); } @BeforeEach @@ -80,7 +81,7 @@ public void after() throws NessieNotFoundException, NessieConflictException { catalog.dropTable(tableIdentifier1); catalog.dropTable(tableIdentifier2); for (Reference reference : api.getAllReferences().get().getReferences()) { - if (!reference.getName().equals("main")) { + if (!reference.getName().equals(SnapshotRef.MAIN_BRANCH)) { api.deleteBranch().branch((Branch) reference).delete(); } } @@ -112,7 +113,7 @@ public void testCatalogOnReference() { testCatalogEquality(refCatalog, testCatalog, true, true, () -> {}); // catalog created with hash points to same catalog as above - NessieCatalog refHashCatalog = initCatalog("main"); + NessieCatalog refHashCatalog = initCatalog(SnapshotRef.MAIN_BRANCH); testCatalogEquality(refHashCatalog, catalog, true, true, () -> {}); } @@ -120,7 +121,7 @@ public void testCatalogOnReference() { public void testCatalogWithTableNames() { updateSchema(testCatalog, tableIdentifier2); - String mainName = "main"; + String mainName = SnapshotRef.MAIN_BRANCH; // asking for table@branch gives expected regardless of catalog assertThat(metadataLocation(catalog, TableIdentifier.of("test-ns", "table1@test"))) diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestCustomNessieClient.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestCustomNessieClient.java index 2e5be4670b70..02c68431b3c0 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestCustomNessieClient.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestCustomNessieClient.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TestCatalogUtil; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -32,7 +33,7 @@ public class TestCustomNessieClient extends BaseTestIceberg { public TestCustomNessieClient() { - super("main"); + super(SnapshotRef.MAIN_BRANCH); } @Test diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java index 5123809a04a9..1ae33325a90c 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java @@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.catalog.CatalogTests; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.util.LocationUtil; @@ -120,7 +121,7 @@ protected NessieCatalog initCatalog( "type", "nessie", "ref", - "main", + SnapshotRef.MAIN_BRANCH, CatalogProperties.URI, uri, CatalogProperties.WAREHOUSE_LOCATION, diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java index de68656c89ab..c9fb97e0f9e9 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java @@ -32,6 +32,7 @@ import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.SortOrder; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.catalog.Namespace; @@ -65,7 +66,8 @@ public TestNessieIcebergClient() { @Test public void testWithNullRefLoadsMain() throws NessieNotFoundException { NessieIcebergClient client = new NessieIcebergClient(api, null, null, ImmutableMap.of()); - assertThat(client.getRef().getReference()).isEqualTo(api.getReference().refName("main").get()); + assertThat(client.getRef().getReference()) + .isEqualTo(api.getReference().refName(SnapshotRef.MAIN_BRANCH).get()); } @Test @@ -76,11 +78,15 @@ public void testWithNullHash() throws NessieNotFoundException { @Test public void testWithReference() throws NessieNotFoundException { - NessieIcebergClient client = new NessieIcebergClient(api, "main", null, ImmutableMap.of()); + NessieIcebergClient client = + new NessieIcebergClient(api, SnapshotRef.MAIN_BRANCH, null, ImmutableMap.of()); assertThat(client.withReference(null, null)).isEqualTo(client); - assertThat(client.withReference("main", null)).isNotEqualTo(client); - assertThat(client.withReference("main", api.getReference().refName("main").get().getHash())) + assertThat(client.withReference(SnapshotRef.MAIN_BRANCH, null)).isNotEqualTo(client); + assertThat( + client.withReference( + SnapshotRef.MAIN_BRANCH, + api.getReference().refName(SnapshotRef.MAIN_BRANCH).get().getHash())) .isEqualTo(client); assertThat(client.withReference(BRANCH, null)).isNotEqualTo(client); diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java index 34de63fbef07..def5328d4297 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java @@ -43,6 +43,7 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.Schema; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; @@ -445,7 +446,10 @@ public void testRegisterTableWithGivenBranch() throws Exception { List metadataVersionFiles = metadataVersionFiles(tableLocation); assertThat(1).isEqualTo(metadataVersionFiles.size()); ImmutableTableReference tableReference = - ImmutableTableReference.builder().reference("main").name(TABLE_NAME).build(); + ImmutableTableReference.builder() + .reference(SnapshotRef.MAIN_BRANCH) + .name(TABLE_NAME) + .build(); TableIdentifier identifier = TableIdentifier.of(DB_NAME, tableReference.toString()); try { api.createNamespace().namespace(DB_NAME).refName(tableReference.getReference()).create(); @@ -517,7 +521,10 @@ public void testRegisterTableMoreThanOneBranch() throws Exception { List metadataVersionFiles = metadataVersionFiles(tableLocation); assertThat(1).isEqualTo(metadataVersionFiles.size()); ImmutableTableReference tableReference = - ImmutableTableReference.builder().reference("main").name(TABLE_NAME).build(); + ImmutableTableReference.builder() + .reference(SnapshotRef.MAIN_BRANCH) + .name(TABLE_NAME) + .build(); TableIdentifier identifier = TableIdentifier.of(DB_NAME, tableReference.toString()); try { api.createNamespace().namespace(DB_NAME).refName(tableReference.getReference()).create(); diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieViewCatalog.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieViewCatalog.java index 7bc24451c814..dc3d5dc0ed4c 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieViewCatalog.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieViewCatalog.java @@ -25,6 +25,7 @@ import java.nio.file.Path; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -82,7 +83,7 @@ public void setUp(NessieClientFactory clientFactory, @NessieClientUri URI nessie initialHashOfDefaultBranch = api.getDefaultBranch().getHash(); uri = nessieUri.toASCIIString(); hadoopConfig = new Configuration(); - catalog = initNessieCatalog("main"); + catalog = initNessieCatalog(SnapshotRef.MAIN_BRANCH); } @AfterEach diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java index 5153a1fcad5b..04947592214d 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java @@ -44,6 +44,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.avro.Avro; @@ -748,7 +749,9 @@ public void testSnapshotReferencesMetatable() throws Exception { .collectAsList(); assertThat(mainBranch) .hasSize(1) - .containsExactly(RowFactory.create("main", "BRANCH", currentSnapshotId, null, null, null)); + .containsExactly( + RowFactory.create( + SnapshotRef.MAIN_BRANCH, "BRANCH", currentSnapshotId, null, null, null)); assertThat(mainBranch.get(0).schema().fieldNames()) .containsExactly( "name", @@ -816,7 +819,7 @@ public void testSnapshotReferencesMetatable() throws Exception { .collectAsList(); assertThat(mainBranchProjection) .hasSize(1) - .containsExactly(RowFactory.create("main", "BRANCH")); + .containsExactly(RowFactory.create(SnapshotRef.MAIN_BRANCH, "BRANCH")); assertThat(mainBranchProjection.get(0).schema().fieldNames()).containsExactly("name", "type"); List testBranchProjection = diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index 8884654fe0b3..4f4c0c7f7aea 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -87,7 +87,7 @@ public class TestSparkDataWrite { public static Object[][] parameters() { return new Object[][] { new Object[] {FileFormat.PARQUET, null}, - new Object[] {FileFormat.PARQUET, "main"}, + new Object[] {FileFormat.PARQUET, SnapshotRef.MAIN_BRANCH}, new Object[] {FileFormat.PARQUET, "testBranch"}, new Object[] {FileFormat.AVRO, null}, new Object[] {FileFormat.ORC, "testBranch"} diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java index be114782cbf4..9ca29635f060 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java @@ -44,6 +44,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.avro.Avro; @@ -748,7 +749,9 @@ public void testSnapshotReferencesMetatable() throws Exception { .collectAsList(); assertThat(mainBranch) .hasSize(1) - .containsExactly(RowFactory.create("main", "BRANCH", currentSnapshotId, null, null, null)); + .containsExactly( + RowFactory.create( + SnapshotRef.MAIN_BRANCH, "BRANCH", currentSnapshotId, null, null, null)); assertThat(mainBranch.get(0).schema().fieldNames()) .containsExactly( "name", @@ -816,7 +819,7 @@ public void testSnapshotReferencesMetatable() throws Exception { .collectAsList(); assertThat(mainBranchProjection) .hasSize(1) - .containsExactly(RowFactory.create("main", "BRANCH")); + .containsExactly(RowFactory.create(SnapshotRef.MAIN_BRANCH, "BRANCH")); assertThat(mainBranchProjection.get(0).schema().fieldNames()).containsExactly("name", "type"); List testBranchProjection = diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index 4ccbf86f1257..94547c2cf8fb 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -87,7 +87,7 @@ public class TestSparkDataWrite { public static Object[][] parameters() { return new Object[][] { new Object[] {FileFormat.PARQUET, null}, - new Object[] {FileFormat.PARQUET, "main"}, + new Object[] {FileFormat.PARQUET, SnapshotRef.MAIN_BRANCH}, new Object[] {FileFormat.PARQUET, "testBranch"}, new Object[] {FileFormat.AVRO, null}, new Object[] {FileFormat.ORC, "testBranch"} diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java index be114782cbf4..9ca29635f060 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java @@ -44,6 +44,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.avro.Avro; @@ -748,7 +749,9 @@ public void testSnapshotReferencesMetatable() throws Exception { .collectAsList(); assertThat(mainBranch) .hasSize(1) - .containsExactly(RowFactory.create("main", "BRANCH", currentSnapshotId, null, null, null)); + .containsExactly( + RowFactory.create( + SnapshotRef.MAIN_BRANCH, "BRANCH", currentSnapshotId, null, null, null)); assertThat(mainBranch.get(0).schema().fieldNames()) .containsExactly( "name", @@ -816,7 +819,7 @@ public void testSnapshotReferencesMetatable() throws Exception { .collectAsList(); assertThat(mainBranchProjection) .hasSize(1) - .containsExactly(RowFactory.create("main", "BRANCH")); + .containsExactly(RowFactory.create(SnapshotRef.MAIN_BRANCH, "BRANCH")); assertThat(mainBranchProjection.get(0).schema().fieldNames()).containsExactly("name", "type"); List testBranchProjection = diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index 4ccbf86f1257..94547c2cf8fb 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -87,7 +87,7 @@ public class TestSparkDataWrite { public static Object[][] parameters() { return new Object[][] { new Object[] {FileFormat.PARQUET, null}, - new Object[] {FileFormat.PARQUET, "main"}, + new Object[] {FileFormat.PARQUET, SnapshotRef.MAIN_BRANCH}, new Object[] {FileFormat.PARQUET, "testBranch"}, new Object[] {FileFormat.AVRO, null}, new Object[] {FileFormat.ORC, "testBranch"} diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java index 96e002979fcf..6082c5a4e4f7 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java @@ -44,6 +44,7 @@ import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.avro.Avro; @@ -749,7 +750,9 @@ public void testSnapshotReferencesMetatable() throws Exception { .collectAsList(); assertThat(mainBranch) .hasSize(1) - .containsExactly(RowFactory.create("main", "BRANCH", currentSnapshotId, null, null, null)); + .containsExactly( + RowFactory.create( + SnapshotRef.MAIN_BRANCH, "BRANCH", currentSnapshotId, null, null, null)); assertThat(mainBranch.get(0).schema().fieldNames()) .containsExactly( "name", @@ -817,7 +820,7 @@ public void testSnapshotReferencesMetatable() throws Exception { .collectAsList(); assertThat(mainBranchProjection) .hasSize(1) - .containsExactly(RowFactory.create("main", "BRANCH")); + .containsExactly(RowFactory.create(SnapshotRef.MAIN_BRANCH, "BRANCH")); assertThat(mainBranchProjection.get(0).schema().fieldNames()).containsExactly("name", "type"); List testBranchProjection = 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 4ccbf86f1257..94547c2cf8fb 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 @@ -87,7 +87,7 @@ public class TestSparkDataWrite { public static Object[][] parameters() { return new Object[][] { new Object[] {FileFormat.PARQUET, null}, - new Object[] {FileFormat.PARQUET, "main"}, + new Object[] {FileFormat.PARQUET, SnapshotRef.MAIN_BRANCH}, new Object[] {FileFormat.PARQUET, "testBranch"}, new Object[] {FileFormat.AVRO, null}, new Object[] {FileFormat.ORC, "testBranch"} From a7b8a08b2abb6125ffbe6ea61817e5826db61ab2 Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Thu, 8 Jan 2026 17:17:17 -0800 Subject: [PATCH 169/201] Spark 4.1: Fix spark 4.1 test for unlink table metadata's last-updated timestamp (#15004) --- .../apache/iceberg/spark/extensions/TestMetadataTables.java | 5 ++--- .../iceberg/spark/source/TestIcebergSourceTablesBase.java | 5 +++++ 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java index 6082c5a4e4f7..6e608f8c43a7 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetadataTables.java @@ -579,7 +579,7 @@ public void testMetadataLogEntries() throws Exception { parentSnapshot.schemaId(), parentSnapshot.sequenceNumber()), row( - DateTimeUtils.toJavaTimestamp(currentSnapshot.timestampMillis() * 1000), + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), tableMetadata.metadataFileLocation(), currentSnapshot.snapshotId(), currentSnapshot.schemaId(), @@ -598,8 +598,7 @@ public void testMetadataLogEntries() throws Exception { "Result should match the latest snapshot entry", ImmutableList.of( row( - DateTimeUtils.toJavaTimestamp( - tableMetadata.currentSnapshot().timestampMillis() * 1000), + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), tableMetadata.metadataFileLocation(), tableMetadata.currentSnapshot().snapshotId(), tableMetadata.currentSnapshot().schemaId(), 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 63994726cafb..67647925e59d 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 @@ -46,6 +46,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.Files; +import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -821,6 +822,10 @@ public void testHistoryTable() { // rollback the table state to the first snapshot table.manageSnapshots().rollbackTo(firstSnapshotId).commit(); long rollbackTimestamp = Iterables.getLast(table.history()).timestampMillis(); + assertThat(rollbackTimestamp) + .as("Rollback history timestamp should be greater than first snapshot timestamp") + .isEqualTo(((HasTableOperations) table).operations().current().lastUpdatedMillis()) + .isGreaterThan(firstSnapshotTimestamp); inputDf .select("id", "data") From a90848e21018d15aa6af12528b1aeef5e3365af5 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Thu, 8 Jan 2026 23:18:24 -0500 Subject: [PATCH 170/201] infra: add gradle cache to github workflows --- .github/workflows/api-binary-compatibility.yml | 7 +++++++ .github/workflows/publish-iceberg-rest-fixture-docker.yml | 7 +++++++ .github/workflows/publish-snapshot.yml | 7 +++++++ 3 files changed, 21 insertions(+) diff --git a/.github/workflows/api-binary-compatibility.yml b/.github/workflows/api-binary-compatibility.yml index 9f7daee250b8..3343ba4035cb 100644 --- a/.github/workflows/api-binary-compatibility.yml +++ b/.github/workflows/api-binary-compatibility.yml @@ -55,6 +55,13 @@ jobs: with: distribution: zulu java-version: 17 + - uses: actions/cache@v5 + with: + path: | + ~/.gradle/caches + ~/.gradle/wrapper + key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} + restore-keys: ${{ runner.os }}-gradle- - run: | echo "Using the old version tag, as per git describe, of $(git describe)"; - run: ./gradlew revapi --rerun-tasks diff --git a/.github/workflows/publish-iceberg-rest-fixture-docker.yml b/.github/workflows/publish-iceberg-rest-fixture-docker.yml index 4b6b4696bec7..5d379615f5fe 100644 --- a/.github/workflows/publish-iceberg-rest-fixture-docker.yml +++ b/.github/workflows/publish-iceberg-rest-fixture-docker.yml @@ -42,6 +42,13 @@ jobs: with: distribution: zulu java-version: 21 + - uses: actions/cache@v5 + with: + path: | + ~/.gradle/caches + ~/.gradle/wrapper + key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} + restore-keys: ${{ runner.os }}-gradle- - name: Build Iceberg Open API project run: ./gradlew :iceberg-open-api:shadowJar - name: Login to Docker Hub diff --git a/.github/workflows/publish-snapshot.yml b/.github/workflows/publish-snapshot.yml index 14e87044132d..c8012b5d02f9 100644 --- a/.github/workflows/publish-snapshot.yml +++ b/.github/workflows/publish-snapshot.yml @@ -38,6 +38,13 @@ jobs: with: distribution: zulu java-version: 17 + - uses: actions/cache@v5 + with: + path: | + ~/.gradle/caches + ~/.gradle/wrapper + key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} + restore-keys: ${{ runner.os }}-gradle- - run: | ./gradlew printVersion ./gradlew -DallModules publishApachePublicationToMavenRepository -PmavenUser=${{ secrets.NEXUS_USER }} -PmavenPassword=${{ secrets.NEXUS_PW }} From 4a4d73408aa771ed7ec2f4e94ca3e1b2df03c1de Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 9 Jan 2026 08:39:51 +0100 Subject: [PATCH 171/201] Core: Add storage credentials to FetchPlanningResultResponse (#14994) --- .../FetchPlanningResultResponse.java | 20 +++- .../FetchPlanningResultResponseParser.java | 31 ++++- ...TestFetchPlanningResultResponseParser.java | 107 +++++++++++++++++- 3 files changed, 149 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/FetchPlanningResultResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/FetchPlanningResultResponse.java index 05d64a235891..59db196244f5 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/FetchPlanningResultResponse.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/FetchPlanningResultResponse.java @@ -24,19 +24,25 @@ import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionSpec; 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.Lists; import org.apache.iceberg.rest.PlanStatus; +import org.apache.iceberg.rest.credentials.Credential; public class FetchPlanningResultResponse extends BaseScanTaskResponse { private final PlanStatus planStatus; + private final List credentials; private FetchPlanningResultResponse( PlanStatus planStatus, List planTasks, List fileScanTasks, List deleteFiles, - Map specsById) { + Map specsById, + List credentials) { super(planTasks, fileScanTasks, deleteFiles, specsById); this.planStatus = planStatus; + this.credentials = credentials; validate(); } @@ -44,6 +50,10 @@ public PlanStatus planStatus() { return planStatus; } + public List credentials() { + return credentials != null ? credentials : ImmutableList.of(); + } + public static Builder builder() { return new Builder(); } @@ -66,16 +76,22 @@ public static class Builder private Builder() {} private PlanStatus planStatus; + private final List credentials = Lists.newArrayList(); public Builder withPlanStatus(PlanStatus status) { this.planStatus = status; return this; } + public Builder withCredentials(List credentialsToAdd) { + credentials.addAll(credentialsToAdd); + return this; + } + @Override public FetchPlanningResultResponse build() { return new FetchPlanningResultResponse( - planStatus, planTasks(), fileScanTasks(), deleteFiles(), specsById()); + planStatus, planTasks(), fileScanTasks(), deleteFiles(), specsById(), credentials); } } } diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/FetchPlanningResultResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/FetchPlanningResultResponseParser.java index 5400ef1dd13d..4a523d3c023b 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/FetchPlanningResultResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/FetchPlanningResultResponseParser.java @@ -30,11 +30,14 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.rest.PlanStatus; import org.apache.iceberg.rest.TableScanResponseParser; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.credentials.CredentialParser; import org.apache.iceberg.util.JsonUtil; public class FetchPlanningResultResponseParser { private static final String STATUS = "status"; private static final String PLAN_TASKS = "plan-tasks"; + private static final String STORAGE_CREDENTIALS = "storage-credentials"; private FetchPlanningResultResponseParser() {} @@ -59,6 +62,15 @@ public static void toJson(FetchPlanningResultResponse response, JsonGenerator ge JsonUtil.writeStringArray(PLAN_TASKS, response.planTasks(), gen); } + if (!response.credentials().isEmpty()) { + gen.writeArrayFieldStart(STORAGE_CREDENTIALS); + for (Credential credential : response.credentials()) { + CredentialParser.toJson(credential, gen); + } + + gen.writeEndArray(); + } + TableScanResponseParser.serializeScanTasks( response.fileScanTasks(), response.deleteFiles(), response.specsById(), gen); gen.writeEndObject(); @@ -82,11 +94,18 @@ public static FetchPlanningResultResponse fromJson( List deleteFiles = TableScanResponseParser.parseDeleteFiles(json, specsById); List fileScanTasks = TableScanResponseParser.parseFileScanTasks(json, deleteFiles, specsById, caseSensitive); - return FetchPlanningResultResponse.builder() - .withPlanStatus(planStatus) - .withPlanTasks(planTasks) - .withFileScanTasks(fileScanTasks) - .withSpecsById(specsById) - .build(); + + FetchPlanningResultResponse.Builder builder = + FetchPlanningResultResponse.builder() + .withPlanStatus(planStatus) + .withPlanTasks(planTasks) + .withFileScanTasks(fileScanTasks) + .withSpecsById(specsById); + + if (json.hasNonNull(STORAGE_CREDENTIALS)) { + builder.withCredentials(LoadCredentialsResponseParser.fromJson(json).credentials()); + } + + return builder.build(); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java index 5d9d2cad828c..5fdfdc281f4f 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestFetchPlanningResultResponseParser.java @@ -40,8 +40,12 @@ import org.apache.iceberg.SchemaParser; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.ResidualEvaluator; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.rest.PlanStatus; import org.apache.iceberg.rest.RESTSerializers; +import org.apache.iceberg.rest.credentials.Credential; +import org.apache.iceberg.rest.credentials.ImmutableCredential; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -140,7 +144,6 @@ public void roundTripSerdeWithInvalidPlanStatusSubmittedWithTasksPresent() { @Test public void roundTripSerdeWithInvalidPlanStatusSubmittedWithDeleteFilesNoFileScanTasksPresent() { - PlanStatus planStatus = PlanStatus.fromName("submitted"); assertThatThrownBy( () -> { @@ -225,4 +228,106 @@ public void roundTripSerdeWithValidStatusAndFileScanTasks() throws JsonProcessin assertThat(FetchPlanningResultResponseParser.toJson(copyResponse, false)) .isEqualTo(expectedToJson); } + + @Test + public void emptyOrInvalidCredentials() { + assertThat( + FetchPlanningResultResponseParser.fromJson( + "{\"status\": \"completed\",\"storage-credentials\": null}", + PARTITION_SPECS_BY_ID, + false) + .credentials()) + .isEmpty(); + + assertThat( + FetchPlanningResultResponseParser.fromJson( + "{\"status\": \"completed\",\"storage-credentials\": []}", + PARTITION_SPECS_BY_ID, + false) + .credentials()) + .isEmpty(); + + assertThatThrownBy( + () -> + FetchPlanningResultResponseParser.fromJson( + "{\"status\": \"completed\",\"storage-credentials\": \"invalid\"}", + PARTITION_SPECS_BY_ID, + false)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse credentials from non-array: \"invalid\""); + } + + @Test + public void roundTripSerdeWithCredentials() { + List credentials = + ImmutableList.of( + ImmutableCredential.builder() + .prefix("s3://custom-uri") + .config( + ImmutableMap.of( + "s3.access-key-id", + "keyId", + "s3.secret-access-key", + "accessKey", + "s3.session-token", + "sessionToken")) + .build(), + ImmutableCredential.builder() + .prefix("gs://custom-uri") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken1", "gcs.oauth2.token-expires-at", "1000")) + .build(), + ImmutableCredential.builder() + .prefix("gs") + .config( + ImmutableMap.of( + "gcs.oauth2.token", "gcsToken2", "gcs.oauth2.token-expires-at", "2000")) + .build()); + + FetchPlanningResultResponse response = + FetchPlanningResultResponse.builder() + .withPlanStatus(PlanStatus.COMPLETED) + .withCredentials(credentials) + .build(); + + String expectedJson = + "{\n" + + " \"status\" : \"completed\",\n" + + " \"storage-credentials\" : [ {\n" + + " \"prefix\" : \"s3://custom-uri\",\n" + + " \"config\" : {\n" + + " \"s3.access-key-id\" : \"keyId\",\n" + + " \"s3.secret-access-key\" : \"accessKey\",\n" + + " \"s3.session-token\" : \"sessionToken\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs://custom-uri\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken1\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"1000\"\n" + + " }\n" + + " }, {\n" + + " \"prefix\" : \"gs\",\n" + + " \"config\" : {\n" + + " \"gcs.oauth2.token\" : \"gcsToken2\",\n" + + " \"gcs.oauth2.token-expires-at\" : \"2000\"\n" + + " }\n" + + " } ]\n" + + "}"; + + String json = FetchPlanningResultResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + + FetchPlanningResultResponse fromResponse = + FetchPlanningResultResponseParser.fromJson(json, PARTITION_SPECS_BY_ID, false); + FetchPlanningResultResponse copyResponse = + FetchPlanningResultResponse.builder() + .withPlanStatus(fromResponse.planStatus()) + .withCredentials(credentials) + .build(); + + assertThat(FetchPlanningResultResponseParser.toJson(copyResponse, true)) + .isEqualTo(expectedJson); + } } From 6f7b5688dada3fa92a2b510b3ea26400b332a25b Mon Sep 17 00:00:00 2001 From: aiborodin Date: Fri, 9 Jan 2026 21:33:41 +1100 Subject: [PATCH 172/201] Flink: Dynamic Sink: Refactor write result aggregation (#14810) --- .../org/apache/iceberg/io/WriteResult.java | 11 + .../iceberg/flink/sink/CommitSummary.java | 7 +- .../sink/dynamic/DynamicCommittable.java | 24 +- .../dynamic/DynamicCommittableSerializer.java | 65 +++-- .../flink/sink/dynamic/DynamicCommitter.java | 182 ++++--------- .../sink/dynamic/DynamicWriteResult.java | 23 +- .../dynamic/DynamicWriteResultAggregator.java | 53 ++-- .../dynamic/DynamicWriteResultSerializer.java | 6 +- .../flink/sink/dynamic/DynamicWriter.java | 6 +- .../iceberg/flink/sink/dynamic/TableKey.java | 84 ++++++ .../flink/sink/dynamic/WriteTarget.java | 13 - .../TestDynamicCommittableSerializer.java | 86 ++++-- .../sink/dynamic/TestDynamicCommitter.java | 249 +++++++----------- .../sink/dynamic/TestDynamicIcebergSink.java | 93 +++++++ .../dynamic/TestDynamicIcebergSinkPerf.java | 6 +- .../TestDynamicWriteResultAggregator.java | 125 +++++++-- .../TestDynamicWriteResultSerializer.java | 12 +- 17 files changed, 639 insertions(+), 406 deletions(-) create mode 100644 flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java diff --git a/core/src/main/java/org/apache/iceberg/io/WriteResult.java b/core/src/main/java/org/apache/iceberg/io/WriteResult.java index 39efaec7d34a..2ab2fff3c5ae 100644 --- a/core/src/main/java/org/apache/iceberg/io/WriteResult.java +++ b/core/src/main/java/org/apache/iceberg/io/WriteResult.java @@ -23,6 +23,7 @@ import java.util.List; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.CharSequenceSet; @@ -134,4 +135,14 @@ public WriteResult build() { return new WriteResult(dataFiles, deleteFiles, referencedDataFiles, rewrittenDeleteFiles); } } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("dataFiles", dataFiles) + .add("deleteFiles", deleteFiles) + .add("referencedDataFiles", referencedDataFiles) + .add("rewrittenDeleteFiles", rewrittenDeleteFiles) + .toString(); + } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java index 1b786e46452f..1da0f8564938 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink; import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.NavigableMap; import java.util.concurrent.atomic.AtomicLong; @@ -44,7 +45,11 @@ public CommitSummary(NavigableMap pendingResults) { } public void addAll(NavigableMap> pendingResults) { - pendingResults.values().forEach(writeResults -> writeResults.forEach(this::addWriteResult)); + pendingResults.values().forEach(this::addAll); + } + + public void addAll(Collection pendingResults) { + pendingResults.forEach(this::addWriteResult); } private void addWriteResult(WriteResult writeResult) { diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java index 33edefe71eb0..4f0b68573ff5 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittable.java @@ -26,7 +26,7 @@ /** * The aggregated results of a single checkpoint which should be committed. Containing the - * serialized {@link DeltaManifests} file - which contains the commit data, and the jobId, + * serialized {@link DeltaManifests} files - which contains the commit data, and the jobId, * operatorId, checkpointId triplet to identify the specific commit. * *

      {@link DynamicCommittableSerializer} is used to serialize {@link DynamicCommittable} between @@ -34,27 +34,27 @@ */ class DynamicCommittable implements Serializable { - private final WriteTarget key; - private final byte[] manifest; + private final TableKey key; + private final byte[][] manifests; private final String jobId; private final String operatorId; private final long checkpointId; DynamicCommittable( - WriteTarget key, byte[] manifest, String jobId, String operatorId, long checkpointId) { + TableKey key, byte[][] manifests, String jobId, String operatorId, long checkpointId) { this.key = key; - this.manifest = manifest; + this.manifests = manifests; this.jobId = jobId; this.operatorId = operatorId; this.checkpointId = checkpointId; } - WriteTarget key() { + TableKey key() { return key; } - byte[] manifest() { - return manifest; + byte[][] manifests() { + return manifests; } String jobId() { @@ -78,14 +78,14 @@ public boolean equals(Object o) { DynamicCommittable that = (DynamicCommittable) o; return checkpointId == that.checkpointId && Objects.equals(key, that.key) - && Objects.deepEquals(manifest, that.manifest) + && Arrays.deepEquals(manifests, that.manifests) && Objects.equals(jobId, that.jobId) && Objects.equals(operatorId, that.operatorId); } @Override public int hashCode() { - return Objects.hash(key, Arrays.hashCode(manifest), jobId, operatorId, checkpointId); + return Objects.hash(key, Arrays.deepHashCode(manifests), jobId, operatorId, checkpointId); } @Override @@ -97,8 +97,4 @@ public String toString() { .add("operatorId", operatorId) .toString(); } - - public WriteTarget writeTarget() { - return key; - } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java index 4aadcf1f3620..d599d29dba01 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommittableSerializer.java @@ -31,11 +31,12 @@ */ class DynamicCommittableSerializer implements SimpleVersionedSerializer { - private static final int VERSION = 1; + private static final int VERSION_1 = 1; + private static final int VERSION_2 = 2; @Override public int getVersion() { - return VERSION; + return VERSION_2; } @Override @@ -46,26 +47,60 @@ public byte[] serialize(DynamicCommittable committable) throws IOException { view.writeUTF(committable.jobId()); view.writeUTF(committable.operatorId()); view.writeLong(committable.checkpointId()); - view.writeInt(committable.manifest().length); - view.write(committable.manifest()); + + int numManifests = committable.manifests().length; + view.writeInt(numManifests); + for (int i = 0; i < numManifests; i++) { + byte[] manifest = committable.manifests()[i]; + view.writeInt(manifest.length); + view.write(manifest); + } + return out.toByteArray(); } @Override public DynamicCommittable deserialize(int version, byte[] serialized) throws IOException { - if (version == 1) { - DataInputDeserializer view = new DataInputDeserializer(serialized); - WriteTarget key = WriteTarget.deserializeFrom(view); - String jobId = view.readUTF(); - String operatorId = view.readUTF(); - long checkpointId = view.readLong(); - int manifestLen = view.readInt(); - byte[] manifestBuf; - manifestBuf = new byte[manifestLen]; - view.read(manifestBuf); - return new DynamicCommittable(key, manifestBuf, jobId, operatorId, checkpointId); + if (version == VERSION_1) { + return deserializeV1(serialized); + } else if (version == VERSION_2) { + return deserializeV2(serialized); } throw new IOException("Unrecognized version or corrupt state: " + version); } + + private DynamicCommittable deserializeV1(byte[] serialized) throws IOException { + DataInputDeserializer view = new DataInputDeserializer(serialized); + WriteTarget key = WriteTarget.deserializeFrom(view); + String jobId = view.readUTF(); + String operatorId = view.readUTF(); + long checkpointId = view.readLong(); + int manifestLen = view.readInt(); + byte[] manifestBuf = new byte[manifestLen]; + view.read(manifestBuf); + return new DynamicCommittable( + new TableKey(key.tableName(), key.branch()), + new byte[][] {manifestBuf}, + jobId, + operatorId, + checkpointId); + } + + private DynamicCommittable deserializeV2(byte[] serialized) throws IOException { + DataInputDeserializer view = new DataInputDeserializer(serialized); + TableKey key = TableKey.deserializeFrom(view); + String jobId = view.readUTF(); + String operatorId = view.readUTF(); + long checkpointId = view.readLong(); + + byte[][] manifestsBuf = new byte[view.readInt()][]; + for (int i = 0; i < manifestsBuf.length; i++) { + byte[] manifest = new byte[view.readInt()]; + view.read(manifest); + manifestsBuf[i] = manifest; + } + + return new DynamicCommittable(key, manifestsBuf, jobId, operatorId, checkpointId); + } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java index 1cddc64d6016..5e824773f4bf 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicCommitter.java @@ -19,15 +19,15 @@ package org.apache.iceberg.flink.sink.dynamic; import java.io.IOException; -import java.io.Serializable; import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Map; import java.util.NavigableMap; -import java.util.Objects; +import java.util.Optional; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; import org.apache.flink.annotation.Internal; import org.apache.flink.api.connector.sink2.Committer; import org.apache.flink.core.io.SimpleVersionedSerialization; @@ -51,12 +51,10 @@ import org.apache.iceberg.flink.sink.FlinkManifestUtil; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; 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.Maps; import org.apache.iceberg.util.ContentFileUtil; -import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; @@ -79,26 +77,15 @@ class DynamicCommitter implements Committer { private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; private static final Logger LOG = LoggerFactory.getLogger(DynamicCommitter.class); - private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; - private static final WriteResult EMPTY_WRITE_RESULT = - WriteResult.builder() - .addDataFiles(Lists.newArrayList()) - .addDeleteFiles(Lists.newArrayList()) - .build(); private static final long INITIAL_CHECKPOINT_ID = -1L; - @VisibleForTesting - static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; - private static final String FLINK_JOB_ID = "flink.job-id"; private static final String OPERATOR_ID = "flink.operator-id"; private final Map snapshotProperties; private final boolean replacePartitions; private final DynamicCommitterMetrics committerMetrics; private final Catalog catalog; - private final Map maxContinuousEmptyCommitsMap; - private final Map continuousEmptyCheckpointsMap; private final ExecutorService workerPool; DynamicCommitter( @@ -112,9 +99,6 @@ class DynamicCommitter implements Committer { this.replacePartitions = replacePartitions; this.committerMetrics = committerMetrics; this.catalog = catalog; - this.maxContinuousEmptyCommitsMap = Maps.newHashMap(); - this.continuousEmptyCheckpointsMap = Maps.newHashMap(); - this.workerPool = ThreadPools.newFixedThreadPool("iceberg-committer-pool-" + sinkId, workerPoolSize); } @@ -126,9 +110,15 @@ public void commit(Collection> commitRequests) return; } - // For every table and every checkpoint, we store the list of to-be-committed - // DynamicCommittable. - // There may be DynamicCommittable from previous checkpoints which have not been committed yet. + /* + Each (table, branch, checkpoint) triplet must have only one commit request. + There may be commit requests from previous checkpoints which have not been committed yet. + + We currently keep a List of commit requests per checkpoint instead of a single CommitRequest + to process the Flink state from previous releases, which had multiple commit requests created by the upstream + DynamicWriteResultAggregator. Iceberg 1.12 will remove this, and users should upgrade to the 1.11 release first + to migrate their state to a single commit request per checkpoint. + */ Map>>> commitRequestMap = Maps.newHashMap(); for (CommitRequest request : commitRequests) { @@ -151,12 +141,16 @@ public void commit(Collection> commitRequests) : List.of(); long maxCommittedCheckpointId = getMaxCommittedCheckpointId(ancestors, last.jobId(), last.operatorId()); + + NavigableMap>> skippedCommitRequests = + entry.getValue().headMap(maxCommittedCheckpointId, true); + LOG.debug( + "Skipping {} commit requests: {}", skippedCommitRequests.size(), skippedCommitRequests); // Mark the already committed FilesCommittable(s) as finished - entry - .getValue() - .headMap(maxCommittedCheckpointId, true) + skippedCommitRequests .values() .forEach(list -> list.forEach(CommitRequest::signalAlreadyCommitted)); + NavigableMap>> uncommitted = entry.getValue().tailMap(maxCommittedCheckpointId, false); if (!uncommitted.isEmpty()) { @@ -210,89 +204,45 @@ private void commitPendingRequests( NavigableMap> pendingResults = Maps.newTreeMap(); for (Map.Entry>> e : commitRequestMap.entrySet()) { for (CommitRequest committable : e.getValue()) { - if (Arrays.equals(EMPTY_MANIFEST_DATA, committable.getCommittable().manifest())) { - pendingResults - .computeIfAbsent(e.getKey(), unused -> Lists.newArrayList()) - .add(EMPTY_WRITE_RESULT); - } else { + for (byte[] manifest : committable.getCommittable().manifests()) { DeltaManifests deltaManifests = SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, committable.getCommittable().manifest()); - - WriteResult writeResult = - FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - if (TableUtil.formatVersion(table) > 2) { - for (DeleteFile deleteFile : writeResult.deleteFiles()) { - if (deleteFile.content() == FileContent.POSITION_DELETES) { - Preconditions.checkArgument( - ContentFileUtil.isDV(deleteFile), - "Can't add position delete file to the %s table. Concurrent table upgrade to V3 is not supported.", - table.name()); - } - } - } - + DeltaManifestsSerializer.INSTANCE, manifest); pendingResults .computeIfAbsent(e.getKey(), unused -> Lists.newArrayList()) - .add(writeResult); + .add(FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs())); manifests.addAll(deltaManifests.manifests()); } } } - CommitSummary summary = new CommitSummary(); - summary.addAll(pendingResults); - commitPendingResult(table, branch, pendingResults, summary, newFlinkJobId, operatorId); - if (committerMetrics != null) { - committerMetrics.updateCommitSummary(table.name(), summary); + if (TableUtil.formatVersion(table) > 2) { + Optional positionalDelete = + pendingResults.values().stream() + .flatMap(List::stream) + .flatMap(writeResult -> Arrays.stream(writeResult.deleteFiles())) + .filter(deleteFile -> deleteFile.content() == FileContent.POSITION_DELETES) + .filter(Predicate.not(ContentFileUtil::isDV)) + .findAny(); + Preconditions.checkArgument( + positionalDelete.isEmpty(), + "Can't add position delete file to the %s table. Concurrent table upgrade to V3 is not supported.", + table.name()); } - FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); - } - - private void commitPendingResult( - Table table, - String branch, - NavigableMap> pendingResults, - CommitSummary summary, - String newFlinkJobId, - String operatorId) { - long totalFiles = summary.dataFilesCount() + summary.deleteFilesCount(); - TableKey key = new TableKey(table.name(), branch); - int continuousEmptyCheckpoints = - continuousEmptyCheckpointsMap.computeIfAbsent(key, unused -> 0); - int maxContinuousEmptyCommits = - maxContinuousEmptyCommitsMap.computeIfAbsent( - key, - unused -> { - int result = - PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); - Preconditions.checkArgument( - result > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); - return result; - }); - continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0; - if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) { - if (replacePartitions) { - replacePartitions(table, branch, pendingResults, summary, newFlinkJobId, operatorId); - } else { - commitDeltaTxn(table, branch, pendingResults, summary, newFlinkJobId, operatorId); - } - - continuousEmptyCheckpoints = 0; + if (replacePartitions) { + replacePartitions(table, branch, pendingResults, newFlinkJobId, operatorId); } else { - long checkpointId = pendingResults.lastKey(); - LOG.info("Skip commit for checkpoint {} due to no data files or delete files.", checkpointId); + commitDeltaTxn(table, branch, pendingResults, newFlinkJobId, operatorId); } - continuousEmptyCheckpointsMap.put(key, continuousEmptyCheckpoints); + FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); } private void replacePartitions( Table table, String branch, NavigableMap> pendingResults, - CommitSummary summary, String newFlinkJobId, String operatorId) { // Iceberg tables are unsorted. So the order of the append data does not matter. @@ -305,6 +255,9 @@ private void replacePartitions( } } + CommitSummary summary = new CommitSummary(); + summary.addAll(pendingResults); + commitOperation( table, branch, @@ -320,7 +273,6 @@ private void commitDeltaTxn( Table table, String branch, NavigableMap> pendingResults, - CommitSummary summary, String newFlinkJobId, String operatorId) { for (Map.Entry> e : pendingResults.entrySet()) { @@ -340,6 +292,9 @@ private void commitDeltaTxn( Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); } + CommitSummary summary = new CommitSummary(); + summary.addAll(writeResults); + // Every Flink checkpoint contains a set of independent changes which can be committed // together. While it is technically feasible to combine append-only data across checkpoints, // for the sake of simplicity, we do not implement this (premature) optimization. Multiple @@ -434,6 +389,7 @@ void commitOperation( durationMs); if (committerMetrics != null) { committerMetrics.commitDuration(table.name(), durationMs); + committerMetrics.updateCommitSummary(table.name(), summary); } } @@ -441,54 +397,4 @@ void commitOperation( public void close() throws IOException { workerPool.shutdown(); } - - private static class TableKey implements Serializable { - private String tableName; - private String branch; - - TableKey(String tableName, String branch) { - this.tableName = tableName; - this.branch = branch; - } - - TableKey(DynamicCommittable committable) { - this.tableName = committable.key().tableName(); - this.branch = committable.key().branch(); - } - - String tableName() { - return tableName; - } - - String branch() { - return branch; - } - - @Override - public boolean equals(Object other) { - if (this == other) { - return true; - } - - if (other == null || getClass() != other.getClass()) { - return false; - } - - TableKey that = (TableKey) other; - return tableName.equals(that.tableName) && branch.equals(that.branch); - } - - @Override - public int hashCode() { - return Objects.hash(tableName, branch); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("tableName", tableName) - .add("branch", branch) - .toString(); - } - } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java index 85806f932ad5..d8d0ed6b573e 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResult.java @@ -19,22 +19,37 @@ package org.apache.iceberg.flink.sink.dynamic; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; class DynamicWriteResult { - - private final WriteTarget key; + private final TableKey key; + private final int specId; private final WriteResult writeResult; - DynamicWriteResult(WriteTarget key, WriteResult writeResult) { + DynamicWriteResult(TableKey key, int specId, WriteResult writeResult) { this.key = key; + this.specId = specId; this.writeResult = writeResult; } - WriteTarget key() { + TableKey key() { return key; } + public int specId() { + return specId; + } + WriteResult writeResult() { return writeResult; } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("key", key) + .add("specId", specId) + .add("writeResult", writeResult) + .toString(); + } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java index 927491fa89ea..47d239200b06 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultAggregator.java @@ -42,6 +42,7 @@ import org.apache.iceberg.flink.sink.FlinkManifestUtil; import org.apache.iceberg.flink.sink.ManifestOutputFileFactory; import org.apache.iceberg.io.WriteResult; +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.slf4j.Logger; @@ -57,11 +58,10 @@ class DynamicWriteResultAggregator implements OneInputStreamOperator< CommittableMessage, CommittableMessage> { private static final Logger LOG = LoggerFactory.getLogger(DynamicWriteResultAggregator.class); - private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; private final CatalogLoader catalogLoader; private final int cacheMaximumSize; - private transient Map> results; + private transient Map>> resultsByTableKeyAndSpec; private transient Map> specs; private transient Map> outputFileFactoriesAndFormatVersions; @@ -82,7 +82,7 @@ public void open() throws Exception { this.operatorId = getOperatorID().toString(); this.subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); this.attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber(); - this.results = Maps.newHashMap(); + this.resultsByTableKeyAndSpec = Maps.newHashMap(); this.specs = new LRUCache<>(cacheMaximumSize); this.outputFileFactoriesAndFormatVersions = new LRUCache<>(cacheMaximumSize); this.catalog = catalogLoader.loadCatalog(); @@ -96,14 +96,15 @@ public void finish() throws IOException { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { Collection> committables = - Sets.newHashSetWithExpectedSize(results.size()); + Sets.newHashSetWithExpectedSize(resultsByTableKeyAndSpec.size()); int count = 0; - for (Map.Entry> entries : results.entrySet()) { + for (Map.Entry>> entries : + resultsByTableKeyAndSpec.entrySet()) { committables.add( new CommittableWithLineage<>( new DynamicCommittable( entries.getKey(), - writeToManifest(entries.getKey(), entries.getValue(), checkpointId), + writeToManifests(entries.getKey().tableName(), entries.getValue(), checkpointId), getContainingTask().getEnvironment().getJobID().toString(), getRuntimeContext().getOperatorUniqueID(), checkpointId), @@ -121,32 +122,42 @@ public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { new StreamRecord<>( new CommittableWithLineage<>(c.getCommittable(), checkpointId, subTaskId)))); LOG.info("Emitted {} commit message to downstream committer operator", count); - results.clear(); + resultsByTableKeyAndSpec.clear(); } /** - * Write all the completed data files to a newly created manifest file and return the manifest's + * Write all the completed data files to a newly created manifest files and return the manifests' * avro serialized bytes. */ @VisibleForTesting - byte[] writeToManifest( - WriteTarget key, Collection writeResults, long checkpointId) + byte[][] writeToManifests( + String tableName, Map> writeResultsBySpec, long checkpointId) throws IOException { - if (writeResults.isEmpty()) { - return EMPTY_MANIFEST_DATA; + byte[][] deltaManifestsBySpec = new byte[writeResultsBySpec.size()][]; + int idx = 0; + for (Map.Entry> entry : writeResultsBySpec.entrySet()) { + deltaManifestsBySpec[idx] = + writeToManifest(tableName, entry.getKey(), entry.getValue(), checkpointId); + idx++; } + return deltaManifestsBySpec; + } + + private byte[] writeToManifest( + String tableName, int specId, Collection writeResults, long checkpointId) + throws IOException { WriteResult.Builder builder = WriteResult.builder(); - writeResults.forEach(w -> builder.add(w.writeResult())); + writeResults.forEach(builder::add); WriteResult result = builder.build(); Tuple2 outputFileFactoryAndVersion = - outputFileFactoryAndFormatVersion(key.tableName()); + outputFileFactoryAndFormatVersion(tableName); DeltaManifests deltaManifests = FlinkManifestUtil.writeCompletedFiles( result, () -> outputFileFactoryAndVersion.f0.create(checkpointId), - spec(key.tableName(), key.specId()), + spec(tableName, specId), outputFileFactoryAndVersion.f1); return SimpleVersionedSerialization.writeVersionAndSerialize( @@ -160,8 +171,16 @@ public void processElement(StreamRecord> if (element.isRecord() && element.getValue() instanceof CommittableWithLineage) { DynamicWriteResult result = ((CommittableWithLineage) element.getValue()).getCommittable(); - WriteTarget key = result.key(); - results.computeIfAbsent(key, unused -> Sets.newHashSet()).add(result); + Collection resultsPerTableKeyAndSpec = + resultsByTableKeyAndSpec + .computeIfAbsent(result.key(), unused -> Maps.newHashMap()) + .computeIfAbsent(result.specId(), unused -> Lists.newArrayList()); + resultsPerTableKeyAndSpec.add(result.writeResult()); + LOG.debug( + "Added {}, specId={}, totalResults={}", + result, + result.specId(), + resultsPerTableKeyAndSpec.size()); } } diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java index cf5f423fd7ff..5153ec6a49ee 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriteResultSerializer.java @@ -41,6 +41,7 @@ public byte[] serialize(DynamicWriteResult writeResult) throws IOException { ByteArrayOutputStream out = new ByteArrayOutputStream(); DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); writeResult.key().serializeTo(view); + view.writeInt(writeResult.specId()); byte[] result = WRITE_RESULT_SERIALIZER.serialize(writeResult.writeResult()); view.write(result); return out.toByteArray(); @@ -50,11 +51,12 @@ public byte[] serialize(DynamicWriteResult writeResult) throws IOException { public DynamicWriteResult deserialize(int version, byte[] serialized) throws IOException { if (version == 1) { DataInputDeserializer view = new DataInputDeserializer(serialized); - WriteTarget key = WriteTarget.deserializeFrom(view); + TableKey key = TableKey.deserializeFrom(view); + int specId = view.readInt(); byte[] resultBuf = new byte[view.available()]; view.read(resultBuf); WriteResult writeResult = WRITE_RESULT_SERIALIZER.deserialize(version, resultBuf); - return new DynamicWriteResult(key, writeResult); + return new DynamicWriteResult(key, specId, writeResult); } throw new IOException("Unrecognized version or corrupt state: " + version); diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java index 907385797495..8425ea747fb7 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/DynamicWriter.java @@ -189,7 +189,11 @@ public Collection prepareCommit() throws IOException { writeResult.dataFiles().length, writeResult.deleteFiles().length); - result.add(new DynamicWriteResult(writeTarget, writeResult)); + result.add( + new DynamicWriteResult( + new TableKey(writeTarget.tableName(), writeTarget.branch()), + writeTarget.specId(), + writeResult)); } writers.clear(); diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java new file mode 100644 index 000000000000..08b755fe14a8 --- /dev/null +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/TableKey.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.dynamic; + +import java.io.IOException; +import java.util.Objects; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +class TableKey { + private final String tableName; + private final String branch; + + TableKey(String tableName, String branch) { + this.tableName = tableName; + this.branch = branch; + } + + TableKey(DynamicCommittable committable) { + this.tableName = committable.key().tableName(); + this.branch = committable.key().branch(); + } + + String tableName() { + return tableName; + } + + String branch() { + return branch; + } + + void serializeTo(DataOutputView view) throws IOException { + view.writeUTF(tableName); + view.writeUTF(branch); + } + + static TableKey deserializeFrom(DataInputView view) throws IOException { + return new TableKey(view.readUTF(), view.readUTF()); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } + + if (other == null || getClass() != other.getClass()) { + return false; + } + + TableKey that = (TableKey) other; + return tableName.equals(that.tableName) && branch.equals(that.branch); + } + + @Override + public int hashCode() { + return Objects.hash(tableName, branch); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableName", tableName) + .add("branch", branch) + .toString(); + } +} diff --git a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java index beb244b72eab..211f28c6518a 100644 --- a/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java +++ b/flink/v2.0/flink/src/main/java/org/apache/iceberg/flink/sink/dynamic/WriteTarget.java @@ -23,7 +23,6 @@ import java.util.Objects; import java.util.Set; import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -76,18 +75,6 @@ Set equalityFields() { return equalityFields; } - void serializeTo(DataOutputView view) throws IOException { - view.writeUTF(tableName); - view.writeUTF(branch); - view.writeInt(schemaId); - view.writeInt(specId); - view.writeBoolean(upsertMode); - view.writeInt(equalityFields.size()); - for (Integer equalityField : equalityFields) { - view.writeInt(equalityField); - } - } - static WriteTarget deserializeFrom(DataInputView view) throws IOException { return new WriteTarget( view.readUTF(), diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java index 13a06d362717..16890d1f63d0 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommittableSerializer.java @@ -21,42 +21,94 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.util.Set; import org.apache.flink.api.common.JobID; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.junit.jupiter.api.Test; class TestDynamicCommittableSerializer { + private static final DynamicCommittable COMMITTABLE = + new DynamicCommittable( + new TableKey("table", "branch"), + new byte[][] {{3, 4}, {5, 6}}, + JobID.generate().toHexString(), + new OperatorID().toHexString(), + 5); @Test - void testRoundtrip() throws IOException { - DynamicCommittable committable = + void testV1() throws IOException { + var committable = new DynamicCommittable( - new WriteTarget("table", "branch", 42, 23, false, Sets.newHashSet(1, 2)), - new byte[] {3, 4}, + new TableKey("table", "branch"), + new byte[][] {{3, 4}}, JobID.generate().toHexString(), new OperatorID().toHexString(), 5); - DynamicCommittableSerializer serializer = new DynamicCommittableSerializer(); - assertThat(serializer.deserialize(serializer.getVersion(), serializer.serialize(committable))) - .isEqualTo(committable); + assertThat(serializer.deserialize(1, serializeV1(committable))).isEqualTo(committable); } @Test - void testUnsupportedVersion() throws IOException { - DynamicCommittable committable = - new DynamicCommittable( - new WriteTarget("table", "branch", 42, 23, false, Sets.newHashSet(1, 2)), - new byte[] {3, 4}, - JobID.generate().toHexString(), - new OperatorID().toHexString(), - 5); + void testLatestVersion() throws IOException { + DynamicCommittableSerializer serializer = new DynamicCommittableSerializer(); + assertThat(serializer.deserialize(serializer.getVersion(), serializer.serialize(COMMITTABLE))) + .isEqualTo(COMMITTABLE); + } + @Test + void testUnsupportedVersion() { DynamicCommittableSerializer serializer = new DynamicCommittableSerializer(); - assertThatThrownBy(() -> serializer.deserialize(-1, serializer.serialize(committable))) + assertThatThrownBy(() -> serializer.deserialize(-1, serializer.serialize(COMMITTABLE))) .hasMessage("Unrecognized version or corrupt state: -1") .isInstanceOf(IOException.class); } + + byte[] serializeV1(DynamicCommittable committable) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + + // Wrap TableKey into a testing WriteTarget to match the V1 format + WriteTarget writeTarget = + new WriteTarget( + committable.key().tableName(), + committable.key().branch(), + -1, + -1, + false, + Set.of(1, 2, 3)); + view.write(serializeV1(writeTarget)); + + view.writeUTF(committable.jobId()); + view.writeUTF(committable.operatorId()); + view.writeLong(committable.checkpointId()); + + Preconditions.checkArgument( + committable.manifests().length == 1, + "V1 serialization format must have only one manifest per committable."); + view.writeInt(committable.manifests()[0].length); + view.write(committable.manifests()[0]); + + return out.toByteArray(); + } + + byte[] serializeV1(WriteTarget writeTarget) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + + view.writeUTF(writeTarget.tableName()); + view.writeUTF(writeTarget.branch()); + view.writeInt(writeTarget.schemaId()); + view.writeInt(writeTarget.specId()); + view.writeBoolean(writeTarget.upsertMode()); + view.writeInt(writeTarget.equalityFields().size()); + for (Integer equalityField : writeTarget.equalityFields()) { + view.writeInt(equalityField); + } + + return out.toByteArray(); + } } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java index 1497458e6083..24832c48be1a 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicCommitter.java @@ -54,6 +54,7 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; 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.relocated.com.google.common.collect.Sets; import org.assertj.core.api.ThrowableAssert.ThrowingCallable; @@ -125,6 +126,15 @@ class TestDynamicCommitter { .ofPositionDeletes() .build(); + private static final Map> WRITE_RESULT_BY_SPEC = + Map.of( + DATA_FILE.specId(), + Lists.newArrayList(WriteResult.builder().addDataFiles(DATA_FILE).build())); + private static final Map> WRITE_RESULT_BY_SPEC_2 = + Map.of( + DATA_FILE_2.specId(), + Lists.newArrayList(WriteResult.builder().addDataFiles(DATA_FILE_2).build())); + @BeforeEach void before() { catalog = CATALOG_EXTENSION.catalog(); @@ -155,12 +165,9 @@ void testCommit() throws Exception { sinkId, committerMetrics); - WriteTarget writeTarget1 = - new WriteTarget(TABLE1, "branch", 42, 0, true, Sets.newHashSet(1, 2)); - WriteTarget writeTarget2 = - new WriteTarget(TABLE1, "branch2", 43, 0, true, Sets.newHashSet(1, 2)); - WriteTarget writeTarget3 = - new WriteTarget(TABLE2, "branch2", 43, 0, true, Sets.newHashSet(1, 2)); + TableKey tableKey1 = new TableKey(TABLE1, "branch"); + TableKey tableKey2 = new TableKey(TABLE1, "branch2"); + TableKey tableKey3 = new TableKey(TABLE2, "branch2"); DynamicWriteResultAggregator aggregator = new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize); @@ -168,27 +175,12 @@ void testCommit() throws Exception { new OneInputStreamOperatorTestHarness(aggregator); aggregatorHarness.open(); - byte[] deltaManifest1 = - aggregator.writeToManifest( - writeTarget1, - Sets.newHashSet( - new DynamicWriteResult( - writeTarget1, WriteResult.builder().addDataFiles(DATA_FILE).build())), - 0); - byte[] deltaManifest2 = - aggregator.writeToManifest( - writeTarget2, - Sets.newHashSet( - new DynamicWriteResult( - writeTarget2, WriteResult.builder().addDataFiles(DATA_FILE).build())), - 0); - byte[] deltaManifest3 = - aggregator.writeToManifest( - writeTarget3, - Sets.newHashSet( - new DynamicWriteResult( - writeTarget3, WriteResult.builder().addDataFiles(DATA_FILE).build())), - 0); + byte[][] deltaManifests1 = + aggregator.writeToManifests(tableKey1.tableName(), WRITE_RESULT_BY_SPEC, 0); + byte[][] deltaManifests2 = + aggregator.writeToManifests(tableKey2.tableName(), WRITE_RESULT_BY_SPEC, 0); + byte[][] deltaManifests3 = + aggregator.writeToManifests(tableKey3.tableName(), WRITE_RESULT_BY_SPEC, 0); final String jobId = JobID.generate().toHexString(); final String operatorId = new OperatorID().toHexString(); @@ -196,15 +188,15 @@ void testCommit() throws Exception { CommitRequest commitRequest1 = new MockCommitRequest<>( - new DynamicCommittable(writeTarget1, deltaManifest1, jobId, operatorId, checkpointId)); + new DynamicCommittable(tableKey1, deltaManifests1, jobId, operatorId, checkpointId)); CommitRequest commitRequest2 = new MockCommitRequest<>( - new DynamicCommittable(writeTarget2, deltaManifest2, jobId, operatorId, checkpointId)); + new DynamicCommittable(tableKey2, deltaManifests2, jobId, operatorId, checkpointId)); CommitRequest commitRequest3 = new MockCommitRequest<>( - new DynamicCommittable(writeTarget3, deltaManifest3, jobId, operatorId, checkpointId)); + new DynamicCommittable(tableKey3, deltaManifests3, jobId, operatorId, checkpointId)); dynamicCommitter.commit(Sets.newHashSet(commitRequest1, commitRequest2, commitRequest3)); @@ -267,7 +259,7 @@ void testCommit() throws Exception { } @Test - void testAlreadyCommitted() throws Exception { + void testSkipsCommitRequestsForPreviousCheckpoints() throws Exception { Table table1 = catalog.loadTable(TableIdentifier.of(TABLE1)); assertThat(table1.snapshots()).isEmpty(); @@ -285,8 +277,7 @@ void testAlreadyCommitted() throws Exception { sinkId, committerMetrics); - WriteTarget writeTarget = - new WriteTarget(TABLE1, "branch", 42, 0, false, Sets.newHashSet(1, 2)); + TableKey tableKey = new TableKey(TABLE1, "branch"); DynamicWriteResultAggregator aggregator = new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize); @@ -298,24 +289,18 @@ void testAlreadyCommitted() throws Exception { final String operatorId = new OperatorID().toHexString(); final int checkpointId = 10; - byte[] deltaManifest = - aggregator.writeToManifest( - writeTarget, - Sets.newHashSet( - new DynamicWriteResult( - writeTarget, WriteResult.builder().addDataFiles(DATA_FILE).build())), - checkpointId); + byte[][] deltaManifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC, 0); CommitRequest commitRequest = new MockCommitRequest<>( - new DynamicCommittable(writeTarget, deltaManifest, jobId, operatorId, checkpointId)); + new DynamicCommittable(tableKey, deltaManifests, jobId, operatorId, checkpointId)); dynamicCommitter.commit(Sets.newHashSet(commitRequest)); CommitRequest oldCommitRequest = new MockCommitRequest<>( - new DynamicCommittable( - writeTarget, deltaManifest, jobId, operatorId, checkpointId - 1)); + new DynamicCommittable(tableKey, deltaManifests, jobId, operatorId, checkpointId - 1)); // Old commits requests shouldn't affect the result dynamicCommitter.commit(Sets.newHashSet(oldCommitRequest)); @@ -360,25 +345,23 @@ void testCommitDeleteInDifferentFormatVersion() throws Exception { sinkId, committerMetrics); - WriteTarget writeTarget = - new WriteTarget(TABLE1, "branch", 42, 0, false, Sets.newHashSet(1, 2)); - DynamicWriteResultAggregator aggregator = new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize); OneInputStreamOperatorTestHarness aggregatorHarness = new OneInputStreamOperatorTestHarness(aggregator); aggregatorHarness.open(); + TableKey tableKey = new TableKey(TABLE1, "branch"); final String jobId = JobID.generate().toHexString(); final String operatorId = new OperatorID().toHexString(); final int checkpointId = 10; - byte[] deltaManifest = - aggregator.writeToManifest( - writeTarget, - Sets.newHashSet( - new DynamicWriteResult( - writeTarget, + byte[][] deltaManifests = + aggregator.writeToManifests( + tableKey.tableName(), + Map.of( + DATA_FILE.specId(), + Sets.newHashSet( WriteResult.builder() .addDataFiles(DATA_FILE) .addDeleteFiles(DELETE_FILE) @@ -387,7 +370,7 @@ void testCommitDeleteInDifferentFormatVersion() throws Exception { CommitRequest commitRequest = new MockCommitRequest<>( - new DynamicCommittable(writeTarget, deltaManifest, jobId, operatorId, checkpointId)); + new DynamicCommittable(tableKey, deltaManifests, jobId, operatorId, checkpointId)); // Upgrade the table version UpdateProperties updateApi = table1.updateProperties(); @@ -421,8 +404,7 @@ void testCommitOnlyDataInDifferentFormatVersion() throws Exception { sinkId, committerMetrics); - WriteTarget writeTarget = - new WriteTarget(TABLE1, "branch", 42, 0, false, Sets.newHashSet(1, 2)); + TableKey tableKey = new TableKey(TABLE1, "branch"); DynamicWriteResultAggregator aggregator = new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize); @@ -434,17 +416,12 @@ void testCommitOnlyDataInDifferentFormatVersion() throws Exception { final String operatorId = new OperatorID().toHexString(); final int checkpointId = 10; - byte[] deltaManifest = - aggregator.writeToManifest( - writeTarget, - Sets.newHashSet( - new DynamicWriteResult( - writeTarget, WriteResult.builder().addDataFiles(DATA_FILE).build())), - checkpointId); + byte[][] deltaManifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC, checkpointId); CommitRequest commitRequest = new MockCommitRequest<>( - new DynamicCommittable(writeTarget, deltaManifest, jobId, operatorId, checkpointId)); + new DynamicCommittable(tableKey, deltaManifests, jobId, operatorId, checkpointId)); dynamicCommitter.commit(Sets.newHashSet(commitRequest)); @@ -486,50 +463,34 @@ void testTableBranchAtomicCommitForAppendOnlyData() throws Exception { new OneInputStreamOperatorTestHarness(aggregator); aggregatorHarness.open(); - WriteTarget writeTarget1 = - new WriteTarget(TABLE1, "branch1", 42, 0, true, Sets.newHashSet(1, 2)); - // writeTarget2 has a different schema - WriteTarget writeTarget2 = new WriteTarget(TABLE1, "branch1", 23, 0, true, Sets.newHashSet()); - // Different branch for writeTarget3 - WriteTarget writeTarget3 = new WriteTarget(TABLE1, "branch2", 23, 0, true, Sets.newHashSet()); - - WriteResult writeResult1 = WriteResult.builder().addDataFiles(DATA_FILE).build(); - WriteResult writeResult2 = WriteResult.builder().addDataFiles(DATA_FILE_2).build(); + TableKey tableKey1 = new TableKey(TABLE1, "branch1"); + TableKey tableKey2 = new TableKey(TABLE1, "branch2"); final String jobId = JobID.generate().toHexString(); final String operatorId = new OperatorID().toHexString(); final int checkpointId1 = 1; final int checkpointId2 = 2; - byte[] deltaManifest1 = - aggregator.writeToManifest( - writeTarget1, - Sets.newHashSet(new DynamicWriteResult(writeTarget1, writeResult1)), - checkpointId1); + byte[][] deltaManifests1 = + aggregator.writeToManifests(tableKey1.tableName(), WRITE_RESULT_BY_SPEC, checkpointId1); CommitRequest commitRequest1 = new MockCommitRequest<>( - new DynamicCommittable(writeTarget1, deltaManifest1, jobId, operatorId, checkpointId1)); + new DynamicCommittable(tableKey1, deltaManifests1, jobId, operatorId, checkpointId1)); - byte[] deltaManifest2 = - aggregator.writeToManifest( - writeTarget2, - Sets.newHashSet(new DynamicWriteResult(writeTarget2, writeResult2)), - checkpointId1); + byte[][] deltaManifests2 = + aggregator.writeToManifests(tableKey1.tableName(), WRITE_RESULT_BY_SPEC_2, checkpointId1); CommitRequest commitRequest2 = new MockCommitRequest<>( - new DynamicCommittable(writeTarget2, deltaManifest2, jobId, operatorId, checkpointId1)); + new DynamicCommittable(tableKey1, deltaManifests2, jobId, operatorId, checkpointId1)); - byte[] deltaManifest3 = - aggregator.writeToManifest( - writeTarget3, - Sets.newHashSet(new DynamicWriteResult(writeTarget3, writeResult2)), - checkpointId2); + byte[][] deltaManifests3 = + aggregator.writeToManifests(tableKey2.tableName(), WRITE_RESULT_BY_SPEC_2, checkpointId2); CommitRequest commitRequest3 = new MockCommitRequest<>( - new DynamicCommittable(writeTarget3, deltaManifest3, jobId, operatorId, checkpointId2)); + new DynamicCommittable(tableKey2, deltaManifests3, jobId, operatorId, checkpointId2)); boolean overwriteMode = false; int workerPoolSize = 1; @@ -601,49 +562,36 @@ void testTableBranchAtomicCommitWithFailures() throws Exception { new OneInputStreamOperatorTestHarness(aggregator); aggregatorHarness.open(); - WriteTarget writeTarget1 = new WriteTarget(TABLE1, "branch", 42, 0, false, Sets.newHashSet()); - // writeTarget2 has a different schema - WriteTarget writeTarget2 = new WriteTarget(TABLE1, "branch", 23, 0, false, Sets.newHashSet()); - WriteTarget writeTarget3 = new WriteTarget(TABLE1, "branch", 23, 0, false, Sets.newHashSet()); + TableKey tableKey = new TableKey(TABLE1, "branch"); - WriteResult writeResult1 = WriteResult.builder().addDataFiles(DATA_FILE).build(); - WriteResult writeResult2 = WriteResult.builder().addDeleteFiles(DELETE_FILE).build(); - WriteResult writeResult3 = WriteResult.builder().addDataFiles(DATA_FILE).build(); + Map> writeResults = + Map.of( + DELETE_FILE.specId(), + Lists.newArrayList(WriteResult.builder().addDeleteFiles(DELETE_FILE).build())); + + byte[][] deltaManifests1 = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC, 0); + byte[][] deltaManifests2 = aggregator.writeToManifests(tableKey.tableName(), writeResults, 0); + byte[][] deltaManifests3 = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC_2, 0); final String jobId = JobID.generate().toHexString(); final String operatorId = new OperatorID().toHexString(); final int checkpointId1 = 1; final int checkpointId2 = 2; - - byte[] deltaManifest1 = - aggregator.writeToManifest( - writeTarget1, - Sets.newHashSet(new DynamicWriteResult(writeTarget1, writeResult1)), - checkpointId1); + final int checkpointId3 = 3; CommitRequest commitRequest1 = new MockCommitRequest<>( - new DynamicCommittable(writeTarget1, deltaManifest1, jobId, operatorId, checkpointId1)); - - byte[] deltaManifest2 = - aggregator.writeToManifest( - writeTarget2, - Sets.newHashSet(new DynamicWriteResult(writeTarget2, writeResult2)), - checkpointId2); + new DynamicCommittable(tableKey, deltaManifests1, jobId, operatorId, checkpointId1)); CommitRequest commitRequest2 = new MockCommitRequest<>( - new DynamicCommittable(writeTarget2, deltaManifest2, jobId, operatorId, checkpointId2)); - - byte[] deltaManifest3 = - aggregator.writeToManifest( - writeTarget3, - Sets.newHashSet(new DynamicWriteResult(writeTarget3, writeResult3)), - checkpointId2); + new DynamicCommittable(tableKey, deltaManifests2, jobId, operatorId, checkpointId2)); CommitRequest commitRequest3 = new MockCommitRequest<>( - new DynamicCommittable(writeTarget3, deltaManifest3, jobId, operatorId, checkpointId2)); + new DynamicCommittable(tableKey, deltaManifests3, jobId, operatorId, checkpointId3)); boolean overwriteMode = false; int workerPoolSize = 1; @@ -693,10 +641,7 @@ void testTableBranchAtomicCommitWithFailures() throws Exception { } table.refresh(); - // Three committables, but only two snapshots! WriteResults from different checkpoints are not - // getting - // combined due to one writeResult2 containing a delete file. - assertThat(table.snapshots()).hasSize(2); + assertThat(table.snapshots()).hasSize(3); Snapshot snapshot1 = Iterables.getFirst(table.snapshots(), null); assertThat(snapshot1.summary()) @@ -720,18 +665,34 @@ void testTableBranchAtomicCommitWithFailures() throws Exception { assertThat(snapshot2.summary()) .containsAllEntriesOf( ImmutableMap.builder() - .put("added-data-files", "1") - .put("added-records", "42") .put("changed-partition-count", "1") .put("flink.job-id", jobId) .put("flink.max-committed-checkpoint-id", "" + checkpointId2) .put("flink.operator-id", operatorId) + .put("total-data-files", "1") + .put("total-delete-files", "1") + .put("total-equality-deletes", "0") + .put("total-files-size", "0") + .put("total-position-deletes", "24") + .put("total-records", "42") + .build()); + + Snapshot snapshot3 = Iterables.get(table.snapshots(), 2); + assertThat(snapshot3.summary()) + .containsAllEntriesOf( + ImmutableMap.builder() + .put("added-data-files", "1") + .put("added-records", "24") + .put("changed-partition-count", "1") + .put("flink.job-id", jobId) + .put("flink.max-committed-checkpoint-id", "" + checkpointId3) + .put("flink.operator-id", operatorId) .put("total-data-files", "2") .put("total-delete-files", "1") .put("total-equality-deletes", "0") .put("total-files-size", "0") .put("total-position-deletes", "24") - .put("total-records", "84") + .put("total-records", "66") .build()); } @@ -755,8 +716,7 @@ void testReplacePartitions() throws Exception { sinkId, committerMetrics); - WriteTarget writeTarget = - new WriteTarget(TABLE1, "branch", 42, 0, false, Sets.newHashSet(1, 2)); + TableKey tableKey = new TableKey(TABLE1, "branch"); DynamicWriteResultAggregator aggregator = new DynamicWriteResultAggregator(CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize); @@ -768,32 +728,22 @@ void testReplacePartitions() throws Exception { final String operatorId = new OperatorID().toHexString(); final int checkpointId = 10; - byte[] deltaManifest = - aggregator.writeToManifest( - writeTarget, - Sets.newHashSet( - new DynamicWriteResult( - writeTarget, WriteResult.builder().addDataFiles(DATA_FILE).build())), - checkpointId); + byte[][] deltaManifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC, 0); CommitRequest commitRequest = new MockCommitRequest<>( - new DynamicCommittable(writeTarget, deltaManifest, jobId, operatorId, checkpointId)); + new DynamicCommittable(tableKey, deltaManifests, jobId, operatorId, checkpointId)); dynamicCommitter.commit(Sets.newHashSet(commitRequest)); - byte[] overwriteManifest = - aggregator.writeToManifest( - writeTarget, - Sets.newHashSet( - new DynamicWriteResult( - writeTarget, WriteResult.builder().addDataFiles(DATA_FILE).build())), - checkpointId + 1); + byte[][] overwriteManifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC, 0); CommitRequest overwriteRequest = new MockCommitRequest<>( new DynamicCommittable( - writeTarget, overwriteManifest, jobId, operatorId, checkpointId + 1)); + tableKey, overwriteManifests, jobId, operatorId, checkpointId + 1)); dynamicCommitter.commit(Sets.newHashSet(overwriteRequest)); @@ -836,18 +786,13 @@ void testThrowsValidationExceptionOnDuplicateCommit(boolean overwriteMode) throw final int checkpointId = 1; final String branch = SnapshotRef.MAIN_BRANCH; - WriteTarget writeTarget = new WriteTarget(TABLE1, branch, 42, 0, false, Sets.newHashSet(1, 2)); - byte[] manifest = - aggregator.writeToManifest( - writeTarget, - Sets.newHashSet( - new DynamicWriteResult( - writeTarget, WriteResult.builder().addDataFiles(DATA_FILE).build())), - checkpointId); + TableKey tableKey = new TableKey(TABLE1, branch); + byte[][] manifests = + aggregator.writeToManifests(tableKey.tableName(), WRITE_RESULT_BY_SPEC, checkpointId); CommitRequest commitRequest1 = new MockCommitRequest<>( - new DynamicCommittable(writeTarget, manifest, jobId, operatorId, checkpointId)); + new DynamicCommittable(tableKey, manifests, jobId, operatorId, checkpointId)); Collection> commitRequests = Sets.newHashSet(commitRequest1); int workerPoolSize = 1; diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java index 0c07bc946189..a428571e9e28 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSink.java @@ -29,8 +29,11 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; +import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.StreamSupport; import javax.annotation.Nullable; import org.apache.flink.api.common.typeinfo.TypeHint; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -58,6 +61,7 @@ import org.apache.iceberg.FileFormat; 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.TableProperties; @@ -79,6 +83,7 @@ import org.apache.iceberg.io.CloseableIterable; 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.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.relocated.com.google.common.collect.Sets; @@ -486,6 +491,21 @@ void testPartitionSpecEvolution() throws Exception { SimpleDataUtil.SCHEMA, "t1", SnapshotRef.MAIN_BRANCH, spec2)); runTest(rows); + + // Validate the table has expected partition specs + Table table = CATALOG_EXTENSION.catalog().loadTable(TableIdentifier.of(DATABASE, "t1")); + Map tableSpecs = table.specs(); + List expectedSpecs = List.of(spec1, spec2, PartitionSpec.unpartitioned()); + + assertThat(tableSpecs).hasSize(expectedSpecs.size()); + expectedSpecs.forEach( + expectedSpec -> + assertThat( + tableSpecs.values().stream() + .anyMatch( + spec -> PartitionSpecEvolution.checkCompatibility(spec, expectedSpec))) + .withFailMessage("Table spec not found: %s.", expectedSpec) + .isTrue()); } @Test @@ -926,6 +946,79 @@ void testCommitsOnceWhenConcurrentDuplicateCommit(boolean overwriteMode) throws assertThat(totalAddedRecords).isEqualTo(records.size()); } + @Test + void testCommitsOncePerTableBranchAndCheckpoint() throws Exception { + String tableName = "t1"; + String branch = SnapshotRef.MAIN_BRANCH; + PartitionSpec spec1 = PartitionSpec.unpartitioned(); + PartitionSpec spec2 = PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("id", 10).build(); + Set equalityFields = Sets.newHashSet("id"); + + List inputRecords = + Lists.newArrayList( + // Two schemas + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, tableName, branch, spec1), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA2, tableName, branch, spec1), + // Two specs + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, tableName, branch, spec1), + new DynamicIcebergDataImpl(SimpleDataUtil.SCHEMA, tableName, branch, spec2), + // Some upserts + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, tableName, branch, spec1, true, equalityFields, false), + new DynamicIcebergDataImpl( + SimpleDataUtil.SCHEMA, tableName, branch, spec1, true, equalityFields, true)); + + executeDynamicSink(inputRecords, env, true, 1, null); + + List actualRecords; + try (CloseableIterable iterable = + IcebergGenerics.read( + CATALOG_EXTENSION.catalog().loadTable(TableIdentifier.of("default", "t1"))) + .build()) { + actualRecords = Lists.newArrayList(iterable); + } + + // Validate records + int expectedRecords = inputRecords.size() - 1; // 1 duplicate + assertThat(actualRecords).hasSize(expectedRecords); + + for (int i = 0; i < expectedRecords; i++) { + Record actual = actualRecords.get(0); + assertThat(inputRecords) + .anySatisfy( + inputRecord -> { + assertThat(actual.get(0)).isEqualTo(inputRecord.rowProvided.getField(0)); + assertThat(actual.get(1)).isEqualTo(inputRecord.rowProvided.getField(1)); + if (inputRecord.schemaProvided.equals(SimpleDataUtil.SCHEMA2)) { + assertThat(actual.get(2)).isEqualTo(inputRecord.rowProvided.getField(2)); + } + // There is an additional _pos field which gets added + }); + } + + TableIdentifier tableIdentifier = TableIdentifier.of("default", tableName); + Table table = CATALOG_EXTENSION.catalog().loadTable(tableIdentifier); + + Snapshot lastSnapshot = Iterables.getLast(table.snapshots()); + assertThat(lastSnapshot).isNotNull(); + assertThat(lastSnapshot.summary()) + .containsAllEntriesOf( + ImmutableMap.builder() + .put("total-equality-deletes", "1") + .put("total-position-deletes", "1") + .put("total-records", "6") + .build()); + + // Count commits per checkpoint + Map commitsPerCheckpoint = + StreamSupport.stream(table.snapshots().spliterator(), false) + .map(snapshot -> snapshot.summary().get("flink.max-committed-checkpoint-id")) + .filter(Objects::nonNull) + .map(Long::parseLong) + .collect(Collectors.groupingBy(Function.identity(), Collectors.counting())); + assertThat(commitsPerCheckpoint.values()).allMatch(count -> count == 1); + } + @Test void testOptInDropUnusedColumns() throws Exception { Schema schema1 = diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java index b9aa56ab2e61..41c1f67d2926 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicIcebergSinkPerf.java @@ -20,7 +20,6 @@ import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.apache.iceberg.flink.TestFixtures.TABLE; -import static org.apache.iceberg.flink.sink.dynamic.DynamicCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; import java.util.List; import java.util.function.Consumer; @@ -123,10 +122,7 @@ void before() { CATALOG_EXTENSION .catalog() .createTable( - IDENTIFIERS[i], - SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of(MAX_CONTINUOUS_EMPTY_COMMITS, "100000")); + IDENTIFIERS[i], SCHEMA, PartitionSpec.unpartitioned(), ImmutableMap.of()); table.manageSnapshots().createBranch(SnapshotRef.MAIN_BRANCH).commit(); } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java index 17a4d98b2938..f68e8dae174e 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultAggregator.java @@ -27,6 +27,7 @@ import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SinkV2Assertions; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.iceberg.DataFile; @@ -58,7 +59,7 @@ class TestDynamicWriteResultAggregator { .build(); @Test - void testAggregator() throws Exception { + void testAggregatesWriteResultsForTwoTables() throws Exception { CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("table"), new Schema()); CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("table2"), new Schema()); @@ -69,13 +70,12 @@ void testAggregator() throws Exception { testHarness = new OneInputStreamOperatorTestHarness<>(aggregator)) { testHarness.open(); - WriteTarget writeTarget1 = new WriteTarget("table", "branch", 42, 0, true, Sets.newHashSet()); + TableKey tableKey1 = new TableKey("table", "branch"); DynamicWriteResult dynamicWriteResult1 = - new DynamicWriteResult(writeTarget1, WriteResult.builder().build()); - WriteTarget writeTarget2 = - new WriteTarget("table2", "branch", 42, 0, true, Sets.newHashSet(1, 2)); + new DynamicWriteResult(tableKey1, -1, WriteResult.builder().build()); + TableKey tableKey2 = new TableKey("table2", "branch"); DynamicWriteResult dynamicWriteResult2 = - new DynamicWriteResult(writeTarget2, WriteResult.builder().build()); + new DynamicWriteResult(tableKey2, -1, WriteResult.builder().build()); CommittableWithLineage committable1 = new CommittableWithLineage<>(dynamicWriteResult1, 0, 0); @@ -113,18 +113,16 @@ void testPreventOutputFileFactoryCacheEvictionDuringFlush() throws Exception { testHarness = new OneInputStreamOperatorTestHarness<>(aggregator)) { testHarness.open(); - WriteTarget writeTarget1 = - new WriteTarget("table", "branch", 42, 0, false, Sets.newHashSet()); + TableKey tableKey1 = new TableKey("table", "branch"); DynamicWriteResult dynamicWriteResult1 = new DynamicWriteResult( - writeTarget1, WriteResult.builder().addDataFiles(DATA_FILE).build()); + tableKey1, DATA_FILE.specId(), WriteResult.builder().addDataFiles(DATA_FILE).build()); - // Different WriteTarget - WriteTarget writeTarget2 = - new WriteTarget("table", "branch2", 23, 0, true, Sets.newHashSet()); + // Different TableKey + TableKey tableKey2 = new TableKey("table", "branch2"); DynamicWriteResult dynamicWriteResult2 = new DynamicWriteResult( - writeTarget2, WriteResult.builder().addDataFiles(DATA_FILE).build()); + tableKey2, DATA_FILE.specId(), WriteResult.builder().addDataFiles(DATA_FILE).build()); CommittableWithLineage committable1 = new CommittableWithLineage<>(dynamicWriteResult1, 0, 0); @@ -151,6 +149,70 @@ void testPreventOutputFileFactoryCacheEvictionDuringFlush() throws Exception { } } + @Test + void testAggregatesWriteResultsForOneTable() throws Exception { + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("table"), new Schema()); + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("table2"), new Schema()); + + long checkpointId = 1L; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = + new OneInputStreamOperatorTestHarness<>( + new DynamicWriteResultAggregator( + CATALOG_EXTENSION.catalogLoader(), cacheMaximumSize))) { + testHarness.open(); + + TableKey tableKey = new TableKey("table", "branch"); + DataFile dataFile1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/data-1.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + DataFile dataFile2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/data-2.parquet") + .withFileSizeInBytes(20) + .withRecordCount(2) + .build(); + + testHarness.processElement( + createRecord(tableKey, checkpointId, dataFile1.specId(), dataFile1)); + testHarness.processElement( + createRecord(tableKey, checkpointId, dataFile2.specId(), dataFile2)); + + assertThat(testHarness.getOutput()).isEmpty(); + + testHarness.prepareSnapshotPreBarrier(checkpointId); + + List> outputValues = testHarness.extractOutputValues(); + // Contains a CommittableSummary + DynamicCommittable + assertThat(outputValues).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(outputValues.get(0))) + .hasOverallCommittables(1) + .hasFailedCommittables(0) + .hasCheckpointId(checkpointId); + + CommittableWithLineage committable = + extractAndAssertCommittableWithLineage(outputValues.get(1)); + + SinkV2Assertions.assertThat(committable).hasCheckpointId(checkpointId); + + DynamicCommittable dynamicCommittable = committable.getCommittable(); + + assertThat(dynamicCommittable.manifests()).hasNumberOfRows(1); + assertThat(dynamicCommittable.key()).isEqualTo(tableKey); + assertThat(dynamicCommittable.checkpointId()).isEqualTo(checkpointId); + assertThat(dynamicCommittable.jobId()) + .isEqualTo(testHarness.getEnvironment().getJobID().toString()); + assertThat(dynamicCommittable.operatorId()) + .isEqualTo(testHarness.getOperator().getOperatorID().toString()); + } + } + private static Set getManifestPaths( List>> messages) throws IOException { Set manifestPaths = Sets.newHashSet(); @@ -158,15 +220,40 @@ private static Set getManifestPaths( for (StreamRecord> record : messages) { CommittableMessage message = record.getValue(); if (message instanceof CommittableWithLineage) { - DeltaManifests deltaManifests = - SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, - (((CommittableWithLineage) message).getCommittable()) - .manifest()); - deltaManifests.manifests().forEach(manifest -> manifestPaths.add(manifest.path())); + for (byte[] manifest : + (((CommittableWithLineage) message).getCommittable()).manifests()) { + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, manifest); + deltaManifests + .manifests() + .forEach(manifestFile -> manifestPaths.add(manifestFile.path())); + } } } return manifestPaths; } + + private static StreamRecord> createRecord( + TableKey tableKey, long checkpointId, int specId, DataFile... dataFiles) { + return new StreamRecord<>( + new CommittableWithLineage<>( + new DynamicWriteResult( + tableKey, specId, WriteResult.builder().addDataFiles(dataFiles).build()), + checkpointId, + 0)); + } + + static CommittableSummary extractAndAssertCommittableSummary( + CommittableMessage message) { + assertThat(message).isInstanceOf(CommittableSummary.class); + return (CommittableSummary) message; + } + + static CommittableWithLineage extractAndAssertCommittableWithLineage( + CommittableMessage message) { + assertThat(message).isInstanceOf(CommittableWithLineage.class); + return (CommittableWithLineage) message; + } } diff --git a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java index b1bca6099d7a..35dc45d8fd41 100644 --- a/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java +++ b/flink/v2.0/flink/src/test/java/org/apache/iceberg/flink/sink/dynamic/TestDynamicWriteResultSerializer.java @@ -29,7 +29,6 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.junit.jupiter.api.Test; class TestDynamicWriteResultSerializer { @@ -48,13 +47,12 @@ class TestDynamicWriteResultSerializer { ImmutableMap.of(1, ByteBuffer.allocate(1)), ImmutableMap.of(1, ByteBuffer.allocate(1)))) .build(); + private static final TableKey TABLE_KEY = new TableKey("table", "branch"); @Test void testRoundtrip() throws IOException { DynamicWriteResult dynamicWriteResult = - new DynamicWriteResult( - new WriteTarget("table", "branch", 42, 23, false, Sets.newHashSet(1, 2)), - WriteResult.builder().addDataFiles(DATA_FILE).build()); + new DynamicWriteResult(TABLE_KEY, 1, WriteResult.builder().addDataFiles(DATA_FILE).build()); DynamicWriteResultSerializer serializer = new DynamicWriteResultSerializer(); DynamicWriteResult copy = @@ -68,11 +66,9 @@ void testRoundtrip() throws IOException { } @Test - void testUnsupportedVersion() throws IOException { + void testUnsupportedVersion() { DynamicWriteResult dynamicWriteResult = - new DynamicWriteResult( - new WriteTarget("table", "branch", 42, 23, false, Sets.newHashSet(1, 2)), - WriteResult.builder().addDataFiles(DATA_FILE).build()); + new DynamicWriteResult(TABLE_KEY, 1, WriteResult.builder().addDataFiles(DATA_FILE).build()); DynamicWriteResultSerializer serializer = new DynamicWriteResultSerializer(); assertThatThrownBy(() -> serializer.deserialize(-1, serializer.serialize(dynamicWriteResult))) From a1c1c1b5a855a0ce63785f62660cf2490765d4e0 Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 9 Jan 2026 12:26:14 +0100 Subject: [PATCH 173/201] Core: Support case-insensitive field lookups in SchemaUpdate (#14734) --- .../java/org/apache/iceberg/SchemaUpdate.java | 13 +++- .../org/apache/iceberg/TestSchemaUpdate.java | 76 +++++++++++++++++++ 2 files changed, 85 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/SchemaUpdate.java b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java index 73d84b2fba0f..e42df2fe5ed3 100644 --- a/core/src/main/java/org/apache/iceberg/SchemaUpdate.java +++ b/core/src/main/java/org/apache/iceberg/SchemaUpdate.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.LinkedList; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -164,7 +165,7 @@ private void internalAddColumn( int newId = assignNewColumnId(); // update tracking for moves - addedNameToId.put(fullName, newId); + addedNameToId.put(caseSensitivityAwareName(fullName), newId); if (parentId != TABLE_ROOT_ID) { idToParent.put(newId, parentId); } @@ -391,7 +392,7 @@ public UpdateSchema caseSensitive(boolean caseSensitivity) { } private boolean isAdded(String name) { - return addedNameToId.containsKey(name); + return addedNameToId.containsKey(caseSensitivityAwareName(name)); } private Types.NestedField findForUpdate(String name) { @@ -405,7 +406,7 @@ private Types.NestedField findForUpdate(String name) { return existing; } - Integer addedId = addedNameToId.get(name); + Integer addedId = addedNameToId.get(caseSensitivityAwareName(name)); if (addedId != null) { return updates.get(addedId); } @@ -414,7 +415,7 @@ private Types.NestedField findForUpdate(String name) { } private Integer findForMove(String name) { - Integer addedId = addedNameToId.get(name); + Integer addedId = addedNameToId.get(caseSensitivityAwareName(name)); if (addedId != null) { return addedId; } @@ -870,4 +871,8 @@ public MoveType type() { private Types.NestedField findField(String fieldName) { return caseSensitive ? schema.findField(fieldName) : schema.caseInsensitiveFindField(fieldName); } + + private String caseSensitivityAwareName(String name) { + return caseSensitive ? name : name.toLowerCase(Locale.ROOT); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java index 3b9ef1871da7..fb942dde2aa2 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java @@ -2503,4 +2503,80 @@ public void testAddRequiredUnknown() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create required field with unknown type: unk"); } + + @Test + public void testCaseInsensitiveAddTopLevelAndMove() { + Schema schema = new Schema(required(1, "id", Types.LongType.get())); + + Schema expected = + new Schema( + optional(2, "data", Types.StringType.get()), required(1, "id", Types.LongType.get())); + + Schema actual = + new SchemaUpdate(schema, schema.highestFieldId()) + .caseSensitive(false) + .addColumn("data", Types.StringType.get()) + .moveFirst("dAtA") + .apply(); + + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); + } + + @Test + public void testCaseInsensitiveAddNestedAndMove() { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), + optional( + 2, "struct", Types.StructType.of(required(3, "field1", Types.StringType.get())))); + + Schema expected = + new Schema( + required(1, "id", Types.LongType.get()), + optional( + 2, + "struct", + Types.StructType.of( + optional(4, "field2", Types.IntegerType.get()), + required(3, "field1", Types.StringType.get())))); + + Schema actual = + new SchemaUpdate(schema, schema.highestFieldId()) + .caseSensitive(false) + .addColumn("STRUCT", "field2", Types.IntegerType.get()) + .moveFirst("STRUCT.FIELD2") + .apply(); + + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); + } + + @Test + public void testCaseInsensitiveMoveAfterNewlyAddedField() { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), + optional( + 2, "struct", Types.StructType.of(required(3, "field1", Types.StringType.get())))); + + Schema expected = + new Schema( + required(1, "id", Types.LongType.get()), + optional( + 2, + "struct", + Types.StructType.of( + required(3, "field1", Types.StringType.get()), + optional(4, "field2", Types.IntegerType.get()), + optional(5, "field3", Types.DoubleType.get())))); + + Schema actual = + new SchemaUpdate(schema, schema.highestFieldId()) + .caseSensitive(false) + .addColumn("struct", "field2", Types.IntegerType.get()) + .addColumn("STRUCT", "field3", Types.DoubleType.get()) + .moveAfter("STRUCT.FIELD3", "struct.FIELD2") + .apply(); + + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); + } } From d85f8a87a8b9c497c9235199cc3787b8814cc270 Mon Sep 17 00:00:00 2001 From: Daniel Weeks Date: Fri, 9 Jan 2026 08:40:24 -0800 Subject: [PATCH 174/201] Kafka Connect: validate table uuid on commit (#14979) * Kafka Connect: validate table uuid on commit * Fix test TableReference usage * Suppress complexity warning * Update to check the table uuid against the ref uuid * Add more complete uuid checking --- .../connect/events/TableReference.java | 52 +++++++++++++++++-- .../events/TestEventSerialization.java | 6 +-- .../iceberg/connect/channel/Coordinator.java | 9 ++++ .../iceberg/connect/channel/Worker.java | 3 +- .../iceberg/connect/data/IcebergWriter.java | 12 ++--- .../connect/data/IcebergWriterFactory.java | 12 ++++- .../connect/data/IcebergWriterResult.java | 32 ++++++++++-- .../iceberg/connect/data/RecordUtils.java | 5 +- .../connect/channel/TestCoordinator.java | 3 +- .../iceberg/connect/data/TestSinkWriter.java | 6 +-- .../iceberg/connect/data/WriterTestBase.java | 7 ++- 11 files changed, 121 insertions(+), 26 deletions(-) diff --git a/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java b/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java index f30eac892400..cd1341822c58 100644 --- a/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java +++ b/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java @@ -21,6 +21,7 @@ import java.util.Arrays; import java.util.List; import java.util.Objects; +import java.util.UUID; import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -32,6 +33,7 @@ import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StringType; import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.types.Types.UUIDType; /** Element representing a table identifier, with namespace and name. */ public class TableReference implements IndexedRecord { @@ -39,23 +41,39 @@ public class TableReference implements IndexedRecord { private String catalog; private List namespace; private String name; + private UUID uuid; private final Schema avroSchema; static final int CATALOG = 10_600; static final int NAMESPACE = 10_601; static final int NAME = 10_603; + static final int TABLE_UUID = 10_604; public static final StructType ICEBERG_SCHEMA = StructType.of( NestedField.required(CATALOG, "catalog", StringType.get()), NestedField.required( NAMESPACE, "namespace", ListType.ofRequired(NAMESPACE + 1, StringType.get())), - NestedField.required(NAME, "name", StringType.get())); + NestedField.required(NAME, "name", StringType.get()), + NestedField.optional(TABLE_UUID, "table_uuid", UUIDType.get())); private static final Schema AVRO_SCHEMA = AvroUtil.convert(ICEBERG_SCHEMA, TableReference.class); + /** + * @deprecated since 1.11.0, will be removed in 1.12.0; use {@link TableReference#of(String, + * TableIdentifier, UUID)} + */ + @Deprecated public static TableReference of(String catalog, TableIdentifier tableIdentifier) { return new TableReference( - catalog, Arrays.asList(tableIdentifier.namespace().levels()), tableIdentifier.name()); + catalog, Arrays.asList(tableIdentifier.namespace().levels()), tableIdentifier.name(), null); + } + + public static TableReference of(String catalog, TableIdentifier tableIdentifier, UUID tableUuid) { + return new TableReference( + catalog, + Arrays.asList(tableIdentifier.namespace().levels()), + tableIdentifier.name(), + tableUuid); } // Used by Avro reflection to instantiate this class when reading events @@ -63,6 +81,11 @@ public TableReference(Schema avroSchema) { this.avroSchema = avroSchema; } + /** + * @deprecated since 1.11.0, will be removed in 1.12.0; use {@link TableReference#of(String, + * TableIdentifier, UUID)}. + */ + @Deprecated public TableReference(String catalog, List namespace, String name) { Preconditions.checkNotNull(catalog, "Catalog cannot be null"); Preconditions.checkNotNull(namespace, "Namespace cannot be null"); @@ -73,10 +96,25 @@ public TableReference(String catalog, List namespace, String name) { this.avroSchema = AVRO_SCHEMA; } + private TableReference(String catalog, List namespace, String name, UUID uuid) { + Preconditions.checkNotNull(catalog, "Catalog cannot be null"); + Preconditions.checkNotNull(namespace, "Namespace cannot be null"); + Preconditions.checkNotNull(name, "Name cannot be null"); + this.catalog = catalog; + this.namespace = namespace; + this.name = name; + this.uuid = uuid; + this.avroSchema = AVRO_SCHEMA; + } + public String catalog() { return catalog; } + public UUID uuid() { + return uuid; + } + public TableIdentifier identifier() { Namespace icebergNamespace = Namespace.of(namespace.toArray(new String[0])); return TableIdentifier.of(icebergNamespace, name); @@ -103,6 +141,9 @@ public void put(int i, Object v) { case NAME: this.name = v == null ? null : v.toString(); return; + case TABLE_UUID: + this.uuid = (UUID) v; + return; default: // ignore the object, it must be from a newer version of the format } @@ -117,6 +158,8 @@ public Object get(int i) { return namespace; case NAME: return name; + case TABLE_UUID: + return uuid; default: throw new UnsupportedOperationException("Unknown field ordinal: " + i); } @@ -133,11 +176,12 @@ public boolean equals(Object o) { TableReference that = (TableReference) o; return Objects.equals(catalog, that.catalog) && Objects.equals(namespace, that.namespace) - && Objects.equals(name, that.name); + && Objects.equals(name, that.name) + && Objects.equals(uuid, that.uuid); } @Override public int hashCode() { - return Objects.hash(catalog, namespace, name); + return Objects.hash(catalog, namespace, name, uuid); } } diff --git a/kafka-connect/kafka-connect-events/src/test/java/org/apache/iceberg/connect/events/TestEventSerialization.java b/kafka-connect/kafka-connect-events/src/test/java/org/apache/iceberg/connect/events/TestEventSerialization.java index 10054c66400d..eea2e6373019 100644 --- a/kafka-connect/kafka-connect-events/src/test/java/org/apache/iceberg/connect/events/TestEventSerialization.java +++ b/kafka-connect/kafka-connect-events/src/test/java/org/apache/iceberg/connect/events/TestEventSerialization.java @@ -21,8 +21,8 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Arrays; -import java.util.Collections; import java.util.UUID; +import org.apache.iceberg.catalog.TableIdentifier; import org.junit.jupiter.api.Test; public class TestEventSerialization { @@ -50,7 +50,7 @@ public void testDataWrittenSerialization() { new DataWritten( EventTestUtil.SPEC.partitionType(), commitId, - new TableReference("catalog", Collections.singletonList("db"), "tbl"), + TableReference.of("catalog", TableIdentifier.of("db", "tbl"), UUID.randomUUID()), Arrays.asList(EventTestUtil.createDataFile(), EventTestUtil.createDataFile()), Arrays.asList(EventTestUtil.createDeleteFile(), EventTestUtil.createDeleteFile()))); @@ -97,7 +97,7 @@ public void testCommitToTableSerialization() { "cg-connector", new CommitToTable( commitId, - new TableReference("catalog", Collections.singletonList("db"), "tbl"), + TableReference.of("catalog", TableIdentifier.of("db", "tbl"), UUID.randomUUID()), 1L, EventTestUtil.now())); diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java index 1e0fa3286fe5..02a09db4764c 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java @@ -193,6 +193,7 @@ private String offsetsToJson(Map offsets) { } } + @SuppressWarnings("checkstyle:CyclomaticComplexity") private void commitToTable( TableReference tableReference, List envelopeList, @@ -207,6 +208,14 @@ private void commitToTable( return; } + if (!Objects.equals(table.uuid(), tableReference.uuid())) { + LOG.warn( + "Skipping commits to table {} due to target table mismatch. Expected: {} Received: {}", + tableIdentifier, + table.uuid(), + tableReference.uuid()); + } + String branch = config.tableConfig(tableIdentifier.toString()).commitBranch(); // Control topic partition offsets may include a subset of partition ids if there were no diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java index 27c5b9622fd3..903be7070370 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java @@ -32,7 +32,6 @@ import org.apache.iceberg.connect.events.Event; import org.apache.iceberg.connect.events.PayloadType; import org.apache.iceberg.connect.events.StartCommit; -import org.apache.iceberg.connect.events.TableReference; import org.apache.iceberg.connect.events.TopicPartitionOffset; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTaskContext; @@ -101,7 +100,7 @@ protected boolean receive(Envelope envelope) { new DataWritten( writeResult.partitionStruct(), commitId, - TableReference.of(config.catalogName(), writeResult.tableIdentifier()), + writeResult.tableReference(), writeResult.dataFiles(), writeResult.deleteFiles()))) .collect(Collectors.toList()); diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java index b5be5b3a0047..a1b47e8c0e92 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java @@ -24,8 +24,8 @@ import java.util.List; import java.util.Locale; import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.events.TableReference; import org.apache.iceberg.data.Record; import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.io.WriteResult; @@ -35,23 +35,23 @@ class IcebergWriter implements RecordWriter { private final Table table; - private final String tableName; + private final TableReference tableReference; private final IcebergSinkConfig config; private final List writerResults; private RecordConverter recordConverter; private TaskWriter writer; - IcebergWriter(Table table, String tableName, IcebergSinkConfig config) { + IcebergWriter(Table table, TableReference tableReference, IcebergSinkConfig config) { this.table = table; - this.tableName = tableName; + this.tableReference = tableReference; this.config = config; this.writerResults = Lists.newArrayList(); initNewWriter(); } private void initNewWriter() { - this.writer = RecordUtils.createTableWriter(table, tableName, config); + this.writer = RecordUtils.createTableWriter(table, tableReference, config); this.recordConverter = new RecordConverter(table, config); } @@ -107,7 +107,7 @@ private void flush() { writerResults.add( new IcebergWriterResult( - TableIdentifier.parse(tableName), + tableReference, Arrays.asList(writeResult.dataFiles()), Arrays.asList(writeResult.deleteFiles()), table.spec().partitionType())); diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java index 92f5af2d7a87..afb68f170136 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java @@ -20,6 +20,7 @@ import java.util.Arrays; import java.util.List; +import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; @@ -28,6 +29,7 @@ import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.events.TableReference; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.ForbiddenException; import org.apache.iceberg.exceptions.NoSuchTableException; @@ -67,7 +69,15 @@ RecordWriter createWriter(String tableName, SinkRecord sample, boolean ignoreMis } } - return new IcebergWriter(table, tableName, config); + UUID tableUuid = table.uuid(); + if (tableUuid == null) { + LOG.warn( + "Table {} does not have a UUID, this may cause issues with commit coordination on table replace", + identifier); + } + TableReference tableReference = TableReference.of(catalog.name(), identifier, tableUuid); + + return new IcebergWriter(table, tableReference, config); } @VisibleForTesting diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterResult.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterResult.java index 58695a5572b5..5667399cd74e 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterResult.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterResult.java @@ -22,28 +22,54 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.events.TableReference; import org.apache.iceberg.types.Types.StructType; public class IcebergWriterResult { - private final TableIdentifier tableIdentifier; + private final TableReference tableReference; private final List dataFiles; private final List deleteFiles; private final StructType partitionStruct; + public IcebergWriterResult( + TableReference tableReference, + List dataFiles, + List deleteFiles, + StructType partitionStruct) { + this.tableReference = tableReference; + this.dataFiles = dataFiles; + this.deleteFiles = deleteFiles; + this.partitionStruct = partitionStruct; + } + + /** + * @deprecated since 1.11.0, will be removed in 1.12.0; use {@link + * IcebergWriterResult#IcebergWriterResult(TableReference, List, List, StructType)} instead + */ + @Deprecated public IcebergWriterResult( TableIdentifier tableIdentifier, List dataFiles, List deleteFiles, StructType partitionStruct) { - this.tableIdentifier = tableIdentifier; + this.tableReference = TableReference.of("unknown", tableIdentifier); this.dataFiles = dataFiles; this.deleteFiles = deleteFiles; this.partitionStruct = partitionStruct; } + public TableReference tableReference() { + return tableReference; + } + + /** + * @deprecated since 1.11.0, will be removed in 1.12.0; use {@code tableReference().identifier()} + * instead + */ + @Deprecated public TableIdentifier tableIdentifier() { - return tableIdentifier; + return tableReference.identifier(); } public List dataFiles() { diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordUtils.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordUtils.java index d4bf4ce2a4d3..5ed820c8cbe1 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordUtils.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordUtils.java @@ -27,6 +27,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.events.TableReference; import org.apache.iceberg.data.GenericFileWriterFactory; import org.apache.iceberg.data.Record; import org.apache.iceberg.io.FileWriterFactory; @@ -95,7 +96,7 @@ private static Object valueFromMap(Map parent, List fields) { } public static TaskWriter createTableWriter( - Table table, String tableName, IcebergSinkConfig config) { + Table table, TableReference tableReference, IcebergSinkConfig config) { Map tableProps = Maps.newHashMap(table.properties()); tableProps.putAll(config.writeProps()); @@ -113,7 +114,7 @@ public static TaskWriter createTableWriter( Set identifierFieldIds = table.schema().identifierFieldIds(); // override the identifier fields if the config is set - List idCols = config.tableConfig(tableName).idColumns(); + List idCols = config.tableConfig(tableReference.identifier().name()).idColumns(); if (!idCols.isEmpty()) { identifierFieldIds = idCols.stream() diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java index 60a085781171..a8fbce7a8b01 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java @@ -33,6 +33,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.connect.events.AvroUtil; import org.apache.iceberg.connect.events.CommitComplete; import org.apache.iceberg.connect.events.CommitToTable; @@ -189,7 +190,7 @@ private UUID coordinatorTest( new DataWritten( StructType.of(), commitId, - new TableReference("catalog", ImmutableList.of("db"), "tbl"), + TableReference.of("catalog", TableIdentifier.of("db", "tbl"), UUID.randomUUID()), dataFiles, deleteFiles)); bytes = AvroUtil.encode(commitResponse); diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/TestSinkWriter.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/TestSinkWriter.java index a14ebcab7336..6baf72117d04 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/TestSinkWriter.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/TestSinkWriter.java @@ -91,7 +91,7 @@ public void testDefaultRoute() { List writerResults = sinkWriterTest(value, config); assertThat(writerResults).hasSize(1); IcebergWriterResult writerResult = writerResults.get(0); - assertThat(writerResult.tableIdentifier()).isEqualTo(TABLE_IDENTIFIER); + assertThat(writerResult.tableReference().identifier()).isEqualTo(TABLE_IDENTIFIER); } @Test @@ -119,7 +119,7 @@ public void testStaticRoute() { List writerResults = sinkWriterTest(value, config); assertThat(writerResults).hasSize(1); IcebergWriterResult writerResult = writerResults.get(0); - assertThat(writerResult.tableIdentifier()).isEqualTo(TABLE_IDENTIFIER); + assertThat(writerResult.tableReference().identifier()).isEqualTo(TABLE_IDENTIFIER); } @Test @@ -150,7 +150,7 @@ public void testDynamicRoute() { List writerResults = sinkWriterTest(value, config); assertThat(writerResults).hasSize(1); IcebergWriterResult writerResult = writerResults.get(0); - assertThat(writerResult.tableIdentifier()).isEqualTo(TABLE_IDENTIFIER); + assertThat(writerResult.tableReference().identifier()).isEqualTo(TABLE_IDENTIFIER); } @Test diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/WriterTestBase.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/WriterTestBase.java index d25bfde85c0b..30b60fb3c542 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/WriterTestBase.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/WriterTestBase.java @@ -25,11 +25,14 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.List; +import java.util.UUID; import org.apache.iceberg.LocationProviders; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.events.TableReference; import org.apache.iceberg.data.Record; import org.apache.iceberg.encryption.PlaintextEncryptionManager; import org.apache.iceberg.inmemory.InMemoryFileIO; @@ -73,7 +76,9 @@ public void before() { protected WriteResult writeTest( List rows, IcebergSinkConfig config, Class expectedWriterClass) { - try (TaskWriter writer = RecordUtils.createTableWriter(table, "name", config)) { + TableReference tableReference = + TableReference.of("test_catalog", TableIdentifier.of("name"), UUID.randomUUID()); + try (TaskWriter writer = RecordUtils.createTableWriter(table, tableReference, config)) { assertThat(writer.getClass()).isEqualTo(expectedWriterClass); rows.forEach( From b2696b9599df762feb3749eef2383b7698437374 Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Fri, 9 Jan 2026 13:20:02 -0800 Subject: [PATCH 175/201] Kafka Connect: fix table UUID check (#15011) --- .../java/org/apache/iceberg/connect/channel/Coordinator.java | 3 ++- .../org/apache/iceberg/connect/channel/TestCoordinator.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java index 02a09db4764c..068e1e1f6e9c 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java @@ -208,12 +208,13 @@ private void commitToTable( return; } - if (!Objects.equals(table.uuid(), tableReference.uuid())) { + if (tableReference.uuid() != null && !tableReference.uuid().equals(table.uuid())) { LOG.warn( "Skipping commits to table {} due to target table mismatch. Expected: {} Received: {}", tableIdentifier, table.uuid(), tableReference.uuid()); + return; } String branch = config.tableConfig(tableIdentifier.toString()).commitBranch(); diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java index a8fbce7a8b01..05526eab5c68 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/TestCoordinator.java @@ -190,7 +190,7 @@ private UUID coordinatorTest( new DataWritten( StructType.of(), commitId, - TableReference.of("catalog", TableIdentifier.of("db", "tbl"), UUID.randomUUID()), + TableReference.of("catalog", TableIdentifier.of("db", "tbl"), null), dataFiles, deleteFiles)); bytes = AvroUtil.encode(commitResponse); From 2c92500e974eb5a58cdcf75126a85d7cf16d2995 Mon Sep 17 00:00:00 2001 From: Varun Lakhyani <130844282+varun-lakhyani@users.noreply.github.com> Date: Sat, 10 Jan 2026 11:48:15 +0530 Subject: [PATCH 176/201] Spark: Add location overlap validation for SnapshotTableAction (#14933) --- .../actions/SnapshotTableSparkAction.java | 11 ++- .../actions/TestSnapshotTableAction.java | 93 +++++++++++++++++++ 2 files changed, 103 insertions(+), 1 deletion(-) diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java index 5f7f408cb099..043b63870ca9 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java @@ -124,7 +124,16 @@ private SnapshotTable.Result doExecute() { StagedSparkTable stagedTable = stageDestTable(); Table icebergTable = stagedTable.table(); - // TODO: Check the dest table location does not overlap with the source table location + String sourceTableLocation = sourceTableLocation(); + String stagedTableLocation = icebergTable.location(); + Preconditions.checkArgument( + !sourceTableLocation.equals(stagedTableLocation) + && !stagedTableLocation.startsWith(sourceTableLocation + "/") + && !sourceTableLocation.startsWith(stagedTableLocation + "/"), + "Cannot create a snapshot at location %s because it would overlap with source table location %s. " + + "Overlapping snapshot and source would mix table files.", + stagedTableLocation, + sourceTableLocation); boolean threw = true; try { diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java index d9c42a07b853..9fac633e75ac 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java @@ -18,7 +18,11 @@ */ package org.apache.iceberg.spark.actions; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP; 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.IOException; import java.nio.file.Files; @@ -33,6 +37,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestSnapshotTableAction extends CatalogTestBase { private static final String SOURCE_NAME = "spark_catalog.default.source"; + private static final String SOURCE = "source"; @AfterEach public void removeTables() { @@ -65,4 +70,92 @@ public void testSnapshotWithParallelTasks() throws IOException { .execute(); assertThat(snapshotThreadsIndex.get()).isEqualTo(2); } + + @TestTemplate + public void testSnapshotWithOverlappingLocation() throws IOException { + // Hadoop Catalogs do not Support Custom Table Locations + String catalogType = catalogConfig.get(ICEBERG_CATALOG_TYPE); + assumeThat(catalogType).isNotEqualTo(ICEBERG_CATALOG_TYPE_HADOOP); + + String sourceLocation = + Files.createTempDirectory(temp, "junit").resolve(SOURCE).toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, sourceLocation); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + String actualSourceLocation = + spark + .sql(String.format("DESCRIBE EXTENDED %s", SOURCE_NAME)) + .filter("col_name = 'Location'") + .select("data_type") + .first() + .getString(0); + + assertThatThrownBy( + () -> + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(actualSourceLocation) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "The snapshot table location cannot be same as the source table location."); + + String destAsSubdirectory = actualSourceLocation + "/nested"; + assertThatThrownBy( + () -> + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(destAsSubdirectory) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create a snapshot at location"); + + String parentLocation = + actualSourceLocation.substring(0, actualSourceLocation.length() - ("/" + SOURCE).length()); + assertThatThrownBy( + () -> + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(parentLocation) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create a snapshot at location"); + } + + @TestTemplate + public void testSnapshotWithNonOverlappingLocation() throws IOException { + // Hadoop Catalogs do not Support Custom Table Locations + String catalogType = catalogConfig.get(ICEBERG_CATALOG_TYPE); + assumeThat(catalogType).isNotEqualTo(ICEBERG_CATALOG_TYPE_HADOOP); + + String sourceLocation = + Files.createTempDirectory(temp, "junit").resolve(SOURCE).toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, sourceLocation); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + String actualSourceLocation = + spark + .sql(String.format("DESCRIBE EXTENDED %s", SOURCE_NAME)) + .filter("col_name = 'Location'") + .select("data_type") + .first() + .getString(0); + + String validDestLocation = + actualSourceLocation.substring(0, actualSourceLocation.length() - SOURCE.length()) + + "newDestination"; + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(validDestLocation) + .execute(); + assertThat(sql("SELECT * FROM %s", tableName)).hasSize(2); + } } From b4bb71fc408d17d2c724f7a2faf622759746523d Mon Sep 17 00:00:00 2001 From: Varun Lakhyani <130844282+varun-lakhyani@users.noreply.github.com> Date: Sat, 10 Jan 2026 23:00:07 +0530 Subject: [PATCH 177/201] Spark: Backport #14933: Snapshot location overlap check to spark v3.4, v3.5, v4.0 (#15016) * Backport #14933 Snapshot location overlap check to spark v3.4, v3.5, v4.0 * spotlessApply --- .../actions/SnapshotTableSparkAction.java | 11 ++- .../actions/TestSnapshotTableAction.java | 93 +++++++++++++++++++ .../actions/SnapshotTableSparkAction.java | 11 ++- .../actions/TestSnapshotTableAction.java | 93 +++++++++++++++++++ .../actions/SnapshotTableSparkAction.java | 11 ++- .../actions/TestSnapshotTableAction.java | 93 +++++++++++++++++++ 6 files changed, 309 insertions(+), 3 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java index 5f7f408cb099..043b63870ca9 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java @@ -124,7 +124,16 @@ private SnapshotTable.Result doExecute() { StagedSparkTable stagedTable = stageDestTable(); Table icebergTable = stagedTable.table(); - // TODO: Check the dest table location does not overlap with the source table location + String sourceTableLocation = sourceTableLocation(); + String stagedTableLocation = icebergTable.location(); + Preconditions.checkArgument( + !sourceTableLocation.equals(stagedTableLocation) + && !stagedTableLocation.startsWith(sourceTableLocation + "/") + && !sourceTableLocation.startsWith(stagedTableLocation + "/"), + "Cannot create a snapshot at location %s because it would overlap with source table location %s. " + + "Overlapping snapshot and source would mix table files.", + stagedTableLocation, + sourceTableLocation); boolean threw = true; try { diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java index d9c42a07b853..9fac633e75ac 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java @@ -18,7 +18,11 @@ */ package org.apache.iceberg.spark.actions; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP; 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.IOException; import java.nio.file.Files; @@ -33,6 +37,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestSnapshotTableAction extends CatalogTestBase { private static final String SOURCE_NAME = "spark_catalog.default.source"; + private static final String SOURCE = "source"; @AfterEach public void removeTables() { @@ -65,4 +70,92 @@ public void testSnapshotWithParallelTasks() throws IOException { .execute(); assertThat(snapshotThreadsIndex.get()).isEqualTo(2); } + + @TestTemplate + public void testSnapshotWithOverlappingLocation() throws IOException { + // Hadoop Catalogs do not Support Custom Table Locations + String catalogType = catalogConfig.get(ICEBERG_CATALOG_TYPE); + assumeThat(catalogType).isNotEqualTo(ICEBERG_CATALOG_TYPE_HADOOP); + + String sourceLocation = + Files.createTempDirectory(temp, "junit").resolve(SOURCE).toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, sourceLocation); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + String actualSourceLocation = + spark + .sql(String.format("DESCRIBE EXTENDED %s", SOURCE_NAME)) + .filter("col_name = 'Location'") + .select("data_type") + .first() + .getString(0); + + assertThatThrownBy( + () -> + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(actualSourceLocation) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "The snapshot table location cannot be same as the source table location."); + + String destAsSubdirectory = actualSourceLocation + "/nested"; + assertThatThrownBy( + () -> + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(destAsSubdirectory) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create a snapshot at location"); + + String parentLocation = + actualSourceLocation.substring(0, actualSourceLocation.length() - ("/" + SOURCE).length()); + assertThatThrownBy( + () -> + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(parentLocation) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create a snapshot at location"); + } + + @TestTemplate + public void testSnapshotWithNonOverlappingLocation() throws IOException { + // Hadoop Catalogs do not Support Custom Table Locations + String catalogType = catalogConfig.get(ICEBERG_CATALOG_TYPE); + assumeThat(catalogType).isNotEqualTo(ICEBERG_CATALOG_TYPE_HADOOP); + + String sourceLocation = + Files.createTempDirectory(temp, "junit").resolve(SOURCE).toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, sourceLocation); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + String actualSourceLocation = + spark + .sql(String.format("DESCRIBE EXTENDED %s", SOURCE_NAME)) + .filter("col_name = 'Location'") + .select("data_type") + .first() + .getString(0); + + String validDestLocation = + actualSourceLocation.substring(0, actualSourceLocation.length() - SOURCE.length()) + + "newDestination"; + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(validDestLocation) + .execute(); + assertThat(sql("SELECT * FROM %s", tableName)).hasSize(2); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java index 5f7f408cb099..043b63870ca9 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java @@ -124,7 +124,16 @@ private SnapshotTable.Result doExecute() { StagedSparkTable stagedTable = stageDestTable(); Table icebergTable = stagedTable.table(); - // TODO: Check the dest table location does not overlap with the source table location + String sourceTableLocation = sourceTableLocation(); + String stagedTableLocation = icebergTable.location(); + Preconditions.checkArgument( + !sourceTableLocation.equals(stagedTableLocation) + && !stagedTableLocation.startsWith(sourceTableLocation + "/") + && !sourceTableLocation.startsWith(stagedTableLocation + "/"), + "Cannot create a snapshot at location %s because it would overlap with source table location %s. " + + "Overlapping snapshot and source would mix table files.", + stagedTableLocation, + sourceTableLocation); boolean threw = true; try { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java index d9c42a07b853..9fac633e75ac 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java @@ -18,7 +18,11 @@ */ package org.apache.iceberg.spark.actions; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP; 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.IOException; import java.nio.file.Files; @@ -33,6 +37,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestSnapshotTableAction extends CatalogTestBase { private static final String SOURCE_NAME = "spark_catalog.default.source"; + private static final String SOURCE = "source"; @AfterEach public void removeTables() { @@ -65,4 +70,92 @@ public void testSnapshotWithParallelTasks() throws IOException { .execute(); assertThat(snapshotThreadsIndex.get()).isEqualTo(2); } + + @TestTemplate + public void testSnapshotWithOverlappingLocation() throws IOException { + // Hadoop Catalogs do not Support Custom Table Locations + String catalogType = catalogConfig.get(ICEBERG_CATALOG_TYPE); + assumeThat(catalogType).isNotEqualTo(ICEBERG_CATALOG_TYPE_HADOOP); + + String sourceLocation = + Files.createTempDirectory(temp, "junit").resolve(SOURCE).toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, sourceLocation); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + String actualSourceLocation = + spark + .sql(String.format("DESCRIBE EXTENDED %s", SOURCE_NAME)) + .filter("col_name = 'Location'") + .select("data_type") + .first() + .getString(0); + + assertThatThrownBy( + () -> + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(actualSourceLocation) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "The snapshot table location cannot be same as the source table location."); + + String destAsSubdirectory = actualSourceLocation + "/nested"; + assertThatThrownBy( + () -> + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(destAsSubdirectory) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create a snapshot at location"); + + String parentLocation = + actualSourceLocation.substring(0, actualSourceLocation.length() - ("/" + SOURCE).length()); + assertThatThrownBy( + () -> + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(parentLocation) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create a snapshot at location"); + } + + @TestTemplate + public void testSnapshotWithNonOverlappingLocation() throws IOException { + // Hadoop Catalogs do not Support Custom Table Locations + String catalogType = catalogConfig.get(ICEBERG_CATALOG_TYPE); + assumeThat(catalogType).isNotEqualTo(ICEBERG_CATALOG_TYPE_HADOOP); + + String sourceLocation = + Files.createTempDirectory(temp, "junit").resolve(SOURCE).toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, sourceLocation); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + String actualSourceLocation = + spark + .sql(String.format("DESCRIBE EXTENDED %s", SOURCE_NAME)) + .filter("col_name = 'Location'") + .select("data_type") + .first() + .getString(0); + + String validDestLocation = + actualSourceLocation.substring(0, actualSourceLocation.length() - SOURCE.length()) + + "newDestination"; + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(validDestLocation) + .execute(); + assertThat(sql("SELECT * FROM %s", tableName)).hasSize(2); + } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java index 5f7f408cb099..043b63870ca9 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SnapshotTableSparkAction.java @@ -124,7 +124,16 @@ private SnapshotTable.Result doExecute() { StagedSparkTable stagedTable = stageDestTable(); Table icebergTable = stagedTable.table(); - // TODO: Check the dest table location does not overlap with the source table location + String sourceTableLocation = sourceTableLocation(); + String stagedTableLocation = icebergTable.location(); + Preconditions.checkArgument( + !sourceTableLocation.equals(stagedTableLocation) + && !stagedTableLocation.startsWith(sourceTableLocation + "/") + && !sourceTableLocation.startsWith(stagedTableLocation + "/"), + "Cannot create a snapshot at location %s because it would overlap with source table location %s. " + + "Overlapping snapshot and source would mix table files.", + stagedTableLocation, + sourceTableLocation); boolean threw = true; try { diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java index d9c42a07b853..9fac633e75ac 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java @@ -18,7 +18,11 @@ */ package org.apache.iceberg.spark.actions; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE; +import static org.apache.iceberg.CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP; 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.IOException; import java.nio.file.Files; @@ -33,6 +37,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestSnapshotTableAction extends CatalogTestBase { private static final String SOURCE_NAME = "spark_catalog.default.source"; + private static final String SOURCE = "source"; @AfterEach public void removeTables() { @@ -65,4 +70,92 @@ public void testSnapshotWithParallelTasks() throws IOException { .execute(); assertThat(snapshotThreadsIndex.get()).isEqualTo(2); } + + @TestTemplate + public void testSnapshotWithOverlappingLocation() throws IOException { + // Hadoop Catalogs do not Support Custom Table Locations + String catalogType = catalogConfig.get(ICEBERG_CATALOG_TYPE); + assumeThat(catalogType).isNotEqualTo(ICEBERG_CATALOG_TYPE_HADOOP); + + String sourceLocation = + Files.createTempDirectory(temp, "junit").resolve(SOURCE).toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, sourceLocation); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + String actualSourceLocation = + spark + .sql(String.format("DESCRIBE EXTENDED %s", SOURCE_NAME)) + .filter("col_name = 'Location'") + .select("data_type") + .first() + .getString(0); + + assertThatThrownBy( + () -> + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(actualSourceLocation) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "The snapshot table location cannot be same as the source table location."); + + String destAsSubdirectory = actualSourceLocation + "/nested"; + assertThatThrownBy( + () -> + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(destAsSubdirectory) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create a snapshot at location"); + + String parentLocation = + actualSourceLocation.substring(0, actualSourceLocation.length() - ("/" + SOURCE).length()); + assertThatThrownBy( + () -> + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(parentLocation) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot create a snapshot at location"); + } + + @TestTemplate + public void testSnapshotWithNonOverlappingLocation() throws IOException { + // Hadoop Catalogs do not Support Custom Table Locations + String catalogType = catalogConfig.get(ICEBERG_CATALOG_TYPE); + assumeThat(catalogType).isNotEqualTo(ICEBERG_CATALOG_TYPE_HADOOP); + + String sourceLocation = + Files.createTempDirectory(temp, "junit").resolve(SOURCE).toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, sourceLocation); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + String actualSourceLocation = + spark + .sql(String.format("DESCRIBE EXTENDED %s", SOURCE_NAME)) + .filter("col_name = 'Location'") + .select("data_type") + .first() + .getString(0); + + String validDestLocation = + actualSourceLocation.substring(0, actualSourceLocation.length() - SOURCE.length()) + + "newDestination"; + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .tableLocation(validDestLocation) + .execute(); + assertThat(sql("SELECT * FROM %s", tableName)).hasSize(2); + } } From d671410367bad6cf1a9139cfa8ede58f7ea80976 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 10 Jan 2026 21:12:23 -0800 Subject: [PATCH 178/201] Build: Bump datamodel-code-generator from 0.52.1 to 0.52.2 (#15018) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.52.1 to 0.52.2. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Changelog](https://github.com/koxudaxi/datamodel-code-generator/blob/main/CHANGELOG.md) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.52.1...0.52.2) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-version: 0.52.2 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index c12adbaaedf2..bbc90243a49a 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,5 +16,5 @@ # under the License. openapi-spec-validator==0.7.2 -datamodel-code-generator==0.52.1 +datamodel-code-generator==0.52.2 yamllint==1.37.1 From 7fd8a2ddc94e32b665bc954a0b409705af36527b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 10 Jan 2026 21:13:43 -0800 Subject: [PATCH 179/201] Build: Bump io.grpc:grpc-netty-shaded from 1.76.2 to 1.78.0 (#15024) Bumps [io.grpc:grpc-netty-shaded](https://github.com/grpc/grpc-java) from 1.76.2 to 1.78.0. - [Release notes](https://github.com/grpc/grpc-java/releases) - [Commits](https://github.com/grpc/grpc-java/compare/v1.76.2...v1.78.0) --- updated-dependencies: - dependency-name: io.grpc:grpc-netty-shaded dependency-version: 1.78.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- kafka-connect/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 5f145312695a..89d118273244 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -76,7 +76,7 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { force 'org.apache.hadoop.thirdparty:hadoop-shaded-guava:1.5.0' force 'com.fasterxml.woodstox:woodstox-core:6.7.0' force 'commons-beanutils:commons-beanutils:1.11.0' - force 'io.grpc:grpc-netty-shaded:1.76.2' + force 'io.grpc:grpc-netty-shaded:1.78.0' } } } From 12f935459ce0801ea3460c81c75bfcceb1f2cb3a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 11 Jan 2026 00:30:25 -0800 Subject: [PATCH 180/201] Build: Bump nessie from 0.106.0 to 0.106.1 (#15019) Bumps `nessie` from 0.106.0 to 0.106.1. Updates `org.projectnessie.nessie:nessie-client` from 0.106.0 to 0.106.1 - [Release notes](https://github.com/projectnessie/nessie/releases) - [Changelog](https://github.com/projectnessie/nessie/blob/main/CHANGELOG.md) - [Commits](https://github.com/projectnessie/nessie/compare/nessie-0.106.0...nessie-0.106.1) Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.106.0 to 0.106.1 - [Release notes](https://github.com/projectnessie/nessie/releases) - [Changelog](https://github.com/projectnessie/nessie/blob/main/CHANGELOG.md) - [Commits](https://github.com/projectnessie/nessie/compare/nessie-0.106.0...nessie-0.106.1) Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.106.0 to 0.106.1 - [Release notes](https://github.com/projectnessie/nessie/releases) - [Changelog](https://github.com/projectnessie/nessie/blob/main/CHANGELOG.md) - [Commits](https://github.com/projectnessie/nessie/compare/nessie-0.106.0...nessie-0.106.1) Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.106.0 to 0.106.1 - [Release notes](https://github.com/projectnessie/nessie/releases) - [Changelog](https://github.com/projectnessie/nessie/blob/main/CHANGELOG.md) - [Commits](https://github.com/projectnessie/nessie/compare/nessie-0.106.0...nessie-0.106.1) --- updated-dependencies: - dependency-name: org.projectnessie.nessie:nessie-client dependency-version: 0.106.1 dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.projectnessie.nessie:nessie-jaxrs-testextension dependency-version: 0.106.1 dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests dependency-version: 0.106.1 dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-testextension dependency-version: 0.106.1 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index bbfb9d3616ba..ce42afa1f422 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -75,7 +75,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.106.0" +nessie = "0.106.1" netty-buffer = "4.2.9.Final" object-client-bundle = "3.3.2" orc = "1.9.7" From a401603a9cafe637baad27bf8291e5a595b020d0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 11 Jan 2026 00:30:39 -0800 Subject: [PATCH 181/201] Build: Bump com.google.errorprone:error_prone_annotations (#15020) Bumps [com.google.errorprone:error_prone_annotations](https://github.com/google/error-prone) from 2.45.0 to 2.46.0. - [Release notes](https://github.com/google/error-prone/releases) - [Commits](https://github.com/google/error-prone/compare/v2.45.0...v2.46.0) --- updated-dependencies: - dependency-name: com.google.errorprone:error_prone_annotations dependency-version: 2.46.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index ce42afa1f422..183e11cad4ef 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -43,7 +43,7 @@ delta-standalone = "3.3.2" delta-spark = "3.3.2" derby = "10.15.2.0" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.45.0" +errorprone-annotations = "2.46.0" failsafe = "3.3.2" findbugs-jsr305 = "3.0.2" flink120 = { strictly = "1.20.1"} From 95d7405d3c0c19f9333dd0759a1e2c49c4958d1d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 11 Jan 2026 09:05:45 -0800 Subject: [PATCH 182/201] Build: Bump junit-platform from 1.14.1 to 1.14.2 (#15021) Bumps `junit-platform` from 1.14.1 to 1.14.2. Updates `org.junit.platform:junit-platform-launcher` from 1.14.1 to 1.14.2 - [Release notes](https://github.com/junit-team/junit-framework/releases) - [Commits](https://github.com/junit-team/junit-framework/commits) Updates `org.junit.platform:junit-platform-suite-api` from 1.14.1 to 1.14.2 - [Release notes](https://github.com/junit-team/junit-framework/releases) - [Commits](https://github.com/junit-team/junit-framework/commits) Updates `org.junit.platform:junit-platform-suite-engine` from 1.14.1 to 1.14.2 - [Release notes](https://github.com/junit-team/junit-framework/releases) - [Commits](https://github.com/junit-team/junit-framework/commits) --- updated-dependencies: - dependency-name: org.junit.platform:junit-platform-launcher dependency-version: 1.14.2 dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.junit.platform:junit-platform-suite-api dependency-version: 1.14.2 dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.junit.platform:junit-platform-suite-engine dependency-version: 1.14.2 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 183e11cad4ef..4389e4b7be0d 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -69,7 +69,7 @@ jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" jetty = "11.0.26" junit = "5.14.1" -junit-platform = "1.14.1" +junit-platform = "1.14.2" kafka = "3.9.1" kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.2" From ccf4dfbb65146d9be11e5027404fb0c3df2af999 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 11 Jan 2026 09:06:11 -0800 Subject: [PATCH 183/201] Build: Bump software.amazon.awssdk:bom from 2.41.1 to 2.41.5 (#15022) Bumps software.amazon.awssdk:bom from 2.41.1 to 2.41.5. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-version: 2.41.5 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 4389e4b7be0d..891b304ea233 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -31,7 +31,7 @@ arrow = "15.0.2" avro = "1.12.1" assertj-core = "3.27.6" awaitility = "4.3.0" -awssdk-bom = "2.41.1" +awssdk-bom = "2.41.5" azuresdk-bom = "1.3.3" awssdk-s3accessgrants = "2.3.0" bson-ver = "4.11.5" From cc966fc96374366b17ed39b1cbd8916438b4bb84 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 11 Jan 2026 09:07:03 -0800 Subject: [PATCH 184/201] Build: Bump org.immutables:value from 2.12.0 to 2.12.1 (#15026) Bumps [org.immutables:value](https://github.com/immutables/immutables) from 2.12.0 to 2.12.1. - [Release notes](https://github.com/immutables/immutables/releases) - [Commits](https://github.com/immutables/immutables/compare/2.12.0...2.12.1) --- updated-dependencies: - dependency-name: org.immutables:value dependency-version: 2.12.1 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 891b304ea233..11005f286326 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -55,7 +55,7 @@ guava = "33.5.0-jre" hadoop3 = "3.4.2" httpcomponents-httpclient5 = "5.6" hive2 = { strictly = "2.3.10"} # see rich version usage explanation above -immutables-value = "2.12.0" +immutables-value = "2.12.1" jackson-annotations = "2.20" jackson-bom = "2.20.1" jackson211 = { strictly = "2.11.4"} # see rich version usage explanation above From c73116ab377261fc3875ce2194f38fd6460e5719 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 11 Jan 2026 09:07:43 -0800 Subject: [PATCH 185/201] Build: Bump orc from 1.9.7 to 1.9.8 (#15025) Bumps `orc` from 1.9.7 to 1.9.8. Updates `org.apache.orc:orc-core` from 1.9.7 to 1.9.8 Updates `org.apache.orc:orc-tools` from 1.9.7 to 1.9.8 --- updated-dependencies: - dependency-name: org.apache.orc:orc-core dependency-version: 1.9.8 dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.apache.orc:orc-tools dependency-version: 1.9.8 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 11005f286326..6d9053a3b480 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -78,7 +78,7 @@ mockserver = "5.15.0" nessie = "0.106.1" netty-buffer = "4.2.9.Final" object-client-bundle = "3.3.2" -orc = "1.9.7" +orc = "1.9.8" parquet = "1.16.0" roaringbitmap = "1.3.0" scala-collection-compat = "2.14.0" From b7d9817cc97fb7b3c11bd105a91f1550b05dc63e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 11 Jan 2026 10:44:30 -0800 Subject: [PATCH 186/201] Build: Bump junit from 5.14.1 to 5.14.2 (#15023) Bumps `junit` from 5.14.1 to 5.14.2. Updates `org.junit.jupiter:junit-jupiter` from 5.14.1 to 5.14.2 - [Release notes](https://github.com/junit-team/junit-framework/releases) - [Commits](https://github.com/junit-team/junit-framework/compare/r5.14.1...r5.14.2) Updates `org.junit.jupiter:junit-jupiter-engine` from 5.14.1 to 5.14.2 - [Release notes](https://github.com/junit-team/junit-framework/releases) - [Commits](https://github.com/junit-team/junit-framework/compare/r5.14.1...r5.14.2) --- updated-dependencies: - dependency-name: org.junit.jupiter:junit-jupiter dependency-version: 5.14.2 dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.junit.jupiter:junit-jupiter-engine dependency-version: 5.14.2 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 6d9053a3b480..80c2c79b21fb 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -68,7 +68,7 @@ jakarta-servlet-api = "6.1.0" jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" jetty = "11.0.26" -junit = "5.14.1" +junit = "5.14.2" junit-platform = "1.14.2" kafka = "3.9.1" kryo-shaded = "4.0.3" From 4206122465e7576e978e2fa56efd739c24301701 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Mon, 12 Jan 2026 15:40:36 +0800 Subject: [PATCH 187/201] Spark 4.1: Upgrade to Spark 4.1.1 (#14946) --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 80c2c79b21fb..8fb929e3673d 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -87,7 +87,7 @@ snowflake-jdbc = "3.28.0" spark34 = "3.4.4" spark35 = "3.5.7" spark40 = "4.0.1" -spark41 = "4.1.0" +spark41 = "4.1.1" sqlite-jdbc = "3.51.1.0" testcontainers = "2.0.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 7f81e1e93084e50fa3676c2e131722f66a26b385 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Mon, 12 Jan 2026 14:14:03 +0100 Subject: [PATCH 188/201] Core: Use scan API to read partition stats (#14989) --- .../apache/iceberg/PartitionStatistics.java | 15 ++ .../iceberg/BasePartitionStatistics.java | 70 +++-- .../apache/iceberg/PartitionStatsHandler.java | 252 +++++++++++++++--- .../iceberg/PartitionStatisticsTestBase.java | 26 +- .../PartitionStatsHandlerTestBase.java | 67 +++-- 5 files changed, 325 insertions(+), 105 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/PartitionStatistics.java b/api/src/main/java/org/apache/iceberg/PartitionStatistics.java index 10df7303d500..c0c4c07b7e27 100644 --- a/api/src/main/java/org/apache/iceberg/PartitionStatistics.java +++ b/api/src/main/java/org/apache/iceberg/PartitionStatistics.java @@ -21,6 +21,21 @@ /** Interface for partition statistics returned from a {@link PartitionStatisticsScan}. */ public interface PartitionStatistics extends StructLike { + /* The positions of each statistics within the full schema of partition statistics. */ + int PARTITION_POSITION = 0; + int SPEC_ID_POSITION = 1; + int DATA_RECORD_COUNT_POSITION = 2; + int DATA_FILE_COUNT_POSITION = 3; + int TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION = 4; + int POSITION_DELETE_RECORD_COUNT_POSITION = 5; + int POSITION_DELETE_FILE_COUNT_POSITION = 6; + int EQUALITY_DELETE_RECORD_COUNT_POSITION = 7; + int EQUALITY_DELETE_FILE_COUNT_POSITION = 8; + int TOTAL_RECORD_COUNT_POSITION = 9; + int LAST_UPDATED_AT_POSITION = 10; + int LAST_UPDATED_SNAPSHOT_ID_POSITION = 11; + int DV_COUNT_POSITION = 12; + /** Returns the partition of these partition statistics */ StructLike partition(); diff --git a/core/src/main/java/org/apache/iceberg/BasePartitionStatistics.java b/core/src/main/java/org/apache/iceberg/BasePartitionStatistics.java index c17718281b57..4b1a3a6dba93 100644 --- a/core/src/main/java/org/apache/iceberg/BasePartitionStatistics.java +++ b/core/src/main/java/org/apache/iceberg/BasePartitionStatistics.java @@ -40,8 +40,24 @@ public class BasePartitionStatistics extends SupportsIndexProjection private static final int STATS_COUNT = 13; + BasePartitionStatistics(StructLike partition, int specId) { + super(STATS_COUNT); + + this.partition = partition; + this.specId = specId; + + this.dataRecordCount = 0L; + this.dataFileCount = 0; + this.totalDataFileSizeInBytes = 0L; + this.positionDeleteRecordCount = 0L; + this.positionDeleteFileCount = 0; + this.equalityDeleteRecordCount = 0L; + this.equalityDeleteFileCount = 0; + this.dvCount = 0; + } + /** Used by internal readers to instantiate this class with a projection schema. */ - public BasePartitionStatistics(Types.StructType projection) { + BasePartitionStatistics(Types.StructType projection) { super(STATS_COUNT); } @@ -117,31 +133,31 @@ protected T internalGet(int pos, Class javaClass) { private Object getByPos(int pos) { switch (pos) { - case 0: + case PARTITION_POSITION: return partition; - case 1: + case SPEC_ID_POSITION: return specId; - case 2: + case DATA_RECORD_COUNT_POSITION: return dataRecordCount; - case 3: + case DATA_FILE_COUNT_POSITION: return dataFileCount; - case 4: + case TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION: return totalDataFileSizeInBytes; - case 5: + case POSITION_DELETE_RECORD_COUNT_POSITION: return positionDeleteRecordCount; - case 6: + case POSITION_DELETE_FILE_COUNT_POSITION: return positionDeleteFileCount; - case 7: + case EQUALITY_DELETE_RECORD_COUNT_POSITION: return equalityDeleteRecordCount; - case 8: + case EQUALITY_DELETE_FILE_COUNT_POSITION: return equalityDeleteFileCount; - case 9: + case TOTAL_RECORD_COUNT_POSITION: return totalRecordCount; - case 10: + case LAST_UPDATED_AT_POSITION: return lastUpdatedAt; - case 11: + case LAST_UPDATED_SNAPSHOT_ID_POSITION: return lastUpdatedSnapshotId; - case 12: + case DV_COUNT_POSITION: return dvCount; default: throw new UnsupportedOperationException("Unknown position: " + pos); @@ -155,43 +171,43 @@ protected void internalSet(int pos, T value) { } switch (pos) { - case 0: + case PARTITION_POSITION: this.partition = (StructLike) value; break; - case 1: + case SPEC_ID_POSITION: this.specId = (int) value; break; - case 2: + case DATA_RECORD_COUNT_POSITION: this.dataRecordCount = (long) value; break; - case 3: + case DATA_FILE_COUNT_POSITION: this.dataFileCount = (int) value; break; - case 4: + case TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION: this.totalDataFileSizeInBytes = (long) value; break; - case 5: + case POSITION_DELETE_RECORD_COUNT_POSITION: this.positionDeleteRecordCount = (long) value; break; - case 6: + case POSITION_DELETE_FILE_COUNT_POSITION: this.positionDeleteFileCount = (int) value; break; - case 7: + case EQUALITY_DELETE_RECORD_COUNT_POSITION: this.equalityDeleteRecordCount = (long) value; break; - case 8: + case EQUALITY_DELETE_FILE_COUNT_POSITION: this.equalityDeleteFileCount = (int) value; break; - case 9: + case TOTAL_RECORD_COUNT_POSITION: this.totalRecordCount = (Long) value; break; - case 10: + case LAST_UPDATED_AT_POSITION: this.lastUpdatedAt = (Long) value; break; - case 11: + case LAST_UPDATED_SNAPSHOT_ID_POSITION: this.lastUpdatedSnapshotId = (Long) value; break; - case 12: + case DV_COUNT_POSITION: this.dvCount = (int) value; break; default: diff --git a/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java b/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java index 7259a1f0684b..acf4ca3c0ce9 100644 --- a/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java +++ b/core/src/main/java/org/apache/iceberg/PartitionStatsHandler.java @@ -208,9 +208,7 @@ public static PartitionStatisticsFile computeAndWriteStatsFile(Table table, long Snapshot snapshot = table.snapshot(snapshotId); Preconditions.checkArgument(snapshot != null, "Snapshot not found: %s", snapshotId); - StructType partitionType = Partitioning.partitionType(table); - - Collection stats; + Collection stats; PartitionStatisticsFile statisticsFile = latestStatsFile(table, snapshot.snapshotId()); if (statisticsFile == null) { LOG.info( @@ -225,7 +223,7 @@ public static PartitionStatisticsFile computeAndWriteStatsFile(Table table, long } try { - stats = computeAndMergeStatsIncremental(table, snapshot, partitionType, statisticsFile); + stats = computeAndMergeStatsIncremental(table, snapshot, statisticsFile.snapshotId()); } catch (InvalidStatsFileException exception) { LOG.warn( "Using full compute as previous statistics file is corrupted for incremental compute."); @@ -240,7 +238,8 @@ public static PartitionStatisticsFile computeAndWriteStatsFile(Table table, long return null; } - List sortedStats = sortStatsByPartition(stats, partitionType); + StructType partitionType = Partitioning.partitionType(table); + List sortedStats = sortStatsByPartition(stats, partitionType); return writePartitionStatsFile( table, snapshot.snapshotId(), @@ -250,7 +249,7 @@ public static PartitionStatisticsFile computeAndWriteStatsFile(Table table, long @VisibleForTesting static PartitionStatisticsFile writePartitionStatsFile( - Table table, long snapshotId, Schema dataSchema, Iterable records) + Table table, long snapshotId, Schema dataSchema, Iterable records) throws IOException { FileFormat fileFormat = FileFormat.fromString( @@ -322,17 +321,11 @@ private static PartitionStats recordToPartitionStats(StructLike record) { return stats; } - private static Collection computeAndMergeStatsIncremental( - Table table, - Snapshot snapshot, - StructType partitionType, - PartitionStatisticsFile previousStatsFile) { - PartitionMap statsMap = PartitionMap.create(table.specs()); - // read previous stats, note that partition field will be read as GenericRecord - try (CloseableIterable oldStats = - readPartitionStatsFile( - schema(partitionType, TableUtil.formatVersion(table)), - table.io().newInputFile(previousStatsFile.path()))) { + private static Collection computeAndMergeStatsIncremental( + Table table, Snapshot snapshot, long lastSnapshotWithStats) { + PartitionMap statsMap = PartitionMap.create(table.specs()); + try (CloseableIterable oldStats = + table.newPartitionStatisticsScan().useSnapshot(lastSnapshotWithStats).scan()) { oldStats.forEach( partitionStats -> statsMap.put(partitionStats.specId(), partitionStats.partition(), partitionStats)); @@ -341,8 +334,8 @@ private static Collection computeAndMergeStatsIncremental( } // incrementally compute the new stats, partition field will be written as PartitionData - PartitionMap incrementalStatsMap = - computeStatsDiff(table, table.snapshot(previousStatsFile.snapshotId()), snapshot); + PartitionMap incrementalStatsMap = + computeStatsDiff(table, table.snapshot(lastSnapshotWithStats), snapshot); // convert PartitionData into GenericRecord and merge stats incrementalStatsMap.forEach( @@ -351,7 +344,7 @@ private static Collection computeAndMergeStatsIncremental( Pair.of(key.first(), partitionDataToRecord((PartitionData) key.second())), value, (existingEntry, newEntry) -> { - existingEntry.appendStats(newEntry); + appendStats(existingEntry, newEntry); return existingEntry; })); @@ -389,7 +382,7 @@ static PartitionStatisticsFile latestStatsFile(Table table, long snapshotId) { return null; } - private static PartitionMap computeStatsDiff( + private static PartitionMap computeStatsDiff( Table table, Snapshot fromSnapshot, Snapshot toSnapshot) { Iterable snapshots = SnapshotUtil.ancestorsBetween( @@ -408,10 +401,10 @@ private static PartitionMap computeStatsDiff( return computeStats(table, manifests, true /* incremental */); } - private static PartitionMap computeStats( + private static PartitionMap computeStats( Table table, List manifests, boolean incremental) { StructType partitionType = Partitioning.partitionType(table); - Queue> statsByManifest = Queues.newConcurrentLinkedQueue(); + Queue> statsByManifest = Queues.newConcurrentLinkedQueue(); Tasks.foreach(manifests) .stopOnFailure() .throwFailureWhenFinished() @@ -421,19 +414,19 @@ private static PartitionMap computeStats( statsByManifest.add( collectStatsForManifest(table, manifest, partitionType, incremental))); - PartitionMap statsMap = PartitionMap.create(table.specs()); - for (PartitionMap stats : statsByManifest) { + PartitionMap statsMap = PartitionMap.create(table.specs()); + for (PartitionMap stats : statsByManifest) { mergePartitionMap(stats, statsMap); } return statsMap; } - private static PartitionMap collectStatsForManifest( + private static PartitionMap collectStatsForManifest( Table table, ManifestFile manifest, StructType partitionType, boolean incremental) { List projection = BaseScan.scanColumns(manifest.content()); try (ManifestReader reader = ManifestFiles.open(manifest, table.io()).select(projection)) { - PartitionMap statsMap = PartitionMap.create(table.specs()); + PartitionMap statsMap = PartitionMap.create(table.specs()); int specId = manifest.partitionSpecId(); PartitionSpec spec = table.specs().get(specId); PartitionData keyTemplate = new PartitionData(partitionType); @@ -444,22 +437,22 @@ private static PartitionMap collectStatsForManifest( PartitionUtil.coercePartition(partitionType, spec, file.partition()); StructLike key = keyTemplate.copyFor(coercedPartition); Snapshot snapshot = table.snapshot(entry.snapshotId()); - PartitionStats stats = + PartitionStatistics stats = statsMap.computeIfAbsent( specId, ((PartitionData) file.partition()).copy(), - () -> new PartitionStats(key, specId)); + () -> new BasePartitionStatistics(key, specId)); if (entry.isLive()) { // Live can have both added and existing entries. Consider only added entries for // incremental compute as existing entries was already included in previous compute. if (!incremental || entry.status() == ManifestEntry.Status.ADDED) { - stats.liveEntry(file, snapshot); + liveEntry(stats, file, snapshot); } } else { if (incremental) { - stats.deletedEntryForIncrementalCompute(file, snapshot); + deletedEntryForIncrementalCompute(stats, file, snapshot); } else { - stats.deletedEntry(snapshot); + deletedEntry(stats, snapshot); } } } @@ -471,26 +464,209 @@ private static PartitionMap collectStatsForManifest( } private static void mergePartitionMap( - PartitionMap fromMap, PartitionMap toMap) { + PartitionMap fromMap, PartitionMap toMap) { fromMap.forEach( (key, value) -> toMap.merge( key, value, (existingEntry, newEntry) -> { - existingEntry.appendStats(newEntry); + appendStats(existingEntry, newEntry); return existingEntry; })); } - private static List sortStatsByPartition( - Collection stats, StructType partitionType) { - List entries = Lists.newArrayList(stats); + private static List sortStatsByPartition( + Collection stats, StructType partitionType) { + List entries = Lists.newArrayList(stats); entries.sort( - Comparator.comparing(PartitionStats::partition, Comparators.forType(partitionType))); + Comparator.comparing(PartitionStatistics::partition, Comparators.forType(partitionType))); return entries; } + /** + * Updates the partition stats from the data/delete file. + * + * @param stats partition statistics to be updated. + * @param file the {@link ContentFile} from the manifest entry. + * @param snapshot the snapshot corresponding to the live entry. + */ + private static void liveEntry(PartitionStatistics stats, ContentFile file, Snapshot snapshot) { + Preconditions.checkArgument(stats.specId() == file.specId(), "Spec IDs must match"); + + switch (file.content()) { + case DATA: + stats.set( + PartitionStatistics.DATA_RECORD_COUNT_POSITION, + stats.dataRecordCount() + file.recordCount()); + stats.set(PartitionStatistics.DATA_FILE_COUNT_POSITION, stats.dataFileCount() + 1); + stats.set( + PartitionStatistics.TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION, + stats.totalDataFileSizeInBytes() + file.fileSizeInBytes()); + break; + case POSITION_DELETES: + stats.set( + PartitionStatistics.POSITION_DELETE_RECORD_COUNT_POSITION, + stats.positionDeleteRecordCount() + file.recordCount()); + if (file.format() == FileFormat.PUFFIN) { + stats.set(PartitionStatistics.DV_COUNT_POSITION, stats.dvCount() + 1); + } else { + stats.set( + PartitionStatistics.POSITION_DELETE_FILE_COUNT_POSITION, + stats.positionDeleteFileCount() + 1); + } + + break; + case EQUALITY_DELETES: + stats.set( + PartitionStatistics.EQUALITY_DELETE_RECORD_COUNT_POSITION, + stats.equalityDeleteRecordCount() + file.recordCount()); + stats.set( + PartitionStatistics.EQUALITY_DELETE_FILE_COUNT_POSITION, + stats.equalityDeleteFileCount() + 1); + break; + default: + throw new UnsupportedOperationException("Unsupported file content type: " + file.content()); + } + + if (snapshot != null) { + updateSnapshotInfo(stats, snapshot.snapshotId(), snapshot.timestampMillis()); + } + + // Note: Not computing the `TOTAL_RECORD_COUNT` for now as it needs scanning the data. + } + + /** + * Updates the modified time and snapshot ID in stats for the deleted manifest entry. + * + * @param stats partition statistics to be updated. + * @param snapshot the snapshot corresponding to the deleted manifest entry. + */ + private static void deletedEntry(PartitionStatistics stats, Snapshot snapshot) { + if (snapshot != null) { + updateSnapshotInfo(stats, snapshot.snapshotId(), snapshot.timestampMillis()); + } + } + + /** + * Decrement the counters in stats as it was included in the previous stats and updates the + * modified time and snapshot ID for the deleted manifest entry. + * + * @param stats partition statistics to be updated. + * @param snapshot the snapshot corresponding to the deleted manifest entry. + */ + private static void deletedEntryForIncrementalCompute( + PartitionStatistics stats, ContentFile file, Snapshot snapshot) { + Preconditions.checkArgument(stats.specId() == file.specId(), "Spec IDs must match"); + + switch (file.content()) { + case DATA: + stats.set( + PartitionStatistics.DATA_RECORD_COUNT_POSITION, + stats.dataRecordCount() - file.recordCount()); + stats.set(PartitionStatistics.DATA_FILE_COUNT_POSITION, stats.dataFileCount() - 1); + stats.set( + PartitionStatistics.TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION, + stats.totalDataFileSizeInBytes() - file.fileSizeInBytes()); + break; + case POSITION_DELETES: + stats.set( + PartitionStatistics.POSITION_DELETE_RECORD_COUNT_POSITION, + stats.positionDeleteRecordCount() - file.recordCount()); + if (file.format() == FileFormat.PUFFIN) { + stats.set(PartitionStatistics.DV_COUNT_POSITION, stats.dvCount() - 1); + } else { + stats.set( + PartitionStatistics.POSITION_DELETE_FILE_COUNT_POSITION, + stats.positionDeleteFileCount() - 1); + } + + break; + case EQUALITY_DELETES: + stats.set( + PartitionStatistics.EQUALITY_DELETE_RECORD_COUNT_POSITION, + stats.equalityDeleteRecordCount() - file.recordCount()); + stats.set( + PartitionStatistics.EQUALITY_DELETE_FILE_COUNT_POSITION, + stats.equalityDeleteFileCount() - 1); + break; + default: + throw new UnsupportedOperationException("Unsupported file content type: " + file.content()); + } + + if (snapshot != null) { + updateSnapshotInfo(stats, snapshot.snapshotId(), snapshot.timestampMillis()); + } + } + + /** + * Appends statistics from given entry to another entry. + * + * @param targetStats partition statistics to be updated. + * @param inputStats the partition statistics used as input. + */ + private static void appendStats(PartitionStatistics targetStats, PartitionStatistics inputStats) { + Preconditions.checkArgument(targetStats.specId() != null, "Invalid spec ID: null"); + Preconditions.checkArgument( + targetStats.specId().equals(inputStats.specId()), "Spec IDs must match"); + + // This is expected to be called on the compute/write path where we use full schemas, hence + // these members can't be null. + targetStats.set( + PartitionStatistics.DATA_RECORD_COUNT_POSITION, + targetStats.dataRecordCount() + inputStats.dataRecordCount()); + targetStats.set( + PartitionStatistics.DATA_FILE_COUNT_POSITION, + targetStats.dataFileCount() + inputStats.dataFileCount()); + targetStats.set( + PartitionStatistics.TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION, + targetStats.totalDataFileSizeInBytes() + inputStats.totalDataFileSizeInBytes()); + targetStats.set( + PartitionStatistics.POSITION_DELETE_RECORD_COUNT_POSITION, + targetStats.positionDeleteRecordCount() + inputStats.positionDeleteRecordCount()); + targetStats.set( + PartitionStatistics.POSITION_DELETE_FILE_COUNT_POSITION, + targetStats.positionDeleteFileCount() + inputStats.positionDeleteFileCount()); + targetStats.set( + PartitionStatistics.EQUALITY_DELETE_RECORD_COUNT_POSITION, + targetStats.equalityDeleteRecordCount() + inputStats.equalityDeleteRecordCount()); + targetStats.set( + PartitionStatistics.EQUALITY_DELETE_FILE_COUNT_POSITION, + targetStats.equalityDeleteFileCount() + inputStats.equalityDeleteFileCount()); + + if (inputStats.dvCount() != null) { + if (targetStats.dvCount() == null) { + targetStats.set(PartitionStatistics.DV_COUNT_POSITION, inputStats.dvCount()); + } else { + targetStats.set( + PartitionStatistics.DV_COUNT_POSITION, targetStats.dvCount() + inputStats.dvCount()); + } + } + + if (inputStats.totalRecords() != null) { + if (targetStats.totalRecords() == null) { + targetStats.set(PartitionStatistics.TOTAL_RECORD_COUNT_POSITION, inputStats.totalRecords()); + } else { + targetStats.set( + PartitionStatistics.TOTAL_RECORD_COUNT_POSITION, + targetStats.totalRecords() + inputStats.totalRecords()); + } + } + + if (inputStats.lastUpdatedAt() != null) { + updateSnapshotInfo( + targetStats, inputStats.lastUpdatedSnapshotId(), inputStats.lastUpdatedAt()); + } + } + + private static void updateSnapshotInfo( + PartitionStatistics stats, long snapshotId, long updatedAt) { + if (stats.lastUpdatedAt() == null || stats.lastUpdatedAt() < updatedAt) { + stats.set(PartitionStatistics.LAST_UPDATED_AT_POSITION, updatedAt); + stats.set(PartitionStatistics.LAST_UPDATED_SNAPSHOT_ID_POSITION, snapshotId); + } + } + private static class InvalidStatsFileException extends RuntimeException { InvalidStatsFileException(Throwable cause) { diff --git a/core/src/test/java/org/apache/iceberg/PartitionStatisticsTestBase.java b/core/src/test/java/org/apache/iceberg/PartitionStatisticsTestBase.java index 72a5405d7f1b..f6326e228cdc 100644 --- a/core/src/test/java/org/apache/iceberg/PartitionStatisticsTestBase.java +++ b/core/src/test/java/org/apache/iceberg/PartitionStatisticsTestBase.java @@ -44,19 +44,6 @@ public abstract class PartitionStatisticsTestBase { @TempDir private File temp; - // positions in StructLike - protected static final int DATA_RECORD_COUNT_POSITION = 2; - protected static final int DATA_FILE_COUNT_POSITION = 3; - protected static final int TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION = 4; - protected static final int POSITION_DELETE_RECORD_COUNT_POSITION = 5; - protected static final int POSITION_DELETE_FILE_COUNT_POSITION = 6; - protected static final int EQUALITY_DELETE_RECORD_COUNT_POSITION = 7; - protected static final int EQUALITY_DELETE_FILE_COUNT_POSITION = 8; - protected static final int TOTAL_RECORD_COUNT_POSITION = 9; - protected static final int LAST_UPDATED_AT_POSITION = 10; - protected static final int LAST_UPDATED_SNAPSHOT_ID_POSITION = 11; - protected static final int DV_COUNT_POSITION = 12; - protected static final Schema SCHEMA = new Schema( optional(1, "c1", Types.IntegerType.get()), @@ -85,18 +72,19 @@ protected Schema invalidOldSchema(Types.StructType unifiedPartitionType) { Types.NestedField.optional(11, LAST_UPDATED_SNAPSHOT_ID.name(), Types.LongType.get())); } - protected PartitionStats randomStats(Types.StructType partitionType) { + protected PartitionStatistics randomStats(Types.StructType partitionType) { PartitionData partitionData = new PartitionData(partitionType); partitionData.set(0, RANDOM.nextInt()); return randomStats(partitionData); } - protected PartitionStats randomStats(PartitionData partitionData) { - PartitionStats stats = new PartitionStats(partitionData, RANDOM.nextInt(10)); - stats.set(DATA_RECORD_COUNT_POSITION, RANDOM.nextLong()); - stats.set(DATA_FILE_COUNT_POSITION, RANDOM.nextInt()); - stats.set(TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION, 1024L * RANDOM.nextInt(20)); + protected PartitionStatistics randomStats(PartitionData partitionData) { + PartitionStatistics stats = new BasePartitionStatistics(partitionData, RANDOM.nextInt(10)); + stats.set(PartitionStatistics.DATA_RECORD_COUNT_POSITION, RANDOM.nextLong()); + stats.set(PartitionStatistics.DATA_FILE_COUNT_POSITION, RANDOM.nextInt()); + stats.set( + PartitionStatistics.TOTAL_DATA_FILE_SIZE_IN_BYTES_POSITION, 1024L * RANDOM.nextInt(20)); return stats; } diff --git a/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java b/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java index 9b93013a9b06..fd1e5ffe29b8 100644 --- a/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java +++ b/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java @@ -180,8 +180,8 @@ public void testAllDatatypePartitionWriting() throws Exception { partitionData.set(13, new BigDecimal("12345678901234567890.1234567890")); partitionData.set(14, Literal.of("10:10:10").to(Types.TimeType.get()).value()); - PartitionStats partitionStats = randomStats(partitionData); - List expected = Collections.singletonList(partitionStats); + PartitionStatistics partitionStats = randomStats(partitionData); + List expected = Collections.singletonList(partitionStats); PartitionStatisticsFile statisticsFile = PartitionStatsHandler.writePartitionStatsFile(testTable, 42L, dataSchema, expected); @@ -214,34 +214,34 @@ public void testOptionalFieldsWriting() throws Exception { Types.StructType partitionSchema = Partitioning.partitionType(testTable); Schema dataSchema = PartitionStatsHandler.schema(partitionSchema, formatVersion); - ImmutableList.Builder partitionListBuilder = ImmutableList.builder(); + ImmutableList.Builder partitionListBuilder = ImmutableList.builder(); for (int i = 0; i < 5; i++) { - PartitionStats stats = + PartitionStatistics stats = randomStats(dataSchema.findField(PARTITION_FIELD_ID).type().asStructType()); - stats.set(POSITION_DELETE_RECORD_COUNT_POSITION, null); - stats.set(POSITION_DELETE_FILE_COUNT_POSITION, null); - stats.set(EQUALITY_DELETE_RECORD_COUNT_POSITION, null); - stats.set(EQUALITY_DELETE_FILE_COUNT_POSITION, null); - stats.set(TOTAL_RECORD_COUNT_POSITION, null); - stats.set(LAST_UPDATED_AT_POSITION, null); - stats.set(LAST_UPDATED_SNAPSHOT_ID_POSITION, null); - stats.set(DV_COUNT_POSITION, null); + stats.set(PartitionStatistics.POSITION_DELETE_RECORD_COUNT_POSITION, null); + stats.set(PartitionStatistics.POSITION_DELETE_FILE_COUNT_POSITION, null); + stats.set(PartitionStatistics.EQUALITY_DELETE_RECORD_COUNT_POSITION, null); + stats.set(PartitionStatistics.EQUALITY_DELETE_FILE_COUNT_POSITION, null); + stats.set(PartitionStatistics.TOTAL_RECORD_COUNT_POSITION, null); + stats.set(PartitionStatistics.LAST_UPDATED_AT_POSITION, null); + stats.set(PartitionStatistics.LAST_UPDATED_SNAPSHOT_ID_POSITION, null); + stats.set(PartitionStatistics.DV_COUNT_POSITION, null); partitionListBuilder.add(stats); } - List expected = partitionListBuilder.build(); + List expected = partitionListBuilder.build(); assertThat(expected.get(0)) .extracting( - PartitionStats::positionDeleteRecordCount, - PartitionStats::positionDeleteFileCount, - PartitionStats::equalityDeleteRecordCount, - PartitionStats::equalityDeleteFileCount, - PartitionStats::totalRecords, - PartitionStats::lastUpdatedAt, - PartitionStats::lastUpdatedSnapshotId, - PartitionStats::dvCount) + PartitionStatistics::positionDeleteRecordCount, + PartitionStatistics::positionDeleteFileCount, + PartitionStatistics::equalityDeleteRecordCount, + PartitionStatistics::equalityDeleteFileCount, + PartitionStatistics::totalRecords, + PartitionStatistics::lastUpdatedAt, + PartitionStatistics::lastUpdatedSnapshotId, + PartitionStatistics::dvCount) .isEqualTo( Arrays.asList( 0L, 0, 0L, 0, null, null, null, 0)); // null counters must be initialized to zero. @@ -734,4 +734,29 @@ private static boolean isEqual( && Objects.equals(stats1.lastUpdatedAt(), stats2.lastUpdatedAt()) && Objects.equals(stats1.lastUpdatedSnapshotId(), stats2.lastUpdatedSnapshotId()); } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + private static boolean isEqual( + Comparator partitionComparator, + PartitionStats stats1, + PartitionStatistics stats2) { + if (stats1 == stats2) { + return true; + } else if (stats1 == null || stats2 == null) { + return false; + } + + return partitionComparator.compare(stats1.partition(), stats2.partition()) == 0 + && stats1.specId() == stats2.specId() + && stats1.dataRecordCount() == stats2.dataRecordCount() + && stats1.dataFileCount() == stats2.dataFileCount() + && stats1.totalDataFileSizeInBytes() == stats2.totalDataFileSizeInBytes() + && stats1.positionDeleteRecordCount() == stats2.positionDeleteRecordCount() + && stats1.positionDeleteFileCount() == stats2.positionDeleteFileCount() + && stats1.equalityDeleteRecordCount() == stats2.equalityDeleteRecordCount() + && stats1.equalityDeleteFileCount() == stats2.equalityDeleteFileCount() + && Objects.equals(stats1.totalRecords(), stats2.totalRecords()) + && Objects.equals(stats1.lastUpdatedAt(), stats2.lastUpdatedAt()) + && Objects.equals(stats1.lastUpdatedSnapshotId(), stats2.lastUpdatedSnapshotId()); + } } From f8ee29e6eb8b5f33ea0e91fa4406a76643cb4ef6 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 13 Jan 2026 00:14:17 +0800 Subject: [PATCH 189/201] Core: Drop support for Java 11 (#14400) --- .github/workflows/delta-conversion-ci.yml | 4 ++-- .github/workflows/flink-ci.yml | 2 +- .github/workflows/hive-ci.yml | 2 +- .github/workflows/java-ci.yml | 6 +++--- .github/workflows/kafka-connect-ci.yml | 2 +- .github/workflows/spark-ci.yml | 6 +----- README.md | 2 +- baseline.gradle | 2 +- build.gradle | 17 ++++++----------- jmh.gradle | 4 ++-- site/docs/contribute.md | 2 +- 11 files changed, 20 insertions(+), 29 deletions(-) diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index c7c71085b3f6..65da3a445f10 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -72,7 +72,7 @@ jobs: runs-on: ubuntu-24.04 strategy: matrix: - jvm: [11, 17, 21] + jvm: [17, 21] env: SPARK_LOCAL_IP: localhost steps: @@ -101,7 +101,7 @@ jobs: runs-on: ubuntu-24.04 strategy: matrix: - jvm: [11, 17, 21] + jvm: [17, 21] env: SPARK_LOCAL_IP: localhost steps: diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 7e8786c672fe..479cafd2fe18 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -74,7 +74,7 @@ jobs: runs-on: ubuntu-24.04 strategy: matrix: - jvm: [11, 17, 21] + jvm: [17, 21] flink: ['1.20', '2.0', '2.1'] env: SPARK_LOCAL_IP: localhost diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index 5827d3d2fd46..7044c30d0f95 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -73,7 +73,7 @@ jobs: runs-on: ubuntu-24.04 strategy: matrix: - jvm: [11, 17, 21] + jvm: [17, 21] env: SPARK_LOCAL_IP: localhost steps: diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 8ff7bb1d4cc8..0e48f3bb18ee 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -68,7 +68,7 @@ jobs: runs-on: ubuntu-24.04 strategy: matrix: - jvm: [11, 17, 21] + jvm: [17, 21] env: SPARK_LOCAL_IP: localhost steps: @@ -97,7 +97,7 @@ jobs: runs-on: ubuntu-24.04 strategy: matrix: - jvm: [11, 17, 21] + jvm: [17, 21] steps: - uses: actions/checkout@v6 - uses: actions/setup-java@v5 @@ -110,7 +110,7 @@ jobs: runs-on: ubuntu-24.04 strategy: matrix: - jvm: [11, 17, 21] + jvm: [17, 21] steps: - uses: actions/checkout@v6 - uses: actions/setup-java@v5 diff --git a/.github/workflows/kafka-connect-ci.yml b/.github/workflows/kafka-connect-ci.yml index 463a7a85f866..8074958f39b5 100644 --- a/.github/workflows/kafka-connect-ci.yml +++ b/.github/workflows/kafka-connect-ci.yml @@ -73,7 +73,7 @@ jobs: runs-on: ubuntu-24.04 strategy: matrix: - jvm: [11, 17, 21] + jvm: [17, 21] env: SPARK_LOCAL_IP: localhost steps: diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index 4daf93048f55..71428a100b6e 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -72,16 +72,12 @@ jobs: runs-on: ubuntu-24.04 strategy: matrix: - jvm: [11, 17, 21] + jvm: [17, 21] spark: ['3.4', '3.5', '4.0', '4.1'] scala: ['2.12', '2.13'] exclude: # Spark 3.5 is the first version not failing on Java 21 (https://issues.apache.org/jira/browse/SPARK-42369) # Full Java 21 support is coming in Spark 4 (https://issues.apache.org/jira/browse/SPARK-43831) - - jvm: 11 - spark: '4.0' - - jvm: 11 - spark: '4.1' - jvm: 21 spark: '3.4' - spark: '4.0' diff --git a/README.md b/README.md index 582e108b76ea..ea967e077fb2 100644 --- a/README.md +++ b/README.md @@ -51,7 +51,7 @@ Community discussions happen primarily on the [dev mailing list][dev-list] or on ### Building -Iceberg is built using Gradle with Java 11, 17, or 21. +Iceberg is built using Gradle with Java 17 or 21. * To invoke a build and run tests: `./gradlew build` * To skip tests: `./gradlew build -x test -x integrationTest` diff --git a/baseline.gradle b/baseline.gradle index 438687f431a1..7884c1a65ae3 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -58,7 +58,7 @@ subprojects { java { target 'src/main/java/**/*.java', 'src/test/java/**/*.java', 'src/testFixtures/java/**/*.java', 'src/jmh/java/**/*.java', 'src/integration/java/**/*.java' // 1.23.0 has an issue in formatting comments https://github.com/google/google-java-format/issues/1155 - // so we stick to 1.22.0 to produce consistent result for JDK 11/17/21 + // so we stick to 1.22.0 to produce consistent result for JDK 17/21 googleJavaFormat("1.22.0") removeUnusedImports() licenseHeaderFile "$rootDir/.baseline/copyright/copyright-header-java.txt" diff --git a/build.gradle b/build.gradle index 258b111ce516..d917b8d4c528 100644 --- a/build.gradle +++ b/build.gradle @@ -51,10 +51,7 @@ try { project.logger.error(e.getMessage()) } -if (JavaVersion.current() == JavaVersion.VERSION_11) { - project.ext.jdkVersion = '11' - project.ext.extraJvmArgs = [] -} else if (JavaVersion.current() == JavaVersion.VERSION_17 || JavaVersion.current() == JavaVersion.VERSION_21) { +if (JavaVersion.current() == JavaVersion.VERSION_17 || JavaVersion.current() == JavaVersion.VERSION_21) { project.ext.jdkVersion = JavaVersion.current().getMajorVersion().toString() project.ext.extraJvmArgs = ["--add-opens", "java.base/java.io=ALL-UNNAMED", "--add-opens", "java.base/java.lang.invoke=ALL-UNNAMED", @@ -78,7 +75,7 @@ if (JavaVersion.current() == JavaVersion.VERSION_11) { "--add-opens", "java.base/sun.security.action=ALL-UNNAMED", "--add-opens", "java.base/sun.util.calendar=ALL-UNNAMED"] } else { - throw new GradleException("This build must be run with JDK 11 or 17 or 21 but was executed with JDK " + JavaVersion.current()) + throw new GradleException("This build must be run with JDK 17 or 21 but was executed with JDK " + JavaVersion.current()) } tasks.withType(AbstractArchiveTask).configureEach { @@ -203,7 +200,7 @@ subprojects { tasks.withType(JavaCompile.class).configureEach { options.encoding = "UTF-8" - options.release = 11 + options.release = 17 } javadoc { @@ -262,9 +259,9 @@ subprojects { tasks.withType(ScalaCompile.class) { scalaCompileOptions.keepAliveMode.set(KeepAliveMode.DAEMON) // `options.release` doesn't seem to work for ScalaCompile :( - sourceCompatibility = "11" - targetCompatibility = "11" - scalaCompileOptions.additionalParameters.add("-release:11") + sourceCompatibility = "17" + targetCompatibility = "17" + scalaCompileOptions.additionalParameters.add("-release:17") } } } @@ -958,8 +955,6 @@ project(':iceberg-arrow') { } project(':iceberg-nessie') { - // Starting from Nessie 0.104.2, JDK 17+ is required for test dependencies - test.onlyIf { JavaVersion.current() != JavaVersion.VERSION_11 } test { useJUnitPlatform() } diff --git a/jmh.gradle b/jmh.gradle index d2c4709bf349..4df063f131a6 100644 --- a/jmh.gradle +++ b/jmh.gradle @@ -17,8 +17,8 @@ * under the License. */ -if (jdkVersion != '11' && jdkVersion != '17' && jdkVersion != '21') { - throw new GradleException("The JMH benchmarks must be run with JDK 11 or JDK 17 or JDK 21") +if (jdkVersion != '17' && jdkVersion != '21') { + throw new GradleException("The JMH benchmarks must be run with JDK 17 or JDK 21") } def flinkVersions = (System.getProperty("flinkVersions") != null ? System.getProperty("flinkVersions") : System.getProperty("defaultFlinkVersions")).split(",") diff --git a/site/docs/contribute.md b/site/docs/contribute.md index 54aa076639ce..2389f01b11b9 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -95,7 +95,7 @@ settle disagreements or to force a decision. ## Building the Project Locally -Iceberg is built using Gradle with Java 11, 17, or 21. +Iceberg is built using Gradle with Java 17 or 21. * To invoke a build and run tests: `./gradlew build` * To skip tests: `./gradlew build -x test -x integrationTest` From cbf07cba8c89a49e42baa7073ee15bd914d2e1e0 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 13 Jan 2026 13:00:28 +0100 Subject: [PATCH 190/201] AWS, Azure, Core, GCP: Pass planId when refreshing vended credentials (#14767) --- .../aws/s3/VendedCredentialsProvider.java | 5 +- .../aws/s3/TestVendedCredentialsProvider.java | 50 +++++++++++++++++++ .../adlsv2/VendedAdlsCredentialProvider.java | 5 +- .../TestVendedAdlsCredentialProvider.java | 44 ++++++++++++++++ .../iceberg/rest/RESTCatalogProperties.java | 2 + .../iceberg/rest/RESTSessionCatalog.java | 4 +- .../org/apache/iceberg/rest/RESTTable.java | 13 ++++- .../apache/iceberg/rest/RESTTableScan.java | 50 ++++++++++++++++++- .../gcs/OAuth2RefreshCredentialsHandler.java | 5 +- .../TestOAuth2RefreshCredentialsHandler.java | 42 ++++++++++++++++ 10 files changed, 212 insertions(+), 8 deletions(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/VendedCredentialsProvider.java b/aws/src/main/java/org/apache/iceberg/aws/s3/VendedCredentialsProvider.java index fc42bd789859..9afa28635c41 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/VendedCredentialsProvider.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/VendedCredentialsProvider.java @@ -29,6 +29,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.rest.ErrorHandlers; import org.apache.iceberg.rest.HTTPClient; +import org.apache.iceberg.rest.RESTCatalogProperties; import org.apache.iceberg.rest.RESTClient; import org.apache.iceberg.rest.auth.AuthManager; import org.apache.iceberg.rest.auth.AuthManagers; @@ -50,6 +51,7 @@ public class VendedCredentialsProvider implements AwsCredentialsProvider, SdkAut private final CachedSupplier credentialCache; private final String catalogEndpoint; private final String credentialsEndpoint; + private final String planId; private AuthManager authManager; private AuthSession authSession; @@ -65,6 +67,7 @@ private VendedCredentialsProvider(Map properties) { .build(); this.catalogEndpoint = properties.get(CatalogProperties.URI); this.credentialsEndpoint = properties.get(URI); + this.planId = properties.getOrDefault(RESTCatalogProperties.REST_SCAN_PLAN_ID, null); } @Override @@ -103,7 +106,7 @@ private LoadCredentialsResponse fetchCredentials() { return httpClient() .get( credentialsEndpoint, - null, + null != planId ? Map.of("planId", planId) : null, LoadCredentialsResponse.class, Map.of(), ErrorHandlers.defaultErrorHandler()); diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestVendedCredentialsProvider.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestVendedCredentialsProvider.java index 8e2f99e0ccbe..d0287dc3080c 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestVendedCredentialsProvider.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestVendedCredentialsProvider.java @@ -26,10 +26,12 @@ import java.time.Instant; import java.time.temporal.ChronoUnit; +import java.util.Map; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.exceptions.RESTException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.rest.HttpMethod; +import org.apache.iceberg.rest.RESTCatalogProperties; import org.apache.iceberg.rest.credentials.Credential; import org.apache.iceberg.rest.credentials.ImmutableCredential; import org.apache.iceberg.rest.responses.ImmutableLoadCredentialsResponse; @@ -488,6 +490,54 @@ public void invalidTokenInProperties() { mockServer.verify(mockRequest, VerificationTimes.once()); } + @Test + public void planIdQueryParamIsSent() { + String planId = "randomPlanId"; + HttpRequest mockRequest = + request("/v1/credentials") + .withMethod(HttpMethod.GET.name()) + .withQueryStringParameter("planId", planId); + Credential credential = + ImmutableCredential.builder() + .prefix("s3") + .config( + ImmutableMap.of( + S3FileIOProperties.ACCESS_KEY_ID, + "randomAccessKey", + S3FileIOProperties.SECRET_ACCESS_KEY, + "randomSecretAccessKey", + S3FileIOProperties.SESSION_TOKEN, + "sessionToken", + S3FileIOProperties.SESSION_TOKEN_EXPIRES_AT_MS, + Long.toString(Instant.now().plus(1, ChronoUnit.MINUTES).toEpochMilli()))) + .build(); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder().addCredentials(credential).build(); + + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + Map properties = + ImmutableMap.builder() + .putAll(PROPERTIES) + .put(RESTCatalogProperties.REST_SCAN_PLAN_ID, planId) + .build(); + + try (VendedCredentialsProvider provider = VendedCredentialsProvider.create(properties)) { + AwsCredentials awsCredentials = provider.resolveCredentials(); + verifyCredentials(awsCredentials, credential); + + // resolving credentials multiple times should hit the credentials endpoint again and send the + // planId again + AwsCredentials refreshedCredentials = provider.resolveCredentials(); + assertThat(refreshedCredentials).isNotSameAs(awsCredentials); + verifyCredentials(refreshedCredentials, credential); + } + + mockServer.verify(mockRequest, VerificationTimes.exactly(2)); + } + private void verifyCredentials(AwsCredentials awsCredentials, Credential credential) { assertThat(awsCredentials).isInstanceOf(AwsSessionCredentials.class); AwsSessionCredentials creds = (AwsSessionCredentials) awsCredentials; diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java index 2ede484a732d..6660e765ecf8 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/VendedAdlsCredentialProvider.java @@ -38,6 +38,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.rest.ErrorHandlers; import org.apache.iceberg.rest.HTTPClient; +import org.apache.iceberg.rest.RESTCatalogProperties; import org.apache.iceberg.rest.RESTClient; import org.apache.iceberg.rest.auth.AuthManager; import org.apache.iceberg.rest.auth.AuthManagers; @@ -54,6 +55,7 @@ public class VendedAdlsCredentialProvider implements Serializable, AutoCloseable private final SerializableMap properties; private final String credentialsEndpoint; private final String catalogEndpoint; + private final String planId; private transient volatile Map sasCredentialByAccount; private transient volatile HTTPClient client; private transient AuthManager authManager; @@ -67,6 +69,7 @@ public VendedAdlsCredentialProvider(Map properties) { this.properties = SerializableMap.copyOf(properties); this.credentialsEndpoint = properties.get(URI); this.catalogEndpoint = properties.get(CatalogProperties.URI); + this.planId = properties.getOrDefault(RESTCatalogProperties.REST_SCAN_PLAN_ID, null); } Mono credentialForAccount(String storageAccount) { @@ -165,7 +168,7 @@ private LoadCredentialsResponse fetchCredentials() { return httpClient() .get( credentialsEndpoint, - null, + null != planId ? Map.of("planId", planId) : null, LoadCredentialsResponse.class, Map.of(), ErrorHandlers.defaultErrorHandler()); diff --git a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/TestVendedAdlsCredentialProvider.java b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/TestVendedAdlsCredentialProvider.java index 43958d39acd8..6af2e4de21f0 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/adlsv2/TestVendedAdlsCredentialProvider.java +++ b/azure/src/test/java/org/apache/iceberg/azure/adlsv2/TestVendedAdlsCredentialProvider.java @@ -34,6 +34,7 @@ import org.apache.iceberg.TestHelpers; import org.apache.iceberg.exceptions.RESTException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.RESTCatalogProperties; import org.apache.iceberg.rest.credentials.Credential; import org.apache.iceberg.rest.credentials.ImmutableCredential; import org.apache.iceberg.rest.responses.ImmutableLoadCredentialsResponse; @@ -445,4 +446,47 @@ public void serializableTest( mockServer.verify(mockRequest, VerificationTimes.exactly(2)); } + + @Test + public void planIdQueryParamIsSent() { + String planId = "randomPlanId"; + HttpRequest mockRequest = + request("/v1/credentials") + .withMethod(HttpMethod.GET.name()) + .withQueryStringParameter("planId", planId); + Credential credential = + ImmutableCredential.builder() + .prefix(CREDENTIAL_PREFIX) + .config( + ImmutableMap.of( + ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT, + "randomSasToken", + ADLS_SAS_TOKEN_EXPIRES_AT_MS_PREFIX + STORAGE_ACCOUNT, + Long.toString(Instant.now().minus(1, ChronoUnit.MINUTES).toEpochMilli()))) + .build(); + LoadCredentialsResponse response = + ImmutableLoadCredentialsResponse.builder().addCredentials(credential).build(); + HttpResponse mockResponse = + response(LoadCredentialsResponseParser.toJson(response)).withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + Map properties = + ImmutableMap.builder() + .putAll(PROPERTIES) + .put(RESTCatalogProperties.REST_SCAN_PLAN_ID, planId) + .build(); + try (VendedAdlsCredentialProvider provider = new VendedAdlsCredentialProvider(properties)) { + String azureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT).block(); + assertThat(azureSasCredential) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + + // resolving credentials multiple times should hit the credentials endpoint again and send the + // planId again + String refreshedAzureSasCredential = provider.credentialForAccount(STORAGE_ACCOUNT).block(); + assertThat(refreshedAzureSasCredential) + .isEqualTo(credential.config().get(ADLS_SAS_TOKEN_PREFIX + STORAGE_ACCOUNT)); + } + + mockServer.verify(mockRequest, VerificationTimes.exactly(2)); + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTCatalogProperties.java b/core/src/main/java/org/apache/iceberg/rest/RESTCatalogProperties.java index 79617b2982ff..72b09fa772c5 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTCatalogProperties.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTCatalogProperties.java @@ -41,6 +41,8 @@ private RESTCatalogProperties() {} public static final String REST_SCAN_PLANNING_ENABLED = "rest-scan-planning-enabled"; public static final boolean REST_SCAN_PLANNING_ENABLED_DEFAULT = false; + public static final String REST_SCAN_PLAN_ID = "rest-scan-plan-id"; + public enum SnapshotMode { ALL, REFS diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 61e25d3d4fc6..4ff13ac82417 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -516,7 +516,9 @@ private RESTTable restTableForScanPlanning( Map::of, finalIdentifier, paths, - endpoints); + endpoints, + properties(), + conf); } return null; } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTTable.java b/core/src/main/java/org/apache/iceberg/rest/RESTTable.java index 0abe41e25f50..8be5590bfa62 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTTable.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTTable.java @@ -38,6 +38,8 @@ class RESTTable extends BaseTable implements RequiresRemoteScanPlanning { private final ResourcePaths resourcePaths; private final TableIdentifier tableIdentifier; private final Set supportedEndpoints; + private final Map catalogProperties; + private final Object hadoopConf; RESTTable( TableOperations ops, @@ -47,7 +49,9 @@ class RESTTable extends BaseTable implements RequiresRemoteScanPlanning { Supplier> headers, TableIdentifier tableIdentifier, ResourcePaths resourcePaths, - Set supportedEndpoints) { + Set supportedEndpoints, + Map catalogProperties, + Object hadoopConf) { super(ops, name, reporter); this.reporter = reporter; this.client = client; @@ -55,6 +59,8 @@ class RESTTable extends BaseTable implements RequiresRemoteScanPlanning { this.tableIdentifier = tableIdentifier; this.resourcePaths = resourcePaths; this.supportedEndpoints = supportedEndpoints; + this.catalogProperties = catalogProperties; + this.hadoopConf = hadoopConf; } @Override @@ -68,7 +74,10 @@ public TableScan newScan() { operations(), tableIdentifier, resourcePaths, - supportedEndpoints); + supportedEndpoints, + io(), + catalogProperties, + hadoopConf); } @Override diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java b/core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java index dd886f9d2717..460c6896e986 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTTableScan.java @@ -26,6 +26,8 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.DataTableScan; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.Schema; @@ -35,8 +37,12 @@ import org.apache.iceberg.TableScanContext; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.StorageCredential; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.credentials.Credential; import org.apache.iceberg.rest.requests.PlanTableScanRequest; import org.apache.iceberg.rest.responses.FetchPlanningResultResponse; import org.apache.iceberg.rest.responses.PlanTableScanResponse; @@ -51,6 +57,7 @@ class RESTTableScan extends DataTableScan { private static final int MAX_ATTEMPTS = 10; // Max number of poll checks private static final long MAX_WAIT_TIME_MS = 5 * 60 * 1000; // Total maximum duration (5 minutes) private static final double SCALE_FACTOR = 2.0; // Exponential scale factor + private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO"; private final RESTClient client; private final Map headers; @@ -60,7 +67,11 @@ class RESTTableScan extends DataTableScan { private final TableIdentifier tableIdentifier; private final Set supportedEndpoints; private final ParserContext parserContext; + private final Map catalogProperties; + private final Object hadoopConf; + private final FileIO tableIo; private String planId = null; + private FileIO fileIOForPlanId = null; RESTTableScan( Table table, @@ -71,7 +82,10 @@ class RESTTableScan extends DataTableScan { TableOperations operations, TableIdentifier tableIdentifier, ResourcePaths resourcePaths, - Set supportedEndpoints) { + Set supportedEndpoints, + FileIO tableIo, + Map catalogProperties, + Object hadoopConf) { super(table, schema, context); this.table = table; this.client = client; @@ -85,6 +99,9 @@ class RESTTableScan extends DataTableScan { .add("specsById", table.specs()) .add("caseSensitive", context().caseSensitive()) .build(); + this.tableIo = tableIo; + this.catalogProperties = catalogProperties; + this.hadoopConf = hadoopConf; } @Override @@ -99,7 +116,15 @@ protected TableScan newRefinedScan( operations, tableIdentifier, resourcePaths, - supportedEndpoints); + supportedEndpoints, + io(), + catalogProperties, + hadoopConf); + } + + @Override + protected FileIO io() { + return null != fileIOForPlanId ? fileIOForPlanId : tableIo; } @Override @@ -151,6 +176,10 @@ private CloseableIterable planTableScan(PlanTableScanRequest planT this.planId = response.planId(); PlanStatus planStatus = response.planStatus(); + if (null != planId && !response.credentials().isEmpty()) { + this.fileIOForPlanId = fileIOForPlanId(response.credentials()); + } + switch (planStatus) { case COMPLETED: return scanTasksIterable(response.planTasks(), response.fileScanTasks()); @@ -169,6 +198,19 @@ private CloseableIterable planTableScan(PlanTableScanRequest planT } } + private FileIO fileIOForPlanId(List storageCredentials) { + return CatalogUtil.loadFileIO( + catalogProperties.getOrDefault(CatalogProperties.FILE_IO_IMPL, DEFAULT_FILE_IO_IMPL), + ImmutableMap.builder() + .putAll(catalogProperties) + .put(RESTCatalogProperties.REST_SCAN_PLAN_ID, planId) + .buildKeepingLast(), + hadoopConf, + storageCredentials.stream() + .map(c -> StorageCredential.create(c.prefix(), c.config())) + .collect(Collectors.toList())); + } + private CloseableIterable fetchPlanningResult() { RetryPolicy retryPolicy = RetryPolicy.builder() @@ -216,6 +258,10 @@ private CloseableIterable fetchPlanningResult() { response.planStatus(), planId); + if (!response.credentials().isEmpty()) { + this.fileIOForPlanId = fileIOForPlanId(response.credentials()); + } + return scanTasksIterable(response.planTasks(), response.fileScanTasks()); } catch (FailsafeException e) { // FailsafeException is thrown when retries are exhausted (Max Attempts/Duration) diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandler.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandler.java index e46e254dd33e..a8cd6a8cf034 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandler.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/OAuth2RefreshCredentialsHandler.java @@ -32,6 +32,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.rest.ErrorHandlers; import org.apache.iceberg.rest.HTTPClient; +import org.apache.iceberg.rest.RESTCatalogProperties; import org.apache.iceberg.rest.RESTClient; import org.apache.iceberg.rest.auth.AuthManager; import org.apache.iceberg.rest.auth.AuthManagers; @@ -43,6 +44,7 @@ public class OAuth2RefreshCredentialsHandler implements OAuth2CredentialsWithRefresh.OAuth2RefreshHandler, AutoCloseable { private final Map properties; private final String credentialsEndpoint; + private final String planId; // will be used to refresh the OAuth2 token private final String catalogEndpoint; private volatile HTTPClient client; @@ -59,6 +61,7 @@ private OAuth2RefreshCredentialsHandler(Map properties) { properties.get(GCPProperties.GCS_OAUTH2_REFRESH_CREDENTIALS_ENDPOINT); this.catalogEndpoint = properties.get(CatalogProperties.URI); this.properties = properties; + this.planId = properties.getOrDefault(RESTCatalogProperties.REST_SCAN_PLAN_ID, null); } @SuppressWarnings("JavaUtilDate") // GCP API uses java.util.Date @@ -68,7 +71,7 @@ public AccessToken refreshAccessToken() { httpClient() .get( credentialsEndpoint, - null, + null != planId ? Map.of("planId", planId) : null, LoadCredentialsResponse.class, Map.of(), ErrorHandlers.defaultErrorHandler()); diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestOAuth2RefreshCredentialsHandler.java b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestOAuth2RefreshCredentialsHandler.java index 8e5e970a9a9c..0f29b9781895 100644 --- a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestOAuth2RefreshCredentialsHandler.java +++ b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/TestOAuth2RefreshCredentialsHandler.java @@ -32,6 +32,7 @@ import org.apache.iceberg.gcp.GCPProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.rest.HttpMethod; +import org.apache.iceberg.rest.RESTCatalogProperties; import org.apache.iceberg.rest.credentials.Credential; import org.apache.iceberg.rest.credentials.ImmutableCredential; import org.apache.iceberg.rest.responses.ImmutableLoadCredentialsResponse; @@ -274,4 +275,45 @@ public void multipleGcsCredentials() { .isInstanceOf(IllegalStateException.class) .hasMessage("Invalid GCS Credentials: only one GCS credential should exist"); } + + @Test + public void planIdQueryParamIsSent() { + String planId = "randomPlanId"; + HttpRequest mockRequest = + HttpRequest.request("/v1/credentials") + .withMethod(HttpMethod.GET.name()) + .withQueryStringParameter("planId", planId); + + Credential credential = + ImmutableCredential.builder() + .prefix("gs") + .config( + ImmutableMap.of( + GCPProperties.GCS_OAUTH2_TOKEN, + "gcsToken", + GCPProperties.GCS_OAUTH2_TOKEN_EXPIRES_AT, + Long.toString(Instant.now().plus(5, ChronoUnit.MINUTES).toEpochMilli()))) + .build(); + HttpResponse mockResponse = + HttpResponse.response( + LoadCredentialsResponseParser.toJson( + ImmutableLoadCredentialsResponse.builder().addCredentials(credential).build())) + .withStatusCode(200); + mockServer.when(mockRequest).respond(mockResponse); + + Map properties = + ImmutableMap.builder() + .putAll(PROPERTIES) + .put(RESTCatalogProperties.REST_SCAN_PLAN_ID, planId) + .build(); + OAuth2RefreshCredentialsHandler handler = OAuth2RefreshCredentialsHandler.create(properties); + + AccessToken accessToken = handler.refreshAccessToken(); + + // refresh always fetches a new token and sends the planId again + AccessToken refreshedToken = handler.refreshAccessToken(); + assertThat(refreshedToken).isNotSameAs(accessToken); + + mockServer.verify(mockRequest, VerificationTimes.exactly(2)); + } } From 42c7f475d2cde9c53fb6578af24c64d407c9b6b1 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Tue, 13 Jan 2026 13:59:03 +0100 Subject: [PATCH 191/201] Core, Data, Spark: Use partition stats scan API in tests (#14996) --- .../PartitionStatisticsScanTestBase.java | 26 ---- .../iceberg/PartitionStatisticsTestBase.java | 27 ++++ .../PartitionStatsHandlerTestBase.java | 118 +++++++++--------- .../PartitionStatsHandlerBenchmark.java | 15 +-- .../TestComputePartitionStatsAction.java | 47 ++++--- .../TestRewriteDataFilesProcedure.java | 43 ++++--- .../TestRewriteManifestsProcedure.java | 43 ++++--- .../TestComputePartitionStatsAction.java | 50 ++++---- .../TestRewriteDataFilesProcedure.java | 43 ++++--- .../TestRewriteManifestsProcedure.java | 43 ++++--- .../TestComputePartitionStatsAction.java | 50 ++++---- .../TestRewriteDataFilesProcedure.java | 43 ++++--- .../TestRewriteManifestsProcedure.java | 43 ++++--- .../TestComputePartitionStatsAction.java | 50 ++++---- 14 files changed, 305 insertions(+), 336 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/PartitionStatisticsScanTestBase.java b/core/src/test/java/org/apache/iceberg/PartitionStatisticsScanTestBase.java index 89eb70959c6d..e48d26a71f97 100644 --- a/core/src/test/java/org/apache/iceberg/PartitionStatisticsScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/PartitionStatisticsScanTestBase.java @@ -27,7 +27,6 @@ import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.Objects; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -452,29 +451,4 @@ private static void computeAndValidatePartitionStats( PartitionStatistics::dvCount) .containsExactlyInAnyOrder(expectedValues); } - - @SuppressWarnings("checkstyle:CyclomaticComplexity") - protected static boolean isEqual( - Comparator partitionComparator, - PartitionStatistics stats1, - PartitionStatistics stats2) { - if (stats1 == stats2) { - return true; - } else if (stats1 == null || stats2 == null) { - return false; - } - - return partitionComparator.compare(stats1.partition(), stats2.partition()) == 0 - && Objects.equals(stats1.specId(), stats2.specId()) - && Objects.equals(stats1.dataRecordCount(), stats2.dataRecordCount()) - && Objects.equals(stats1.dataFileCount(), stats2.dataFileCount()) - && Objects.equals(stats1.totalDataFileSizeInBytes(), stats2.totalDataFileSizeInBytes()) - && Objects.equals(stats1.positionDeleteRecordCount(), stats2.positionDeleteRecordCount()) - && Objects.equals(stats1.positionDeleteFileCount(), stats2.positionDeleteFileCount()) - && Objects.equals(stats1.equalityDeleteRecordCount(), stats2.equalityDeleteRecordCount()) - && Objects.equals(stats1.equalityDeleteFileCount(), stats2.equalityDeleteFileCount()) - && Objects.equals(stats1.totalRecords(), stats2.totalRecords()) - && Objects.equals(stats1.lastUpdatedAt(), stats2.lastUpdatedAt()) - && Objects.equals(stats1.lastUpdatedSnapshotId(), stats2.lastUpdatedSnapshotId()); - } } diff --git a/core/src/test/java/org/apache/iceberg/PartitionStatisticsTestBase.java b/core/src/test/java/org/apache/iceberg/PartitionStatisticsTestBase.java index f6326e228cdc..5324db50f5da 100644 --- a/core/src/test/java/org/apache/iceberg/PartitionStatisticsTestBase.java +++ b/core/src/test/java/org/apache/iceberg/PartitionStatisticsTestBase.java @@ -34,6 +34,8 @@ import java.io.File; import java.io.IOException; +import java.util.Comparator; +import java.util.Objects; import java.util.Random; import java.util.concurrent.ThreadLocalRandom; import org.apache.iceberg.data.GenericRecord; @@ -99,4 +101,29 @@ protected static StructLike partitionRecord( record.set(1, val2); return record; } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + protected static boolean isEqual( + Comparator partitionComparator, + PartitionStatistics stats1, + PartitionStatistics stats2) { + if (stats1 == stats2) { + return true; + } else if (stats1 == null || stats2 == null) { + return false; + } + + return partitionComparator.compare(stats1.partition(), stats2.partition()) == 0 + && Objects.equals(stats1.specId(), stats2.specId()) + && Objects.equals(stats1.dataRecordCount(), stats2.dataRecordCount()) + && Objects.equals(stats1.dataFileCount(), stats2.dataFileCount()) + && Objects.equals(stats1.totalDataFileSizeInBytes(), stats2.totalDataFileSizeInBytes()) + && Objects.equals(stats1.positionDeleteRecordCount(), stats2.positionDeleteRecordCount()) + && Objects.equals(stats1.positionDeleteFileCount(), stats2.positionDeleteFileCount()) + && Objects.equals(stats1.equalityDeleteRecordCount(), stats2.equalityDeleteRecordCount()) + && Objects.equals(stats1.equalityDeleteFileCount(), stats2.equalityDeleteFileCount()) + && Objects.equals(stats1.totalRecords(), stats2.totalRecords()) + && Objects.equals(stats1.lastUpdatedAt(), stats2.lastUpdatedAt()) + && Objects.equals(stats1.lastUpdatedSnapshotId(), stats2.lastUpdatedSnapshotId()); + } } diff --git a/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java b/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java index fd1e5ffe29b8..d5442e209dcc 100644 --- a/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java +++ b/core/src/test/java/org/apache/iceberg/PartitionStatsHandlerTestBase.java @@ -182,13 +182,29 @@ public void testAllDatatypePartitionWriting() throws Exception { PartitionStatistics partitionStats = randomStats(partitionData); List expected = Collections.singletonList(partitionStats); - PartitionStatisticsFile statisticsFile = - PartitionStatsHandler.writePartitionStatsFile(testTable, 42L, dataSchema, expected); - List written; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, testTable.io().newInputFile(statisticsFile.path()))) { + // Add a dummy file to the table to have a snapshot + DataFile dataFile = + DataFiles.builder(spec) + .withPath("some_path") + .withPartition(partitionData) + .withFileSizeInBytes(15) + .withFormat(FileFormat.PARQUET) + .withRecordCount(1) + .build(); + testTable.newAppend().appendFile(dataFile).commit(); + long snapshotId = testTable.currentSnapshot().snapshotId(); + + testTable + .updatePartitionStatistics() + .setPartitionStatistics( + PartitionStatsHandler.writePartitionStatsFile( + testTable, snapshotId, dataSchema, expected)) + .commit(); + + List written; + try (CloseableIterable recordIterator = + testTable.newPartitionStatisticsScan().useSnapshot(snapshotId).scan()) { written = Lists.newArrayList(recordIterator); } @@ -246,13 +262,27 @@ public void testOptionalFieldsWriting() throws Exception { Arrays.asList( 0L, 0, 0L, 0, null, null, null, 0)); // null counters must be initialized to zero. - PartitionStatisticsFile statisticsFile = - PartitionStatsHandler.writePartitionStatsFile(testTable, 42L, dataSchema, expected); - - List written; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, testTable.io().newInputFile(statisticsFile.path()))) { + // Add a dummy file to the table to have a snapshot + DataFile dataFile = + DataFiles.builder(spec) + .withPath("some_path") + .withFileSizeInBytes(15) + .withFormat(FileFormat.PARQUET) + .withRecordCount(1) + .build(); + testTable.newAppend().appendFile(dataFile).commit(); + long snapshotId = testTable.currentSnapshot().snapshotId(); + + testTable + .updatePartitionStatistics() + .setPartitionStatistics( + PartitionStatsHandler.writePartitionStatsFile( + testTable, snapshotId, dataSchema, expected)) + .commit(); + + List written; + try (CloseableIterable recordIterator = + testTable.newPartitionStatisticsScan().useSnapshot(snapshotId).scan()) { written = Lists.newArrayList(recordIterator); } @@ -464,27 +494,24 @@ public void testCopyOnWriteDelete() throws Exception { testTable.newAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); - PartitionStatisticsFile statisticsFile = - PartitionStatsHandler.computeAndWriteStatsFile(testTable); - testTable.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); + testTable + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(testTable)) + .commit(); - assertThat( - PartitionStatsHandler.readPartitionStatsFile( - PartitionStatsHandler.schema(Partitioning.partitionType(testTable), 2), - testTable.io().newInputFile(statisticsFile.path()))) + assertThat(testTable.newPartitionStatisticsScan().scan()) .allMatch(s -> (s.dataRecordCount() != 0 && s.dataFileCount() != 0)); testTable.newDelete().deleteFile(dataFile1).commit(); testTable.newDelete().deleteFile(dataFile2).commit(); - PartitionStatisticsFile statisticsFileNew = - PartitionStatsHandler.computeAndWriteStatsFile(testTable); + testTable + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(testTable)) + .commit(); // stats must be decremented to zero as all the files removed from table. - assertThat( - PartitionStatsHandler.readPartitionStatsFile( - PartitionStatsHandler.schema(Partitioning.partitionType(testTable), 2), - testTable.io().newInputFile(statisticsFileNew.path()))) + assertThat(testTable.newPartitionStatisticsScan().scan()) .allMatch(s -> (s.dataRecordCount() == 0 && s.dataFileCount() == 0)); } @@ -616,15 +643,15 @@ public void testFullComputeFallbackWithInvalidStats() throws Exception { testTable.updatePartitionStatistics().setPartitionStatistics(invalidStatisticsFile).commit(); testTable.newAppend().appendFile(dataFile).commit(); - PartitionStatisticsFile statisticsFile = - PartitionStatsHandler.computeAndWriteStatsFile(testTable); + testTable + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(testTable)) + .commit(); // read the partition entries from the stats file - List partitionStats; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - PartitionStatsHandler.schema(partitionType, 2), - testTable.io().newInputFile(statisticsFile.path()))) { + List partitionStats; + try (CloseableIterable recordIterator = + testTable.newPartitionStatisticsScan().scan()) { partitionStats = Lists.newArrayList(recordIterator); } @@ -734,29 +761,4 @@ private static boolean isEqual( && Objects.equals(stats1.lastUpdatedAt(), stats2.lastUpdatedAt()) && Objects.equals(stats1.lastUpdatedSnapshotId(), stats2.lastUpdatedSnapshotId()); } - - @SuppressWarnings("checkstyle:CyclomaticComplexity") - private static boolean isEqual( - Comparator partitionComparator, - PartitionStats stats1, - PartitionStatistics stats2) { - if (stats1 == stats2) { - return true; - } else if (stats1 == null || stats2 == null) { - return false; - } - - return partitionComparator.compare(stats1.partition(), stats2.partition()) == 0 - && stats1.specId() == stats2.specId() - && stats1.dataRecordCount() == stats2.dataRecordCount() - && stats1.dataFileCount() == stats2.dataFileCount() - && stats1.totalDataFileSizeInBytes() == stats2.totalDataFileSizeInBytes() - && stats1.positionDeleteRecordCount() == stats2.positionDeleteRecordCount() - && stats1.positionDeleteFileCount() == stats2.positionDeleteFileCount() - && stats1.equalityDeleteRecordCount() == stats2.equalityDeleteRecordCount() - && stats1.equalityDeleteFileCount() == stats2.equalityDeleteFileCount() - && Objects.equals(stats1.totalRecords(), stats2.totalRecords()) - && Objects.equals(stats1.lastUpdatedAt(), stats2.lastUpdatedAt()) - && Objects.equals(stats1.lastUpdatedSnapshotId(), stats2.lastUpdatedSnapshotId()); - } } diff --git a/data/src/jmh/java/org/apache/iceberg/PartitionStatsHandlerBenchmark.java b/data/src/jmh/java/org/apache/iceberg/PartitionStatsHandlerBenchmark.java index 938dc2863784..1995e8d02d28 100644 --- a/data/src/jmh/java/org/apache/iceberg/PartitionStatsHandlerBenchmark.java +++ b/data/src/jmh/java/org/apache/iceberg/PartitionStatsHandlerBenchmark.java @@ -99,13 +99,14 @@ public void tearDownBenchmark() { @Benchmark @Threads(1) public void benchmarkPartitionStats() throws IOException { - PartitionStatisticsFile statisticsFile = PartitionStatsHandler.computeAndWriteStatsFile(table); - - List stats; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - PartitionStatsHandler.schema(Partitioning.partitionType(table), 2), - Files.localInput(statisticsFile.path()))) { + table + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(table)) + .commit(); + + List stats; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().scan()) { stats = Lists.newArrayList(recordIterator); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java index 303411eb7ddd..cb405886cb55 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java @@ -23,9 +23,8 @@ import java.io.IOException; import java.util.List; -import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionStatistics; import org.apache.iceberg.PartitionStatisticsFile; -import org.apache.iceberg.PartitionStats; import org.apache.iceberg.PartitionStatsHandler; import org.apache.iceberg.Partitioning; import org.apache.iceberg.Schema; @@ -120,10 +119,9 @@ public void partitionStatsComputeOnLatestSnapshot() throws IOException { assertThat(table.partitionStatisticsFiles()).containsExactly(statisticsFile); Types.StructType partitionType = Partitioning.partitionType(table); - Schema dataSchema = PartitionStatsHandler.schema(partitionType, 2); validatePartitionStats( - statisticsFile, - dataSchema, + table, + statisticsFile.snapshotId(), Tuple.tuple( partitionRecord(partitionType, "foo", "A"), DEFAULT_SPEC_ID, @@ -212,8 +210,8 @@ public void partitionStatsComputeOnSnapshot() throws IOException { Schema dataSchema = PartitionStatsHandler.schema(partitionType, 2); // should contain stats for only partitions of snapshot1 (no entry for partition bar, A) validatePartitionStats( - statisticsFile, - dataSchema, + table, + statisticsFile.snapshotId(), Tuple.tuple( partitionRecord(partitionType, "foo", "A"), DEFAULT_SPEC_ID, @@ -272,31 +270,28 @@ private void createPartitionedTableV1() { tableName); } - private void validatePartitionStats( - PartitionStatisticsFile result, Schema recordSchema, Tuple... expectedValues) + private void validatePartitionStats(Table table, long snapshotId, Tuple... expectedValues) throws IOException { - // read the partition entries from the stats file - List partitionStats; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - recordSchema, Files.localInput(result.path()))) { + List partitionStats; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().useSnapshot(snapshotId).scan()) { partitionStats = Lists.newArrayList(recordIterator); } assertThat(partitionStats) .extracting( - PartitionStats::partition, - PartitionStats::specId, - PartitionStats::dataRecordCount, - PartitionStats::dataFileCount, - PartitionStats::totalDataFileSizeInBytes, - PartitionStats::positionDeleteRecordCount, - PartitionStats::positionDeleteFileCount, - PartitionStats::equalityDeleteRecordCount, - PartitionStats::equalityDeleteFileCount, - PartitionStats::totalRecords, - PartitionStats::lastUpdatedAt, - PartitionStats::lastUpdatedSnapshotId) + PartitionStatistics::partition, + PartitionStatistics::specId, + PartitionStatistics::dataRecordCount, + PartitionStatistics::dataFileCount, + PartitionStatistics::totalDataFileSizeInBytes, + PartitionStatistics::positionDeleteRecordCount, + PartitionStatistics::positionDeleteFileCount, + PartitionStatistics::equalityDeleteRecordCount, + PartitionStatistics::equalityDeleteFileCount, + PartitionStatistics::totalRecords, + PartitionStatistics::lastUpdatedAt, + PartitionStatistics::lastUpdatedSnapshotId) .containsExactlyInAnyOrder(expectedValues); } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index 00734dd1dc27..5c3093e70796 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -32,13 +32,9 @@ import java.util.stream.LongStream; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.EnvironmentContext; -import org.apache.iceberg.Files; import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.PartitionStatisticsFile; -import org.apache.iceberg.PartitionStats; +import org.apache.iceberg.PartitionStatistics; import org.apache.iceberg.PartitionStatsHandler; -import org.apache.iceberg.Partitioning; -import org.apache.iceberg.Schema; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -155,33 +151,36 @@ public void testPartitionStatsIncrementalCompute() throws IOException { insertData(10); Table table = validationCatalog.loadTable(tableIdent); - PartitionStatisticsFile statisticsFile = PartitionStatsHandler.computeAndWriteStatsFile(table); - table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); - - Schema dataSchema = PartitionStatsHandler.schema(Partitioning.partitionType(table), 2); - List statsBeforeCompaction; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, Files.localInput(statisticsFile.path()))) { + + table + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(table)) + .commit(); + + List statsBeforeCompaction; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().scan()) { statsBeforeCompaction = Lists.newArrayList(recordIterator); } sql("CALL %s.system.rewrite_data_files(table => '%s')", catalogName, tableIdent); table.refresh(); - statisticsFile = - PartitionStatsHandler.computeAndWriteStatsFile(table, table.currentSnapshot().snapshotId()); - table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); - List statsAfterCompaction; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, Files.localInput(statisticsFile.path()))) { + + table + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(table)) + .commit(); + + List statsAfterCompaction; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().scan()) { statsAfterCompaction = Lists.newArrayList(recordIterator); } for (int index = 0; index < statsBeforeCompaction.size(); index++) { - PartitionStats statsAfter = statsAfterCompaction.get(index); - PartitionStats statsBefore = statsBeforeCompaction.get(index); + PartitionStatistics statsAfter = statsAfterCompaction.get(index); + PartitionStatistics statsBefore = statsBeforeCompaction.get(index); assertThat(statsAfter.partition()).isEqualTo(statsBefore.partition()); // data count should match after compaction diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index fcebb8b04057..90fa85501070 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -26,13 +26,9 @@ import java.sql.Date; import java.sql.Timestamp; import java.util.List; -import org.apache.iceberg.Files; import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.PartitionStatisticsFile; -import org.apache.iceberg.PartitionStats; +import org.apache.iceberg.PartitionStatistics; import org.apache.iceberg.PartitionStatsHandler; -import org.apache.iceberg.Partitioning; -import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -408,14 +404,15 @@ public void testPartitionStatsIncrementalCompute() throws IOException { sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); Table table = validationCatalog.loadTable(tableIdent); - PartitionStatisticsFile statisticsFile = PartitionStatsHandler.computeAndWriteStatsFile(table); - table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); - - Schema dataSchema = PartitionStatsHandler.schema(Partitioning.partitionType(table), 2); - List statsBeforeRewrite; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, Files.localInput(statisticsFile.path()))) { + + table + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(table)) + .commit(); + + List statsBeforeRewrite; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().scan()) { statsBeforeRewrite = Lists.newArrayList(recordIterator); } @@ -424,19 +421,21 @@ public void testPartitionStatsIncrementalCompute() throws IOException { catalogName, tableIdent); table.refresh(); - statisticsFile = - PartitionStatsHandler.computeAndWriteStatsFile(table, table.currentSnapshot().snapshotId()); - table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); - List statsAfterRewrite; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, Files.localInput(statisticsFile.path()))) { + + table + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(table)) + .commit(); + + List statsAfterRewrite; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().scan()) { statsAfterRewrite = Lists.newArrayList(recordIterator); } for (int index = 0; index < statsBeforeRewrite.size(); index++) { - PartitionStats statsAfter = statsAfterRewrite.get(index); - PartitionStats statsBefore = statsBeforeRewrite.get(index); + PartitionStatistics statsAfter = statsAfterRewrite.get(index); + PartitionStatistics statsBefore = statsBeforeRewrite.get(index); assertThat(statsAfter.partition()).isEqualTo(statsBefore.partition()); // data count should match diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java index 303411eb7ddd..9ca6a9e57b8c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java @@ -23,12 +23,9 @@ import java.io.IOException; import java.util.List; -import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionStatistics; import org.apache.iceberg.PartitionStatisticsFile; -import org.apache.iceberg.PartitionStats; -import org.apache.iceberg.PartitionStatsHandler; import org.apache.iceberg.Partitioning; -import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; @@ -120,10 +117,9 @@ public void partitionStatsComputeOnLatestSnapshot() throws IOException { assertThat(table.partitionStatisticsFiles()).containsExactly(statisticsFile); Types.StructType partitionType = Partitioning.partitionType(table); - Schema dataSchema = PartitionStatsHandler.schema(partitionType, 2); validatePartitionStats( - statisticsFile, - dataSchema, + table, + statisticsFile.snapshotId(), Tuple.tuple( partitionRecord(partitionType, "foo", "A"), DEFAULT_SPEC_ID, @@ -209,11 +205,10 @@ public void partitionStatsComputeOnSnapshot() throws IOException { assertThat(table.partitionStatisticsFiles()).containsExactly(statisticsFile); Types.StructType partitionType = Partitioning.partitionType(table); - Schema dataSchema = PartitionStatsHandler.schema(partitionType, 2); // should contain stats for only partitions of snapshot1 (no entry for partition bar, A) validatePartitionStats( - statisticsFile, - dataSchema, + table, + statisticsFile.snapshotId(), Tuple.tuple( partitionRecord(partitionType, "foo", "A"), DEFAULT_SPEC_ID, @@ -272,31 +267,28 @@ private void createPartitionedTableV1() { tableName); } - private void validatePartitionStats( - PartitionStatisticsFile result, Schema recordSchema, Tuple... expectedValues) + private void validatePartitionStats(Table table, long snapshotId, Tuple... expectedValues) throws IOException { - // read the partition entries from the stats file - List partitionStats; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - recordSchema, Files.localInput(result.path()))) { + List partitionStats; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().useSnapshot(snapshotId).scan()) { partitionStats = Lists.newArrayList(recordIterator); } assertThat(partitionStats) .extracting( - PartitionStats::partition, - PartitionStats::specId, - PartitionStats::dataRecordCount, - PartitionStats::dataFileCount, - PartitionStats::totalDataFileSizeInBytes, - PartitionStats::positionDeleteRecordCount, - PartitionStats::positionDeleteFileCount, - PartitionStats::equalityDeleteRecordCount, - PartitionStats::equalityDeleteFileCount, - PartitionStats::totalRecords, - PartitionStats::lastUpdatedAt, - PartitionStats::lastUpdatedSnapshotId) + PartitionStatistics::partition, + PartitionStatistics::specId, + PartitionStatistics::dataRecordCount, + PartitionStatistics::dataFileCount, + PartitionStatistics::totalDataFileSizeInBytes, + PartitionStatistics::positionDeleteRecordCount, + PartitionStatistics::positionDeleteFileCount, + PartitionStatistics::equalityDeleteRecordCount, + PartitionStatistics::equalityDeleteFileCount, + PartitionStatistics::totalRecords, + PartitionStatistics::lastUpdatedAt, + PartitionStatistics::lastUpdatedSnapshotId) .containsExactlyInAnyOrder(expectedValues); } diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index 15bbfb02eb89..bd6d5abcd054 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -32,13 +32,9 @@ import java.util.stream.LongStream; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.EnvironmentContext; -import org.apache.iceberg.Files; import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.PartitionStatisticsFile; -import org.apache.iceberg.PartitionStats; +import org.apache.iceberg.PartitionStatistics; import org.apache.iceberg.PartitionStatsHandler; -import org.apache.iceberg.Partitioning; -import org.apache.iceberg.Schema; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -154,33 +150,36 @@ public void testPartitionStatsIncrementalCompute() throws IOException { insertData(10); Table table = validationCatalog.loadTable(tableIdent); - PartitionStatisticsFile statisticsFile = PartitionStatsHandler.computeAndWriteStatsFile(table); - table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); - - Schema dataSchema = PartitionStatsHandler.schema(Partitioning.partitionType(table), 2); - List statsBeforeCompaction; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, Files.localInput(statisticsFile.path()))) { + + table + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(table)) + .commit(); + + List statsBeforeCompaction; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().scan()) { statsBeforeCompaction = Lists.newArrayList(recordIterator); } sql("CALL %s.system.rewrite_data_files(table => '%s')", catalogName, tableIdent); table.refresh(); - statisticsFile = - PartitionStatsHandler.computeAndWriteStatsFile(table, table.currentSnapshot().snapshotId()); - table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); - List statsAfterCompaction; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, Files.localInput(statisticsFile.path()))) { + + table + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(table)) + .commit(); + + List statsAfterCompaction; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().scan()) { statsAfterCompaction = Lists.newArrayList(recordIterator); } for (int index = 0; index < statsBeforeCompaction.size(); index++) { - PartitionStats statsAfter = statsAfterCompaction.get(index); - PartitionStats statsBefore = statsBeforeCompaction.get(index); + PartitionStatistics statsAfter = statsAfterCompaction.get(index); + PartitionStatistics statsBefore = statsBeforeCompaction.get(index); assertThat(statsAfter.partition()).isEqualTo(statsBefore.partition()); // data count should match after compaction diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index b8dca4b2cd18..b9379fdbd587 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -26,13 +26,9 @@ import java.sql.Date; import java.sql.Timestamp; import java.util.List; -import org.apache.iceberg.Files; import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.PartitionStatisticsFile; -import org.apache.iceberg.PartitionStats; +import org.apache.iceberg.PartitionStatistics; import org.apache.iceberg.PartitionStatsHandler; -import org.apache.iceberg.Partitioning; -import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -408,14 +404,15 @@ public void testPartitionStatsIncrementalCompute() throws IOException { sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); Table table = validationCatalog.loadTable(tableIdent); - PartitionStatisticsFile statisticsFile = PartitionStatsHandler.computeAndWriteStatsFile(table); - table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); - - Schema dataSchema = PartitionStatsHandler.schema(Partitioning.partitionType(table), 2); - List statsBeforeRewrite; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, Files.localInput(statisticsFile.path()))) { + + table + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(table)) + .commit(); + + List statsBeforeRewrite; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().scan()) { statsBeforeRewrite = Lists.newArrayList(recordIterator); } @@ -424,19 +421,21 @@ public void testPartitionStatsIncrementalCompute() throws IOException { catalogName, tableIdent); table.refresh(); - statisticsFile = - PartitionStatsHandler.computeAndWriteStatsFile(table, table.currentSnapshot().snapshotId()); - table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); - List statsAfterRewrite; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, Files.localInput(statisticsFile.path()))) { + + table + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(table)) + .commit(); + + List statsAfterRewrite; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().scan()) { statsAfterRewrite = Lists.newArrayList(recordIterator); } for (int index = 0; index < statsBeforeRewrite.size(); index++) { - PartitionStats statsAfter = statsAfterRewrite.get(index); - PartitionStats statsBefore = statsBeforeRewrite.get(index); + PartitionStatistics statsAfter = statsAfterRewrite.get(index); + PartitionStatistics statsBefore = statsBeforeRewrite.get(index); assertThat(statsAfter.partition()).isEqualTo(statsBefore.partition()); // data count should match diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java index 303411eb7ddd..9ca6a9e57b8c 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java @@ -23,12 +23,9 @@ import java.io.IOException; import java.util.List; -import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionStatistics; import org.apache.iceberg.PartitionStatisticsFile; -import org.apache.iceberg.PartitionStats; -import org.apache.iceberg.PartitionStatsHandler; import org.apache.iceberg.Partitioning; -import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; @@ -120,10 +117,9 @@ public void partitionStatsComputeOnLatestSnapshot() throws IOException { assertThat(table.partitionStatisticsFiles()).containsExactly(statisticsFile); Types.StructType partitionType = Partitioning.partitionType(table); - Schema dataSchema = PartitionStatsHandler.schema(partitionType, 2); validatePartitionStats( - statisticsFile, - dataSchema, + table, + statisticsFile.snapshotId(), Tuple.tuple( partitionRecord(partitionType, "foo", "A"), DEFAULT_SPEC_ID, @@ -209,11 +205,10 @@ public void partitionStatsComputeOnSnapshot() throws IOException { assertThat(table.partitionStatisticsFiles()).containsExactly(statisticsFile); Types.StructType partitionType = Partitioning.partitionType(table); - Schema dataSchema = PartitionStatsHandler.schema(partitionType, 2); // should contain stats for only partitions of snapshot1 (no entry for partition bar, A) validatePartitionStats( - statisticsFile, - dataSchema, + table, + statisticsFile.snapshotId(), Tuple.tuple( partitionRecord(partitionType, "foo", "A"), DEFAULT_SPEC_ID, @@ -272,31 +267,28 @@ private void createPartitionedTableV1() { tableName); } - private void validatePartitionStats( - PartitionStatisticsFile result, Schema recordSchema, Tuple... expectedValues) + private void validatePartitionStats(Table table, long snapshotId, Tuple... expectedValues) throws IOException { - // read the partition entries from the stats file - List partitionStats; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - recordSchema, Files.localInput(result.path()))) { + List partitionStats; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().useSnapshot(snapshotId).scan()) { partitionStats = Lists.newArrayList(recordIterator); } assertThat(partitionStats) .extracting( - PartitionStats::partition, - PartitionStats::specId, - PartitionStats::dataRecordCount, - PartitionStats::dataFileCount, - PartitionStats::totalDataFileSizeInBytes, - PartitionStats::positionDeleteRecordCount, - PartitionStats::positionDeleteFileCount, - PartitionStats::equalityDeleteRecordCount, - PartitionStats::equalityDeleteFileCount, - PartitionStats::totalRecords, - PartitionStats::lastUpdatedAt, - PartitionStats::lastUpdatedSnapshotId) + PartitionStatistics::partition, + PartitionStatistics::specId, + PartitionStatistics::dataRecordCount, + PartitionStatistics::dataFileCount, + PartitionStatistics::totalDataFileSizeInBytes, + PartitionStatistics::positionDeleteRecordCount, + PartitionStatistics::positionDeleteFileCount, + PartitionStatistics::equalityDeleteRecordCount, + PartitionStatistics::equalityDeleteFileCount, + PartitionStatistics::totalRecords, + PartitionStatistics::lastUpdatedAt, + PartitionStatistics::lastUpdatedSnapshotId) .containsExactlyInAnyOrder(expectedValues); } diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java index 66c95621bf2d..3954b0a31455 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteDataFilesProcedure.java @@ -32,13 +32,9 @@ import java.util.stream.LongStream; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.EnvironmentContext; -import org.apache.iceberg.Files; import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.PartitionStatisticsFile; -import org.apache.iceberg.PartitionStats; +import org.apache.iceberg.PartitionStatistics; import org.apache.iceberg.PartitionStatsHandler; -import org.apache.iceberg.Partitioning; -import org.apache.iceberg.Schema; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -154,33 +150,36 @@ public void testPartitionStatsIncrementalCompute() throws IOException { insertData(10); Table table = validationCatalog.loadTable(tableIdent); - PartitionStatisticsFile statisticsFile = PartitionStatsHandler.computeAndWriteStatsFile(table); - table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); - - Schema dataSchema = PartitionStatsHandler.schema(Partitioning.partitionType(table), 2); - List statsBeforeCompaction; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, Files.localInput(statisticsFile.path()))) { + + table + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(table)) + .commit(); + + List statsBeforeCompaction; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().scan()) { statsBeforeCompaction = Lists.newArrayList(recordIterator); } sql("CALL %s.system.rewrite_data_files(table => '%s')", catalogName, tableIdent); table.refresh(); - statisticsFile = - PartitionStatsHandler.computeAndWriteStatsFile(table, table.currentSnapshot().snapshotId()); - table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); - List statsAfterCompaction; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, Files.localInput(statisticsFile.path()))) { + + table + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(table)) + .commit(); + + List statsAfterCompaction; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().scan()) { statsAfterCompaction = Lists.newArrayList(recordIterator); } for (int index = 0; index < statsBeforeCompaction.size(); index++) { - PartitionStats statsAfter = statsAfterCompaction.get(index); - PartitionStats statsBefore = statsBeforeCompaction.get(index); + PartitionStatistics statsAfter = statsAfterCompaction.get(index); + PartitionStatistics statsBefore = statsBeforeCompaction.get(index); assertThat(statsAfter.partition()).isEqualTo(statsBefore.partition()); // data count should match after compaction diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java index 98bcd401a51b..46bbe84e472d 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewriteManifestsProcedure.java @@ -26,13 +26,9 @@ import java.sql.Date; import java.sql.Timestamp; import java.util.List; -import org.apache.iceberg.Files; import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.PartitionStatisticsFile; -import org.apache.iceberg.PartitionStats; +import org.apache.iceberg.PartitionStatistics; import org.apache.iceberg.PartitionStatsHandler; -import org.apache.iceberg.Partitioning; -import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -408,14 +404,15 @@ public void testPartitionStatsIncrementalCompute() throws IOException { sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); Table table = validationCatalog.loadTable(tableIdent); - PartitionStatisticsFile statisticsFile = PartitionStatsHandler.computeAndWriteStatsFile(table); - table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); - - Schema dataSchema = PartitionStatsHandler.schema(Partitioning.partitionType(table), 2); - List statsBeforeRewrite; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, Files.localInput(statisticsFile.path()))) { + + table + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(table)) + .commit(); + + List statsBeforeRewrite; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().scan()) { statsBeforeRewrite = Lists.newArrayList(recordIterator); } @@ -424,19 +421,21 @@ public void testPartitionStatsIncrementalCompute() throws IOException { catalogName, tableIdent); table.refresh(); - statisticsFile = - PartitionStatsHandler.computeAndWriteStatsFile(table, table.currentSnapshot().snapshotId()); - table.updatePartitionStatistics().setPartitionStatistics(statisticsFile).commit(); - List statsAfterRewrite; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - dataSchema, Files.localInput(statisticsFile.path()))) { + + table + .updatePartitionStatistics() + .setPartitionStatistics(PartitionStatsHandler.computeAndWriteStatsFile(table)) + .commit(); + + List statsAfterRewrite; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().scan()) { statsAfterRewrite = Lists.newArrayList(recordIterator); } for (int index = 0; index < statsBeforeRewrite.size(); index++) { - PartitionStats statsAfter = statsAfterRewrite.get(index); - PartitionStats statsBefore = statsBeforeRewrite.get(index); + PartitionStatistics statsAfter = statsAfterRewrite.get(index); + PartitionStatistics statsBefore = statsBeforeRewrite.get(index); assertThat(statsAfter.partition()).isEqualTo(statsBefore.partition()); // data count should match diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java index 303411eb7ddd..9ca6a9e57b8c 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputePartitionStatsAction.java @@ -23,12 +23,9 @@ import java.io.IOException; import java.util.List; -import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionStatistics; import org.apache.iceberg.PartitionStatisticsFile; -import org.apache.iceberg.PartitionStats; -import org.apache.iceberg.PartitionStatsHandler; import org.apache.iceberg.Partitioning; -import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; @@ -120,10 +117,9 @@ public void partitionStatsComputeOnLatestSnapshot() throws IOException { assertThat(table.partitionStatisticsFiles()).containsExactly(statisticsFile); Types.StructType partitionType = Partitioning.partitionType(table); - Schema dataSchema = PartitionStatsHandler.schema(partitionType, 2); validatePartitionStats( - statisticsFile, - dataSchema, + table, + statisticsFile.snapshotId(), Tuple.tuple( partitionRecord(partitionType, "foo", "A"), DEFAULT_SPEC_ID, @@ -209,11 +205,10 @@ public void partitionStatsComputeOnSnapshot() throws IOException { assertThat(table.partitionStatisticsFiles()).containsExactly(statisticsFile); Types.StructType partitionType = Partitioning.partitionType(table); - Schema dataSchema = PartitionStatsHandler.schema(partitionType, 2); // should contain stats for only partitions of snapshot1 (no entry for partition bar, A) validatePartitionStats( - statisticsFile, - dataSchema, + table, + statisticsFile.snapshotId(), Tuple.tuple( partitionRecord(partitionType, "foo", "A"), DEFAULT_SPEC_ID, @@ -272,31 +267,28 @@ private void createPartitionedTableV1() { tableName); } - private void validatePartitionStats( - PartitionStatisticsFile result, Schema recordSchema, Tuple... expectedValues) + private void validatePartitionStats(Table table, long snapshotId, Tuple... expectedValues) throws IOException { - // read the partition entries from the stats file - List partitionStats; - try (CloseableIterable recordIterator = - PartitionStatsHandler.readPartitionStatsFile( - recordSchema, Files.localInput(result.path()))) { + List partitionStats; + try (CloseableIterable recordIterator = + table.newPartitionStatisticsScan().useSnapshot(snapshotId).scan()) { partitionStats = Lists.newArrayList(recordIterator); } assertThat(partitionStats) .extracting( - PartitionStats::partition, - PartitionStats::specId, - PartitionStats::dataRecordCount, - PartitionStats::dataFileCount, - PartitionStats::totalDataFileSizeInBytes, - PartitionStats::positionDeleteRecordCount, - PartitionStats::positionDeleteFileCount, - PartitionStats::equalityDeleteRecordCount, - PartitionStats::equalityDeleteFileCount, - PartitionStats::totalRecords, - PartitionStats::lastUpdatedAt, - PartitionStats::lastUpdatedSnapshotId) + PartitionStatistics::partition, + PartitionStatistics::specId, + PartitionStatistics::dataRecordCount, + PartitionStatistics::dataFileCount, + PartitionStatistics::totalDataFileSizeInBytes, + PartitionStatistics::positionDeleteRecordCount, + PartitionStatistics::positionDeleteFileCount, + PartitionStatistics::equalityDeleteRecordCount, + PartitionStatistics::equalityDeleteFileCount, + PartitionStatistics::totalRecords, + PartitionStatistics::lastUpdatedAt, + PartitionStatistics::lastUpdatedSnapshotId) .containsExactlyInAnyOrder(expectedValues); } From d5b83fc3a5035244ba279a1204e52efe118e7f46 Mon Sep 17 00:00:00 2001 From: Thomas Powell Date: Tue, 13 Jan 2026 16:15:20 +0000 Subject: [PATCH 192/201] Include key metadata in manifest tables (Spark 4.1) (#15041) * Include key metadata in manifest tables (Spark 4.1) * spotless --------- Co-authored-by: Thomas Powell --- .../iceberg/spark/actions/BaseSparkAction.java | 3 ++- .../iceberg/spark/actions/ManifestFileBean.java | 12 +++++++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index a8e82d101fbf..f53859ef97d6 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -161,7 +161,8 @@ protected Dataset contentFileDS(Table table, Set snapshotIds) { "length", "0 as sequenceNumber", "partition_spec_id as partitionSpecId", - "added_snapshot_id as addedSnapshotId") + "added_snapshot_id as addedSnapshotId", + "key_metadata as keyMetadata") .dropDuplicates("path") .repartition(numShufflePartitions) // avoid adaptive execution combining tasks .as(ManifestFileBean.ENCODER); diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java index fd4639897743..599e27b71c45 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/actions/ManifestFileBean.java @@ -37,6 +37,7 @@ public class ManifestFileBean implements ManifestFile, Serializable { private Integer content = null; private Long sequenceNumber = null; private Long firstRowId = null; + private byte[] keyMetadata = null; public static ManifestFileBean fromManifest(ManifestFile manifest) { ManifestFileBean bean = new ManifestFileBean(); @@ -48,6 +49,7 @@ public static ManifestFileBean fromManifest(ManifestFile manifest) { bean.setContent(manifest.content().id()); bean.setSequenceNumber(manifest.sequenceNumber()); bean.setFirstRowId(manifest.firstRowId()); + bean.setKeyMetadata(manifest.keyMetadata() == null ? null : manifest.keyMetadata().array()); return bean; } @@ -104,6 +106,14 @@ public void setFirstRowId(Long firstRowId) { this.firstRowId = firstRowId; } + public byte[] getKeyMetadata() { + return keyMetadata; + } + + public void setKeyMetadata(byte[] keyMetadata) { + this.keyMetadata = keyMetadata; + } + @Override public String path() { return path; @@ -176,7 +186,7 @@ public List partitions() { @Override public ByteBuffer keyMetadata() { - return null; + return keyMetadata == null ? null : ByteBuffer.wrap(keyMetadata); } @Override From 243badb3faa9290e66e710091dd9cd4e6c71a5a2 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Tue, 13 Jan 2026 12:00:46 -0500 Subject: [PATCH 193/201] site: Apache Iceberg Project News and Blog (#15013) --------- Co-authored-by: Danica Fine --- site/docs/blog/.authors.yml | 20 ++++ site/docs/blog/index.md | 18 ++++ .../blog/posts/2026-01-10-iceberg-summit.md | 92 +++++++++++++++++++ site/mkdocs-dev.yml | 1 + site/mkdocs.yml | 5 + site/nav.yml | 2 +- site/overrides/partials/footer.html | 1 + 7 files changed, 138 insertions(+), 1 deletion(-) create mode 100644 site/docs/blog/.authors.yml create mode 100644 site/docs/blog/index.md create mode 100644 site/docs/blog/posts/2026-01-10-iceberg-summit.md diff --git a/site/docs/blog/.authors.yml b/site/docs/blog/.authors.yml new file mode 100644 index 000000000000..fc3b66a3896c --- /dev/null +++ b/site/docs/blog/.authors.yml @@ -0,0 +1,20 @@ +# 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. + +authors: + iceberg-pmc: + name: Apache Iceberg PMC + description: Apache Iceberg PMC + avatar: assets/images/iceberg-logo-icon.png diff --git a/site/docs/blog/index.md b/site/docs/blog/index.md new file mode 100644 index 000000000000..24ce88f00062 --- /dev/null +++ b/site/docs/blog/index.md @@ -0,0 +1,18 @@ + + +# Project News and Blog diff --git a/site/docs/blog/posts/2026-01-10-iceberg-summit.md b/site/docs/blog/posts/2026-01-10-iceberg-summit.md new file mode 100644 index 000000000000..71d7e0c1e320 --- /dev/null +++ b/site/docs/blog/posts/2026-01-10-iceberg-summit.md @@ -0,0 +1,92 @@ +--- +date: 2026-01-10 +title: Announcing Iceberg Summit 2026 +authors: + - iceberg-pmc +categories: + - announcement +--- + + + +**Iceberg Summit is officially coming back in 2026!** + +Mark your calendars! On **April 8 and 9, 2026**, the Apache Iceberg community will gather once again in **San Francisco** at the Marriott Marquis for two full days dedicated to all things Iceberg. + + + +## Two Full Days of Apache Iceberg + +Following the tremendous success of the first in-person Iceberg Summit in 2025, we're excited to expand the event to **TWO full in-person days** for 2026. The community showed up in incredible ways last year, and we're ready to make this year even bigger and better. + +This third edition of Iceberg Summit is sanctioned by the Apache Software Foundation (ASF) with oversight from the Apache Iceberg™ PMC. The event is dedicated to sharing practical knowledge about using Apache Iceberg as the foundation of a modern data stack, bringing together practitioners, data engineers, and developers—from the highly experienced to the "Iceberg-curious." + +## Call for Papers is OPEN! + +If you've been working on something exciting in the Iceberg ecosystem, now's the perfect time to start crafting your submission for a chance to share your work with the entire community. Whether you're: + +- **Building on top of Iceberg** in production +- **Implementing new features** or exploring what's next +- **Compiling best practices** from real-world deployments +- **Working on ecosystem integrations** with other technologies +- **Sharing thought leadership** about the future of data architecture + +**We want to hear from YOU!** + +[Submit Now!](https://sessionize.com/iceberg-summit-2026/) The Call for Papers is still open! + +!!! info "Important Dates" + - **Call for Papers Closes**: January 18, 2026, 11:59 PM PT + - **Speaker Announcements**: February 6, 2026 + - **Event Dates**: April 8-9, 2026 + +### Session Formats + +We're accepting several types of submissions: + +- **Breakout Sessions** (30 minutes) - The standard conference talk format +- **Lightning Talks** (15 minutes) - Quick, digestible insights +- **Keynotes** (30 minutes) - Forward-thinking and thought leadership +- **Panels** (45 minutes) - Multi-perspective discussions with 4-6 panelists +- **Workshops/Labs** (60-180 minutes) - Hands-on, interactive sessions + +### Topics We're Looking For + +- **Iceberg in Production**: Real-world case studies, implementation journeys, lessons learned +- **Iceberg Technology and Features**: Deep dives into Iceberg, PyIceberg, Puffin, and upcoming enhancements +- **Data Architecture**: Stack options, integration patterns, and decision criteria +- **Best Practices**: Tips, tricks, and techniques for specific use cases +- **Thought Leadership**: Your vision for data architecture in the age of big data, cloud, and AI + +### Submit Your Talk + +What are you waiting for? Head over to [Sessionize](https://sessionize.com/iceberg-summit-2026/) to submit your proposal. + +**Need help crafting your submission?** Join us on the [Iceberg Community Slack](https://join.slack.com/t/apache-iceberg/shared_invite/zt-287g3akar-K9Oe_En5j1UL7Y_Ikpai3A) #abstracts channel for feedback and advice. + +- ~~Session 1: Wednesday, December 17, 2025 @ 8:00 AM PT~~ +- ~~Session 2: Wednesday, January 7, 2026 @ 8:00 AM PT~~ +- Session 3: Wednesday, January 14, 2026 @ 8:00 AM PT + +## Spread the Word + +Last year, we were blown away by the volume and quality of submissions from across the community. Let's make Iceberg Summit 2026 the best one yet! If you know someone working with Iceberg, encourage them to submit a talk and [register](https://www.icebergsummit.org/) to join us this April. + +Can't make it in person? Be sure to register for a _free_ streaming pass to catch the keynotes and one track of content live. + +Hope to see you there! diff --git a/site/mkdocs-dev.yml b/site/mkdocs-dev.yml index ce02dd99357a..acb8f3a7ce2b 100644 --- a/site/mkdocs-dev.yml +++ b/site/mkdocs-dev.yml @@ -90,6 +90,7 @@ nav: - Community: community.md - Talks: talks.md - Vendors: vendors.md + - Blog: blog/ - Specification: - Terms: terms.md - REST Catalog Spec: rest-catalog-spec.md diff --git a/site/mkdocs.yml b/site/mkdocs.yml index 5bbf64bf0013..6d71564de558 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -48,6 +48,11 @@ theme: plugins: - search + - blog: + blog_dir: blog + blog_toc: true + post_date_format: long + post_url_format: "{slug}" - macros - monorepo - privacy diff --git a/site/nav.yml b/site/nav.yml index 8b1ffaa25b90..47d81e8c71e7 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -106,7 +106,7 @@ nav: - Community: community.md - Talks: talks.md - Vendors: vendors.md - + - Blog: blog/ - Specification: - Terms: terms.md - REST Catalog Spec: rest-catalog-spec.md diff --git a/site/overrides/partials/footer.html b/site/overrides/partials/footer.html index 85a07bd4b29c..46c0323841f0 100644 --- a/site/overrides/partials/footer.html +++ b/site/overrides/partials/footer.html @@ -52,6 +52,7 @@

      Get Started

    • Hive Quickstart
    • Open Table Spec
    • Docs
    • +
    • Blog
    • Talks

    • From 779af12312fcf70c1e6e52d610d64cf947fd0a4f Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Tue, 13 Jan 2026 12:41:01 -0500 Subject: [PATCH 194/201] add registeristration link closer to the top (#15044) --- site/docs/blog/posts/2026-01-10-iceberg-summit.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/site/docs/blog/posts/2026-01-10-iceberg-summit.md b/site/docs/blog/posts/2026-01-10-iceberg-summit.md index 71d7e0c1e320..3ffde047dc33 100644 --- a/site/docs/blog/posts/2026-01-10-iceberg-summit.md +++ b/site/docs/blog/posts/2026-01-10-iceberg-summit.md @@ -28,6 +28,8 @@ categories: Mark your calendars! On **April 8 and 9, 2026**, the Apache Iceberg community will gather once again in **San Francisco** at the Marriott Marquis for two full days dedicated to all things Iceberg. +**[Register now](https://www.icebergsummit.org/#register)** to reserve your spot! + ## Two Full Days of Apache Iceberg From bd96b79b4229f27c0aa4a0ed776a1493499cfd79 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 14 Jan 2026 07:41:39 +0100 Subject: [PATCH 195/201] Core, Hive: Detect if a view already exists when registering a table (#15010) This was already working for all catalogs that support views, except for the `HiveCatalog`. All catalogs already had some logic that would perform table/view conflict detection when committing metadata in their `doCommit()` methods. However, the `HiveCatalog` currently does view/table detection one step earlier, namely when `doRefresh()` is called in https://github.com/apache/iceberg/blob/cbd35799dc70e989528b1c14d640fe91cdafa52d/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java#L175-L177. One alternative would be to override `registerTable()` for view catalgos and add early view detection (before metadata is actually committed). However, that would also require overriding table registration in the `JDBCCatalog` in order to detect whether view support is available or not. Hence it's probably best to only override table registration in the `HiveCatalog` --- .../apache/iceberg/view/ViewCatalogTests.java | 38 ++++++++++++ .../org/apache/iceberg/hive/HiveCatalog.java | 59 +++++++++++++------ 2 files changed, 79 insertions(+), 18 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index fdf4138f3ce5..5b51d5a3bf23 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -25,7 +25,10 @@ import java.nio.file.Path; import java.util.UUID; +import org.apache.iceberg.BaseTable; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; import org.apache.iceberg.Transaction; import org.apache.iceberg.UpdateLocation; import org.apache.iceberg.catalog.Catalog; @@ -1968,4 +1971,39 @@ public void dropNonEmptyNamespace() { .as("Namespace should not exist") .isFalse(); } + + @Test + public void registerTableThatAlreadyExistsAsView() { + TableIdentifier identifier = TableIdentifier.of("ns", "tbl"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + tableCatalog().createTable(identifier, SCHEMA); + assertThat(tableCatalog().tableExists(identifier)).as("Table should exist").isTrue(); + Table table = tableCatalog().loadTable(identifier); + TableOperations ops = ((BaseTable) table).operations(); + String metadataLocation = ops.current().metadataFileLocation(); + + // don't purge the metadata + tableCatalog().dropTable(identifier, false); + assertThat(tableCatalog().tableExists(identifier)).as("Table should not exist").isFalse(); + + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); + + assertThatThrownBy(() -> tableCatalog().registerTable(identifier, metadataLocation)) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageStartingWith("View with same name already exists: %s", identifier); + + assertThat(tableCatalog().tableExists(identifier)).as("Table should not exist").isFalse(); + assertThat(catalog().dropView(identifier)).isTrue(); + } } diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java index 5e7a249af84e..a6264c67fd88 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.TableType; -import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; @@ -48,6 +47,7 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.encryption.EncryptionUtil; import org.apache.iceberg.encryption.KeyManagementClient; +import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchIcebergViewException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; @@ -352,13 +352,11 @@ private void renameTableOrView( } if (tableExists(to)) { - throw new org.apache.iceberg.exceptions.AlreadyExistsException( - "Cannot rename %s to %s. Table already exists", from, to); + throw new AlreadyExistsException("Cannot rename %s to %s. Table already exists", from, to); } if (viewExists(to)) { - throw new org.apache.iceberg.exceptions.AlreadyExistsException( - "Cannot rename %s to %s. View already exists", from, to); + throw new AlreadyExistsException("Cannot rename %s to %s. View already exists", from, to); } String toDatabase = to.namespace().level(0); @@ -391,8 +389,7 @@ private void renameTableOrView( } catch (InvalidOperationException e) { if (e.getMessage() != null && e.getMessage().contains(String.format("new table %s already exists", to))) { - throw new org.apache.iceberg.exceptions.AlreadyExistsException( - "Table already exists: %s", to); + throw new AlreadyExistsException("Table already exists: %s", to); } else { throw new RuntimeException("Failed to rename " + from + " to " + to, e); } @@ -499,9 +496,8 @@ public void createNamespace(Namespace namespace, Map meta) { LOG.info("Created namespace: {}", namespace); - } catch (AlreadyExistsException e) { - throw new org.apache.iceberg.exceptions.AlreadyExistsException( - e, "Namespace already exists: %s", namespace); + } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) { + throw new AlreadyExistsException(e, "Namespace already exists: %s", namespace); } catch (TException e) { throw new RuntimeException( @@ -862,8 +858,7 @@ private ViewAwareTableBuilder(TableIdentifier identifier, Schema schema) { @Override public Transaction createOrReplaceTransaction() { if (viewExists(identifier)) { - throw new org.apache.iceberg.exceptions.AlreadyExistsException( - "View with same name already exists: %s", identifier); + throw new AlreadyExistsException("View with same name already exists: %s", identifier); } return super.createOrReplaceTransaction(); } @@ -871,8 +866,7 @@ public Transaction createOrReplaceTransaction() { @Override public org.apache.iceberg.Table create() { if (viewExists(identifier)) { - throw new org.apache.iceberg.exceptions.AlreadyExistsException( - "View with same name already exists: %s", identifier); + throw new AlreadyExistsException("View with same name already exists: %s", identifier); } return super.create(); } @@ -896,8 +890,7 @@ private TableAwareViewBuilder(TableIdentifier identifier) { @Override public View createOrReplace() { if (tableExists(identifier)) { - throw new org.apache.iceberg.exceptions.AlreadyExistsException( - "Table with same name already exists: %s", identifier); + throw new AlreadyExistsException("Table with same name already exists: %s", identifier); } return super.createOrReplace(); } @@ -905,10 +898,40 @@ public View createOrReplace() { @Override public View create() { if (tableExists(identifier)) { - throw new org.apache.iceberg.exceptions.AlreadyExistsException( - "Table with same name already exists: %s", identifier); + throw new AlreadyExistsException("Table with same name already exists: %s", identifier); } return super.create(); } } + + /** + * Register a table with the catalog if it does not exist. This is overridden in order to add view + * existence detection before registering a table. + * + * @param identifier a table identifier + * @param metadataFileLocation the location of a metadata file + * @return a Table instance + * @throws AlreadyExistsException if a table or view with the same identifier already exists in + * the catalog. + */ + @Override + public org.apache.iceberg.Table registerTable( + TableIdentifier identifier, String metadataFileLocation) { + Preconditions.checkArgument( + identifier != null && isValidIdentifier(identifier), "Invalid identifier: %s", identifier); + Preconditions.checkArgument( + metadataFileLocation != null && !metadataFileLocation.isEmpty(), + "Cannot register an empty metadata file location as a table"); + + // throw an exception in case the table identifier already exists as a table/view + if (tableExists(identifier)) { + throw new AlreadyExistsException("Table already exists: %s", identifier); + } + + if (viewExists(identifier)) { + throw new AlreadyExistsException("View with same name already exists: %s", identifier); + } + + return super.registerTable(identifier, metadataFileLocation); + } } From 046298f63bdfe3a6dc9d6cf6ed37223eda3ea6cc Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 14 Jan 2026 11:06:38 +0100 Subject: [PATCH 196/201] Bump to Parquet 1.17.0 (#14924) * Bump to Parquet 1.17.0 * Remove staging --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 8fb929e3673d..e9e74643d977 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -79,7 +79,7 @@ nessie = "0.106.1" netty-buffer = "4.2.9.Final" object-client-bundle = "3.3.2" orc = "1.9.8" -parquet = "1.16.0" +parquet = "1.17.0" roaringbitmap = "1.3.0" scala-collection-compat = "2.14.0" slf4j = "2.0.17" From b62802faf9edc2aca89a85a1ab0a6e13e1b61472 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 14 Jan 2026 18:08:56 +0100 Subject: [PATCH 197/201] Spark: Add test coverage for Hive View catalog (#15048) --- .../iceberg/spark/extensions/TestViews.java | 47 ++++++++++++------- .../iceberg/spark/SparkCatalogConfig.java | 6 ++- 2 files changed, 34 insertions(+), 19 deletions(-) diff --git a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java index 2406a909049e..237563860366 100644 --- a/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java +++ b/spark/v4.1/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java @@ -25,7 +25,6 @@ import java.nio.file.Paths; import java.util.List; -import java.util.Locale; import java.util.Random; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -108,7 +107,12 @@ public static Object[][] parameters() { .putAll(SparkCatalogConfig.SPARK_SESSION_WITH_VIEWS.properties()) .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) .build() - } + }, + { + SparkCatalogConfig.SPARK_WITH_HIVE_VIEWS.catalogName(), + SparkCatalogConfig.SPARK_WITH_HIVE_VIEWS.implementation(), + SparkCatalogConfig.SPARK_WITH_HIVE_VIEWS.properties() + }, }; } @@ -1350,6 +1354,10 @@ public void createViewWithSubqueryExpressionUsingTempFunction() { @TestTemplate public void createViewWithSubqueryExpressionInFilterThatIsRewritten() throws NoSuchTableException { + assumeThat(catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE)) + .as( + "Executing subquery expression with Hive fails due to an exception when instantiating the FileInputFormat") + .isNotEqualTo("hive"); insertRows(5); String viewName = viewName("viewWithSubqueryExpression"); String sql = @@ -1376,6 +1384,10 @@ public void createViewWithSubqueryExpressionInFilterThatIsRewritten() @TestTemplate public void createViewWithSubqueryExpressionInQueryThatIsRewritten() throws NoSuchTableException { + assumeThat(catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE)) + .as( + "Executing subquery expression with Hive fails due to an exception when instantiating the FileInputFormat") + .isNotEqualTo("hive"); insertRows(3); String viewName = viewName("viewWithSubqueryExpression"); String sql = @@ -1533,25 +1545,26 @@ public void showViews() throws NoSuchTableException { insertRows(6); String sql = String.format("SELECT * from %s", tableName); String v1 = viewName("v1"); - String prefixV2 = viewName("prefixV2"); - String prefixV3 = viewName("prefixV3"); - String globalViewForListing = viewName("globalViewForListing"); - String tempViewForListing = viewName("tempViewForListing"); + String prefixV2 = viewName("prefix_v2"); + String prefixV3 = viewName("prefix_v3"); + String globalViewForListing = viewName("global_view_for_listing"); + String tempViewForListing = viewName("temp_view_for_listing"); sql("CREATE VIEW %s AS %s", v1, sql); sql("CREATE VIEW %s AS %s", prefixV2, sql); sql("CREATE VIEW %s AS %s", prefixV3, sql); sql("CREATE GLOBAL TEMPORARY VIEW %s AS %s", globalViewForListing, sql); sql("CREATE TEMPORARY VIEW %s AS %s", tempViewForListing, sql); - // spark stores temp views case-insensitive by default - Object[] tempView = row("", tempViewForListing.toLowerCase(Locale.ROOT), true); + Object[] globalView = row("global_temp", globalViewForListing, true); + Object[] tempView = row("", tempViewForListing, true); Object[] v1Row = row(NAMESPACE.toString(), v1, false); Object[] v2Row = row(NAMESPACE.toString(), prefixV2, false); Object[] v3Row = row(NAMESPACE.toString(), prefixV3, false); assertThat(sql("SHOW VIEWS")).contains(v2Row, v3Row, v1Row, tempView); - if (!"rest".equals(catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE))) { - // REST catalog requires a namespace + String catalogType = catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE); + if (!"rest".equals(catalogType) && !"hive".equals(catalogType)) { + // REST & Hive catalog require a namespace assertThat(sql("SHOW VIEWS IN %s", catalogName)) .contains(tempView) .doesNotContain(v1Row, v2Row, v3Row); @@ -1566,7 +1579,7 @@ public void showViews() throws NoSuchTableException { assertThat(sql("SHOW VIEWS LIKE 'non-existing'")).isEmpty(); - if (!catalogName.equals(SPARK_CATALOG)) { + if (catalogName.equals(SparkCatalogConfig.SPARK_WITH_VIEWS.catalogName())) { sql("CREATE NAMESPACE IF NOT EXISTS spark_catalog.%s", NAMESPACE); assertThat(sql("SHOW VIEWS IN spark_catalog.%s", NAMESPACE)) .contains(tempView) @@ -1574,9 +1587,7 @@ public void showViews() throws NoSuchTableException { } assertThat(sql("SHOW VIEWS IN global_temp")) - .contains( - // spark stores temp views case-insensitive by default - row("global_temp", globalViewForListing.toLowerCase(Locale.ROOT), true), tempView) + .contains(globalView, tempView) .doesNotContain(v1Row, v2Row, v3Row); sql("USE spark_catalog"); @@ -1589,8 +1600,8 @@ public void showViews() throws NoSuchTableException { public void showViewsWithCurrentNamespace() { String namespaceOne = "show_views_ns1"; String namespaceTwo = "show_views_ns2"; - String viewOne = viewName("viewOne"); - String viewTwo = viewName("viewTwo"); + String viewOne = viewName("view_one"); + String viewTwo = viewName("view_two"); sql("CREATE NAMESPACE IF NOT EXISTS %s", namespaceOne); sql("CREATE NAMESPACE IF NOT EXISTS %s", namespaceTwo); @@ -1606,14 +1617,14 @@ public void showViewsWithCurrentNamespace() { .doesNotContain(v2); sql("USE %s", namespaceOne); assertThat(sql("SHOW VIEWS")).contains(v1).doesNotContain(v2); - assertThat(sql("SHOW VIEWS LIKE 'viewOne*'")).contains(v1).doesNotContain(v2); + assertThat(sql("SHOW VIEWS LIKE 'view_one*'")).contains(v1).doesNotContain(v2); assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, namespaceTwo)) .contains(v2) .doesNotContain(v1); sql("USE %s", namespaceTwo); assertThat(sql("SHOW VIEWS")).contains(v2).doesNotContain(v1); - assertThat(sql("SHOW VIEWS LIKE 'viewTwo*'")).contains(v2).doesNotContain(v1); + assertThat(sql("SHOW VIEWS LIKE 'view_two*'")).contains(v2).doesNotContain(v1); } @TestTemplate diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java index 1d96d6f93959..a9fbee2fc262 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java @@ -61,7 +61,11 @@ public enum SparkCatalogConfig { SPARK_SESSION_WITH_VIEWS( "spark_catalog", SparkSessionCatalog.class.getName(), - ImmutableMap.of("type", "rest", "default-namespace", "default", "cache-enabled", "false")); + ImmutableMap.of("type", "rest", "default-namespace", "default", "cache-enabled", "false")), + SPARK_WITH_HIVE_VIEWS( + "spark_hive_with_views", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "hive", "default-namespace", "default", "cache-enabled", "false")); private final String catalogName; private final String implementation; From c1aed477d1c0c8ef93aa8dea23f75e5ab11f5be6 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 14 Jan 2026 22:14:44 +0100 Subject: [PATCH 198/201] Spark 3.5,4.0: Add test coverage for Hive View catalog (#15052) --- .../iceberg/spark/extensions/TestViews.java | 102 ++++++++---------- .../iceberg/spark/SparkCatalogConfig.java | 17 ++- .../iceberg/spark/extensions/TestViews.java | 47 ++++---- .../iceberg/spark/SparkCatalogConfig.java | 17 ++- 4 files changed, 100 insertions(+), 83 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java index 6144b249305c..153f580e5ba6 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java @@ -25,7 +25,6 @@ import java.nio.file.Paths; import java.util.List; -import java.util.Locale; import java.util.Random; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -108,7 +107,12 @@ public static Object[][] parameters() { .putAll(SparkCatalogConfig.SPARK_SESSION_WITH_VIEWS.properties()) .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) .build() - } + }, + { + SparkCatalogConfig.SPARK_WITH_HIVE_VIEWS.catalogName(), + SparkCatalogConfig.SPARK_WITH_HIVE_VIEWS.implementation(), + SparkCatalogConfig.SPARK_WITH_HIVE_VIEWS.properties() + }, }; } @@ -215,9 +219,7 @@ public void readFromViewUsingNonExistingTable() throws NoSuchTableException { assertThatThrownBy(() -> sql("SELECT * FROM %s", viewName)) .isInstanceOf(AnalysisException.class) .hasMessageContaining( - String.format( - "The table or view `%s`.`%s`.`non_existing` cannot be found", - catalogName, NAMESPACE)); + "The table or view `%s`.`%s`.`non_existing` cannot be found", catalogName, NAMESPACE); } @TestTemplate @@ -876,8 +878,7 @@ public void renameViewTargetAlreadyExistsAsView() { assertThatThrownBy(() -> sql("ALTER VIEW %s RENAME TO %s", viewName, target)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - String.format("Cannot create view default.%s because it already exists", target)); + .hasMessageContaining("Cannot create view default.%s because it already exists", target); } @TestTemplate @@ -901,8 +902,7 @@ public void renameViewTargetAlreadyExistsAsTable() { catalogName, NAMESPACE, target, catalogName.equals(SPARK_CATALOG) ? " USING iceberg" : ""); assertThatThrownBy(() -> sql("ALTER VIEW %s RENAME TO %s", viewName, target)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - String.format("Cannot create view default.%s because it already exists", target)); + .hasMessageContaining("Cannot create view default.%s because it already exists", target); } @TestTemplate @@ -996,8 +996,7 @@ public void createViewIfNotExists() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS SELECT id FROM %s", viewName, tableName)) .isInstanceOf(AnalysisException.class) .hasMessageContaining( - String.format( - "Cannot create view %s.%s because it already exists", NAMESPACE, viewName)); + "Cannot create view %s.%s because it already exists", NAMESPACE, viewName); // using IF NOT EXISTS should work assertThatNoException() @@ -1041,8 +1040,7 @@ public void createViewReferencingTempView() throws NoSuchTableException { () -> sql("CREATE VIEW %s AS SELECT id FROM %s", viewReferencingTempView, tempView)) .isInstanceOf(AnalysisException.class) .hasMessageContaining( - String.format( - "Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewReferencingTempView)) + "Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewReferencingTempView) .hasMessageContaining("that references temporary view:") .hasMessageContaining(tempView); } @@ -1065,8 +1063,7 @@ public void createViewReferencingGlobalTempView() throws NoSuchTableException { viewReferencingTempView, globalTempView)) .isInstanceOf(AnalysisException.class) .hasMessageContaining( - String.format( - "Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewReferencingTempView)) + "Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewReferencingTempView) .hasMessageContaining("that references temporary view:") .hasMessageContaining("%s.%s", "global_temp", globalTempView); } @@ -1084,8 +1081,7 @@ public void createViewReferencingTempFunction() { assertThatThrownBy( () -> sql("CREATE VIEW %s AS SELECT %s(id) FROM %s", viewName, functionName, tableName)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) .hasMessageContaining("that references temporary function:") .hasMessageContaining(functionName); } @@ -1136,8 +1132,7 @@ public void createViewWithMismatchedColumnCounts() { assertThatThrownBy( () -> sql("CREATE VIEW %s (id, data) AS SELECT id FROM %s", viewName, tableName)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) .hasMessageContaining("not enough data columns") .hasMessageContaining("View columns: id, data") .hasMessageContaining("Data columns: id"); @@ -1145,8 +1140,7 @@ public void createViewWithMismatchedColumnCounts() { assertThatThrownBy( () -> sql("CREATE VIEW %s (id) AS SELECT id, data FROM %s", viewName, tableName)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) .hasMessageContaining("too many data columns") .hasMessageContaining("View columns: id") .hasMessageContaining("Data columns: id, data"); @@ -1265,8 +1259,7 @@ public void createViewWithCTEReferencingTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) .hasMessageContaining("that references temporary view:") .hasMessageContaining(tempViewInCTE); } @@ -1287,8 +1280,7 @@ public void createViewWithCTEReferencingTempFunction() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) .hasMessageContaining("that references temporary function:") .hasMessageContaining(functionName); } @@ -1316,8 +1308,7 @@ public void createViewWithSubqueryExpressionUsingTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) .hasMessageContaining("that references temporary view:") .hasMessageContaining(tempView); } @@ -1337,8 +1328,7 @@ public void createViewWithSubqueryExpressionUsingGlobalTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) .hasMessageContaining("that references temporary view:") .hasMessageContaining("%s.%s", "global_temp", globalTempView); } @@ -1358,8 +1348,7 @@ public void createViewWithSubqueryExpressionUsingTempFunction() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining( - String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) + .hasMessageContaining("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName) .hasMessageContaining("that references temporary function:") .hasMessageContaining(functionName); } @@ -1367,6 +1356,10 @@ public void createViewWithSubqueryExpressionUsingTempFunction() { @TestTemplate public void createViewWithSubqueryExpressionInFilterThatIsRewritten() throws NoSuchTableException { + assumeThat(catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE)) + .as( + "Executing subquery expression with Hive fails due to an exception when instantiating the FileInputFormat") + .isNotEqualTo("hive"); insertRows(5); String viewName = viewName("viewWithSubqueryExpression"); String sql = @@ -1393,6 +1386,10 @@ public void createViewWithSubqueryExpressionInFilterThatIsRewritten() @TestTemplate public void createViewWithSubqueryExpressionInQueryThatIsRewritten() throws NoSuchTableException { + assumeThat(catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE)) + .as( + "Executing subquery expression with Hive fails due to an exception when instantiating the FileInputFormat") + .isNotEqualTo("hive"); insertRows(3); String viewName = viewName("viewWithSubqueryExpression"); String sql = @@ -1550,25 +1547,26 @@ public void showViews() throws NoSuchTableException { insertRows(6); String sql = String.format("SELECT * from %s", tableName); String v1 = viewName("v1"); - String prefixV2 = viewName("prefixV2"); - String prefixV3 = viewName("prefixV3"); - String globalViewForListing = viewName("globalViewForListing"); - String tempViewForListing = viewName("tempViewForListing"); + String prefixV2 = viewName("prefix_v2"); + String prefixV3 = viewName("prefix_v3"); + String globalViewForListing = viewName("global_view_for_listing"); + String tempViewForListing = viewName("temp_view_for_listing"); sql("CREATE VIEW %s AS %s", v1, sql); sql("CREATE VIEW %s AS %s", prefixV2, sql); sql("CREATE VIEW %s AS %s", prefixV3, sql); sql("CREATE GLOBAL TEMPORARY VIEW %s AS %s", globalViewForListing, sql); sql("CREATE TEMPORARY VIEW %s AS %s", tempViewForListing, sql); - // spark stores temp views case-insensitive by default - Object[] tempView = row("", tempViewForListing.toLowerCase(Locale.ROOT), true); + Object[] globalView = row("global_temp", globalViewForListing, true); + Object[] tempView = row("", tempViewForListing, true); Object[] v1Row = row(NAMESPACE.toString(), v1, false); Object[] v2Row = row(NAMESPACE.toString(), prefixV2, false); Object[] v3Row = row(NAMESPACE.toString(), prefixV3, false); assertThat(sql("SHOW VIEWS")).contains(v2Row, v3Row, v1Row, tempView); - if (!"rest".equals(catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE))) { - // REST catalog requires a namespace + String catalogType = catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE); + if (!"rest".equals(catalogType) && !"hive".equals(catalogType)) { + // REST & Hive catalog require a namespace assertThat(sql("SHOW VIEWS IN %s", catalogName)) .contains(tempView) .doesNotContain(v1Row, v2Row, v3Row); @@ -1583,7 +1581,7 @@ public void showViews() throws NoSuchTableException { assertThat(sql("SHOW VIEWS LIKE 'non-existing'")).isEmpty(); - if (!catalogName.equals(SPARK_CATALOG)) { + if (catalogName.equals(SparkCatalogConfig.SPARK_WITH_VIEWS.catalogName())) { sql("CREATE NAMESPACE IF NOT EXISTS spark_catalog.%s", NAMESPACE); assertThat(sql("SHOW VIEWS IN spark_catalog.%s", NAMESPACE)) .contains(tempView) @@ -1591,9 +1589,7 @@ public void showViews() throws NoSuchTableException { } assertThat(sql("SHOW VIEWS IN global_temp")) - .contains( - // spark stores temp views case-insensitive by default - row("global_temp", globalViewForListing.toLowerCase(Locale.ROOT), true), tempView) + .contains(globalView, tempView) .doesNotContain(v1Row, v2Row, v3Row); sql("USE spark_catalog"); @@ -1606,8 +1602,8 @@ public void showViews() throws NoSuchTableException { public void showViewsWithCurrentNamespace() { String namespaceOne = "show_views_ns1"; String namespaceTwo = "show_views_ns2"; - String viewOne = viewName("viewOne"); - String viewTwo = viewName("viewTwo"); + String viewOne = viewName("view_one"); + String viewTwo = viewName("view_two"); sql("CREATE NAMESPACE IF NOT EXISTS %s", namespaceOne); sql("CREATE NAMESPACE IF NOT EXISTS %s", namespaceTwo); @@ -1623,14 +1619,14 @@ public void showViewsWithCurrentNamespace() { .doesNotContain(v2); sql("USE %s", namespaceOne); assertThat(sql("SHOW VIEWS")).contains(v1).doesNotContain(v2); - assertThat(sql("SHOW VIEWS LIKE 'viewOne*'")).contains(v1).doesNotContain(v2); + assertThat(sql("SHOW VIEWS LIKE 'view_one*'")).contains(v1).doesNotContain(v2); assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, namespaceTwo)) .contains(v2) .doesNotContain(v1); sql("USE %s", namespaceTwo); assertThat(sql("SHOW VIEWS")).contains(v2).doesNotContain(v1); - assertThat(sql("SHOW VIEWS LIKE 'viewTwo*'")).contains(v2).doesNotContain(v1); + assertThat(sql("SHOW VIEWS LIKE 'view_two*'")).contains(v2).doesNotContain(v1); } @TestTemplate @@ -2021,8 +2017,7 @@ public void createViewWithRecursiveCycle() { String cycle = String.format("%s -> %s -> %s", view1, view2, view1); assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS SELECT * FROM %s", viewOne, view2)) .isInstanceOf(AnalysisException.class) - .hasMessageStartingWith( - String.format("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle)); + .hasMessageStartingWith("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle); } @TestTemplate @@ -2043,8 +2038,7 @@ public void createViewWithRecursiveCycleToV1View() { String cycle = String.format("%s -> %s -> %s", view1, view2, view1); assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS SELECT * FROM %s", viewOne, view2)) .isInstanceOf(AnalysisException.class) - .hasMessageStartingWith( - String.format("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle)); + .hasMessageStartingWith("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle); } @TestTemplate @@ -2068,8 +2062,7 @@ public void createViewWithRecursiveCycleInCTE() { String cycle = String.format("%s -> %s -> %s", view1, viewTwo, view1); assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewOne, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageStartingWith( - String.format("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle)); + .hasMessageStartingWith("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle); } @TestTemplate @@ -2090,8 +2083,7 @@ public void createViewWithRecursiveCycleInSubqueryExpression() { String cycle = String.format("%s -> %s -> %s", view1, viewTwo, view1); assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewOne, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageStartingWith( - String.format("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle)); + .hasMessageStartingWith("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle); } @TestTemplate diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java index 1d96d6f93959..2350aab09b64 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java @@ -42,11 +42,14 @@ public enum SparkCatalogConfig { "spark_catalog", SparkSessionCatalog.class.getName(), ImmutableMap.of( - "type", "hive", - "default-namespace", "default", - "parquet-enabled", "true", + "type", + "hive", + "default-namespace", + "default", + "parquet-enabled", + "true", "cache-enabled", - "false" // Spark will delete tables using v1, leaving the cache out of sync + "false" // Spark will delete tables using v1, leaving the cache out of sync )), SPARK_WITH_VIEWS( "spark_with_views", @@ -61,7 +64,11 @@ public enum SparkCatalogConfig { SPARK_SESSION_WITH_VIEWS( "spark_catalog", SparkSessionCatalog.class.getName(), - ImmutableMap.of("type", "rest", "default-namespace", "default", "cache-enabled", "false")); + ImmutableMap.of("type", "rest", "default-namespace", "default", "cache-enabled", "false")), + SPARK_WITH_HIVE_VIEWS( + "spark_hive_with_views", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "hive", "default-namespace", "default", "cache-enabled", "false")); private final String catalogName; private final String implementation; diff --git a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java index 2406a909049e..237563860366 100644 --- a/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java +++ b/spark/v4.0/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestViews.java @@ -25,7 +25,6 @@ import java.nio.file.Paths; import java.util.List; -import java.util.Locale; import java.util.Random; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -108,7 +107,12 @@ public static Object[][] parameters() { .putAll(SparkCatalogConfig.SPARK_SESSION_WITH_VIEWS.properties()) .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) .build() - } + }, + { + SparkCatalogConfig.SPARK_WITH_HIVE_VIEWS.catalogName(), + SparkCatalogConfig.SPARK_WITH_HIVE_VIEWS.implementation(), + SparkCatalogConfig.SPARK_WITH_HIVE_VIEWS.properties() + }, }; } @@ -1350,6 +1354,10 @@ public void createViewWithSubqueryExpressionUsingTempFunction() { @TestTemplate public void createViewWithSubqueryExpressionInFilterThatIsRewritten() throws NoSuchTableException { + assumeThat(catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE)) + .as( + "Executing subquery expression with Hive fails due to an exception when instantiating the FileInputFormat") + .isNotEqualTo("hive"); insertRows(5); String viewName = viewName("viewWithSubqueryExpression"); String sql = @@ -1376,6 +1384,10 @@ public void createViewWithSubqueryExpressionInFilterThatIsRewritten() @TestTemplate public void createViewWithSubqueryExpressionInQueryThatIsRewritten() throws NoSuchTableException { + assumeThat(catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE)) + .as( + "Executing subquery expression with Hive fails due to an exception when instantiating the FileInputFormat") + .isNotEqualTo("hive"); insertRows(3); String viewName = viewName("viewWithSubqueryExpression"); String sql = @@ -1533,25 +1545,26 @@ public void showViews() throws NoSuchTableException { insertRows(6); String sql = String.format("SELECT * from %s", tableName); String v1 = viewName("v1"); - String prefixV2 = viewName("prefixV2"); - String prefixV3 = viewName("prefixV3"); - String globalViewForListing = viewName("globalViewForListing"); - String tempViewForListing = viewName("tempViewForListing"); + String prefixV2 = viewName("prefix_v2"); + String prefixV3 = viewName("prefix_v3"); + String globalViewForListing = viewName("global_view_for_listing"); + String tempViewForListing = viewName("temp_view_for_listing"); sql("CREATE VIEW %s AS %s", v1, sql); sql("CREATE VIEW %s AS %s", prefixV2, sql); sql("CREATE VIEW %s AS %s", prefixV3, sql); sql("CREATE GLOBAL TEMPORARY VIEW %s AS %s", globalViewForListing, sql); sql("CREATE TEMPORARY VIEW %s AS %s", tempViewForListing, sql); - // spark stores temp views case-insensitive by default - Object[] tempView = row("", tempViewForListing.toLowerCase(Locale.ROOT), true); + Object[] globalView = row("global_temp", globalViewForListing, true); + Object[] tempView = row("", tempViewForListing, true); Object[] v1Row = row(NAMESPACE.toString(), v1, false); Object[] v2Row = row(NAMESPACE.toString(), prefixV2, false); Object[] v3Row = row(NAMESPACE.toString(), prefixV3, false); assertThat(sql("SHOW VIEWS")).contains(v2Row, v3Row, v1Row, tempView); - if (!"rest".equals(catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE))) { - // REST catalog requires a namespace + String catalogType = catalogConfig.get(CatalogUtil.ICEBERG_CATALOG_TYPE); + if (!"rest".equals(catalogType) && !"hive".equals(catalogType)) { + // REST & Hive catalog require a namespace assertThat(sql("SHOW VIEWS IN %s", catalogName)) .contains(tempView) .doesNotContain(v1Row, v2Row, v3Row); @@ -1566,7 +1579,7 @@ public void showViews() throws NoSuchTableException { assertThat(sql("SHOW VIEWS LIKE 'non-existing'")).isEmpty(); - if (!catalogName.equals(SPARK_CATALOG)) { + if (catalogName.equals(SparkCatalogConfig.SPARK_WITH_VIEWS.catalogName())) { sql("CREATE NAMESPACE IF NOT EXISTS spark_catalog.%s", NAMESPACE); assertThat(sql("SHOW VIEWS IN spark_catalog.%s", NAMESPACE)) .contains(tempView) @@ -1574,9 +1587,7 @@ public void showViews() throws NoSuchTableException { } assertThat(sql("SHOW VIEWS IN global_temp")) - .contains( - // spark stores temp views case-insensitive by default - row("global_temp", globalViewForListing.toLowerCase(Locale.ROOT), true), tempView) + .contains(globalView, tempView) .doesNotContain(v1Row, v2Row, v3Row); sql("USE spark_catalog"); @@ -1589,8 +1600,8 @@ public void showViews() throws NoSuchTableException { public void showViewsWithCurrentNamespace() { String namespaceOne = "show_views_ns1"; String namespaceTwo = "show_views_ns2"; - String viewOne = viewName("viewOne"); - String viewTwo = viewName("viewTwo"); + String viewOne = viewName("view_one"); + String viewTwo = viewName("view_two"); sql("CREATE NAMESPACE IF NOT EXISTS %s", namespaceOne); sql("CREATE NAMESPACE IF NOT EXISTS %s", namespaceTwo); @@ -1606,14 +1617,14 @@ public void showViewsWithCurrentNamespace() { .doesNotContain(v2); sql("USE %s", namespaceOne); assertThat(sql("SHOW VIEWS")).contains(v1).doesNotContain(v2); - assertThat(sql("SHOW VIEWS LIKE 'viewOne*'")).contains(v1).doesNotContain(v2); + assertThat(sql("SHOW VIEWS LIKE 'view_one*'")).contains(v1).doesNotContain(v2); assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, namespaceTwo)) .contains(v2) .doesNotContain(v1); sql("USE %s", namespaceTwo); assertThat(sql("SHOW VIEWS")).contains(v2).doesNotContain(v1); - assertThat(sql("SHOW VIEWS LIKE 'viewTwo*'")).contains(v2).doesNotContain(v1); + assertThat(sql("SHOW VIEWS LIKE 'view_two*'")).contains(v2).doesNotContain(v1); } @TestTemplate diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java index 1d96d6f93959..2350aab09b64 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogConfig.java @@ -42,11 +42,14 @@ public enum SparkCatalogConfig { "spark_catalog", SparkSessionCatalog.class.getName(), ImmutableMap.of( - "type", "hive", - "default-namespace", "default", - "parquet-enabled", "true", + "type", + "hive", + "default-namespace", + "default", + "parquet-enabled", + "true", "cache-enabled", - "false" // Spark will delete tables using v1, leaving the cache out of sync + "false" // Spark will delete tables using v1, leaving the cache out of sync )), SPARK_WITH_VIEWS( "spark_with_views", @@ -61,7 +64,11 @@ public enum SparkCatalogConfig { SPARK_SESSION_WITH_VIEWS( "spark_catalog", SparkSessionCatalog.class.getName(), - ImmutableMap.of("type", "rest", "default-namespace", "default", "cache-enabled", "false")); + ImmutableMap.of("type", "rest", "default-namespace", "default", "cache-enabled", "false")), + SPARK_WITH_HIVE_VIEWS( + "spark_hive_with_views", + SparkCatalog.class.getName(), + ImmutableMap.of("type", "hive", "default-namespace", "default", "cache-enabled", "false")); private final String catalogName; private final String implementation; From 035e0fb39d2a949f6343552ade0a7d6c2967e0db Mon Sep 17 00:00:00 2001 From: Daniel Weeks Date: Thu, 15 Jan 2026 11:44:25 -0800 Subject: [PATCH 199/201] REST Spec: clarify uniqueness of ETags for table metadata responses (#15045) * REST Spec: clarify uniqueness of ETags --- open-api/rest-catalog-open-api.yaml | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index d322b0c7c7c0..e9a40ed129b1 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -1940,7 +1940,13 @@ components: etag: name: ETag in: header - description: Identifies a unique version of the table metadata. + description: + Identifies a unique version of the table metadata. + + Implementations that support ETags should produce unique tags for responses that return + different metadata content but represent the same version of table metadata. For example, + the `snapshots` query parameter may result in different metadata representations depending + on whether `refs` or `all` is provided, therefore should have distinct ETags. required: false schema: type: string From 38cc88136684a57b61be4ae0d2c1886eff742a28 Mon Sep 17 00:00:00 2001 From: Joy Haldar Date: Fri, 16 Jan 2026 04:22:53 +0530 Subject: [PATCH 200/201] BigQuery: Eliminate redundant table load by using ETag for conflict detection (#14940) * BigQuery: Reuse table from doRefresh() in updateTable() to reduce API calls --- .../gcp/bigquery/BigQueryTableOperations.java | 43 ++++++++----------- .../bigquery/FakeBigQueryMetastoreClient.java | 13 ++---- .../bigquery/TestBigQueryTableOperations.java | 21 +++++---- 3 files changed, 32 insertions(+), 45 deletions(-) diff --git a/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryTableOperations.java b/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryTableOperations.java index e5f0a449574c..37728aa15794 100644 --- a/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryTableOperations.java +++ b/bigquery/src/main/java/org/apache/iceberg/gcp/bigquery/BigQueryTableOperations.java @@ -33,6 +33,7 @@ import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,6 +49,9 @@ final class BigQueryTableOperations extends BaseMetastoreTableOperations { private final FileIO fileIO; private final TableReference tableReference; + /** Table loaded in doRefresh() for reuse in updateTable() to avoid redundant API call. */ + private volatile Table metastoreTable; + BigQueryTableOperations( BigQueryMetastoreClient client, FileIO fileIO, TableReference tableReference) { this.client = client; @@ -60,9 +64,11 @@ final class BigQueryTableOperations extends BaseMetastoreTableOperations { public void doRefresh() { // Must default to null. String metadataLocation = null; + this.metastoreTable = null; try { + this.metastoreTable = client.load(tableReference); metadataLocation = - loadMetadataLocationOrThrow(client.load(tableReference).getExternalCatalogTableOptions()); + loadMetadataLocationOrThrow(metastoreTable.getExternalCatalogTableOptions()); } catch (NoSuchTableException e) { if (currentMetadataLocation() != null) { // Re-throws the exception because the table must exist in this case. @@ -86,7 +92,7 @@ public void doCommit(TableMetadata base, TableMetadata metadata) { if (base == null) { createTable(newMetadataLocation, metadata); } else { - updateTable(base.metadataFileLocation(), newMetadataLocation, metadata); + updateTable(newMetadataLocation, metadata); } commitStatus = BaseMetastoreOperations.CommitStatus.SUCCESS; } catch (CommitFailedException | CommitStateUnknownException e) { @@ -149,35 +155,24 @@ private void addConnectionIfProvided(Table tableBuilder, Map met } /** Update table properties with concurrent update detection using etag. */ - private void updateTable( - String oldMetadataLocation, String newMetadataLocation, TableMetadata metadata) { - Table table = client.load(tableReference); - if (table.getEtag().isEmpty()) { + private void updateTable(String newMetadataLocation, TableMetadata metadata) { + Preconditions.checkState( + metastoreTable != null, + "Table %s must be loaded during refresh before commit", + tableName()); + + if (metastoreTable.getEtag().isEmpty()) { throw new ValidationException( "Etag of legacy table %s is empty, manually update the table via the BigQuery API or" + " recreate and retry", tableName()); } - ExternalCatalogTableOptions options = table.getExternalCatalogTableOptions(); - addConnectionIfProvided(table, metadata.properties()); - - // If `metadataLocationFromMetastore` is different from metadata location of base, it means - // someone has updated metadata location in metastore, which is a conflict update. - String metadataLocationFromMetastore = - options.getParameters().getOrDefault(METADATA_LOCATION_PROP, ""); - if (!metadataLocationFromMetastore.isEmpty() - && !metadataLocationFromMetastore.equals(oldMetadataLocation)) { - throw new CommitFailedException( - "Cannot commit base metadata location '%s' is not same as the current table metadata location '%s' for" - + " %s.%s", - oldMetadataLocation, - metadataLocationFromMetastore, - tableReference.getDatasetId(), - tableReference.getTableId()); - } + ExternalCatalogTableOptions options = metastoreTable.getExternalCatalogTableOptions(); + addConnectionIfProvided(metastoreTable, metadata.properties()); options.setParameters(buildTableParameters(newMetadataLocation, metadata)); - client.update(tableReference, table); + client.update(tableReference, metastoreTable); + this.metastoreTable = null; } // To make the table queryable from Hive, the user would likely be setting the HIVE_ENGINE_ENABLED diff --git a/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/FakeBigQueryMetastoreClient.java b/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/FakeBigQueryMetastoreClient.java index 0c6df15091a6..3619f7908c38 100644 --- a/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/FakeBigQueryMetastoreClient.java +++ b/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/FakeBigQueryMetastoreClient.java @@ -180,18 +180,11 @@ public Table update(TableReference tableReference, Table table) { String incomingEtag = table.getEtag(); String requiredEtag = existingTable.getEtag(); - // The real patch() uses an If-Match header which is passed separately, - // NOT on the incoming table object. - // The BigQueryTableOperations does NOT set the ETag on the Table object - // it passes to the client update() method. - // For a fake, we assume the ETag check needs to be simulated based on - // state, BUT the real client.update() expects the ETAG as a separate parameter - // (or implicitly via setIfMatch header, which this Fake doesn't see). - // To make the fake usable, we'll assume that if an ETag *is* present - // on the incoming table object, it must match. + // Simulate ETag-based optimistic locking. If the incoming table has an ETag, + // it must match the current ETag in the store. if (incomingEtag != null && !incomingEtag.equals(requiredEtag)) { throw new CommitFailedException( - "Etag mismatch for table: %s. Required: %s, Found: %s", + "Cannot commit: Etag mismatch for table: %s. Required: %s, Found: %s", tableReference, requiredEtag, incomingEtag); } diff --git a/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryTableOperations.java b/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryTableOperations.java index 4666ec61f4d3..c5095aedb000 100644 --- a/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryTableOperations.java +++ b/bigquery/src/test/java/org/apache/iceberg/gcp/bigquery/TestBigQueryTableOperations.java @@ -185,26 +185,25 @@ public void failWhenEtagMismatch() throws Exception { } @Test - public void failWhenMetadataLocationDiff() throws Exception { + public void failWhenConcurrentModificationDetected() throws Exception { Table tableWithEtag = createTestTable().setEtag("etag"); - Table tableWithNewMetadata = - new Table() - .setEtag("etag") - .setExternalCatalogTableOptions( - new ExternalCatalogTableOptions() - .setParameters(ImmutableMap.of(METADATA_LOCATION_PROP, "a/new/location"))); reset(client); - // Two invocations, for loadTable and commit. - when(client.load(TABLE_REFERENCE)).thenReturn(tableWithEtag, tableWithNewMetadata); + when(client.load(TABLE_REFERENCE)).thenReturn(tableWithEtag); org.apache.iceberg.Table loadedTable = catalog.loadTable(IDENTIFIER); - when(client.update(any(), any())).thenReturn(tableWithEtag); + // Simulate concurrent modification detected via ETag mismatch + when(client.update(any(), any())) + .thenThrow(new CommitFailedException("Cannot commit: Etag mismatch")); + assertThatThrownBy( () -> loadedTable.updateSchema().addColumn("n", Types.IntegerType.get()).commit()) .isInstanceOf(CommitFailedException.class) - .hasMessageContaining("is not same as the current table metadata location"); + .hasMessageContaining("Cannot commit"); + + // Verify table is loaded only once + verify(client, times(1)).load(TABLE_REFERENCE); } @Test From 1d438fd145a812ab3abbbb26e705ff7560db6f0f Mon Sep 17 00:00:00 2001 From: Becker Ewing Date: Tue, 25 Nov 2025 11:54:49 -0500 Subject: [PATCH 201/201] Set data file `sort_order_id` in manifest for writes from Spark --- .../apache/iceberg/util/SortOrderUtil.java | 17 +++ .../iceberg/util/TestSortOrderUtil.java | 64 ++++++++++ .../apache/iceberg/spark/SparkWriteConf.java | 32 +++++ .../iceberg/spark/SparkWriteOptions.java | 1 + .../iceberg/spark/SparkWriteRequirements.java | 31 ++++- .../apache/iceberg/spark/SparkWriteUtil.java | 111 +++++++++++++----- .../SparkShufflingFileRewriteRunner.java | 16 +++ .../spark/source/SparkPositionDeltaWrite.java | 9 +- .../iceberg/spark/source/SparkWrite.java | 9 +- .../spark/source/SparkWriteBuilder.java | 4 +- .../iceberg/spark/TestSparkWriteConf.java | 22 ++++ .../actions/TestRewriteDataFilesAction.java | 74 ++++++++++++ .../spark/source/TestSparkDataWrite.java | 51 ++++++++ .../apache/iceberg/spark/SparkWriteConf.java | 32 +++++ .../iceberg/spark/SparkWriteOptions.java | 1 + .../iceberg/spark/SparkWriteRequirements.java | 31 ++++- .../apache/iceberg/spark/SparkWriteUtil.java | 111 +++++++++++++----- .../SparkShufflingFileRewriteRunner.java | 16 +++ .../spark/source/SparkPositionDeltaWrite.java | 9 +- .../iceberg/spark/source/SparkWrite.java | 9 +- .../spark/source/SparkWriteBuilder.java | 4 +- .../iceberg/spark/TestSparkWriteConf.java | 22 ++++ .../actions/TestRewriteDataFilesAction.java | 74 ++++++++++++ .../spark/source/TestSparkDataWrite.java | 51 ++++++++ 24 files changed, 727 insertions(+), 74 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/util/SortOrderUtil.java b/core/src/main/java/org/apache/iceberg/util/SortOrderUtil.java index 37e0c1fffab0..4d7a631ab559 100644 --- a/core/src/main/java/org/apache/iceberg/util/SortOrderUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SortOrderUtil.java @@ -46,6 +46,23 @@ public static SortOrder buildSortOrder(Table table, SortOrder sortOrder) { return buildSortOrder(table.schema(), table.spec(), sortOrder); } + /** + * Attempts to match a user-supplied {@link SortOrder} with an equivalent sort order from a {@link + * Table}. + * + * @param table the table to try and match the sort order against + * @param userSuppliedSortOrder the user supplied sort order to try and match with a table sort + * order + * @return the matching {@link SortOrder} from the table (with the orderId set) or {@link + * SortOrder#unsorted()} if no match is found. + */ + public static SortOrder maybeFindTableSortOrder(Table table, SortOrder userSuppliedSortOrder) { + return table.sortOrders().values().stream() + .filter(sortOrder -> sortOrder.sameOrder(userSuppliedSortOrder)) + .findFirst() + .orElseGet(SortOrder::unsorted); + } + /** * Build a final sort order that satisfies the clustering required by the partition spec. * diff --git a/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java b/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java index 02c81de93222..3757b70dd334 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestSortOrderUtil.java @@ -287,4 +287,68 @@ public void testSortOrderClusteringWithRedundantPartitionFieldsMissing() { .as("Should add spec fields as prefix") .isEqualTo(expected); } + + @Test + public void testFindSortOrderForTable() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + SortOrder order = SortOrder.builderFor(SCHEMA).withOrderId(1).asc("id", NULLS_LAST).build(); + TestTables.TestTable table = TestTables.create(tableDir, "test", SCHEMA, spec, order, 2); + + SortOrder tableSortOrder = table.sortOrder(); + + SortOrder actualOrder = SortOrderUtil.maybeFindTableSortOrder(table, tableSortOrder); + + assertThat(actualOrder).as("Should find current table sort order").isEqualTo(table.sortOrder()); + } + + @Test + public void testFindSortOrderForTableWithoutFieldId() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + SortOrder order = SortOrder.builderFor(SCHEMA).withOrderId(1).asc("id", NULLS_LAST).build(); + TestTables.TestTable table = TestTables.create(tableDir, "test", SCHEMA, spec, order, 2); + + SortOrder userSuppliedOrder = + SortOrder.builderFor(table.schema()).asc("id", NULLS_LAST).build(); + + SortOrder actualOrder = SortOrderUtil.maybeFindTableSortOrder(table, userSuppliedOrder); + + assertThat(actualOrder).as("Should find current table sort order").isEqualTo(table.sortOrder()); + } + + @Test + public void testFindSortOrderForTableThatIsNotCurrentOrder() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + SortOrder order = SortOrder.builderFor(SCHEMA).withOrderId(1).asc("id", NULLS_LAST).build(); + TestTables.TestTable table = TestTables.create(tableDir, "test", SCHEMA, spec, order, 2); + + table.replaceSortOrder().asc("data").desc("ts").commit(); + + SortOrder userSuppliedOrder = + SortOrder.builderFor(table.schema()).asc("id", NULLS_LAST).build(); + + SortOrder actualOrder = SortOrderUtil.maybeFindTableSortOrder(table, userSuppliedOrder); + + assertThat(actualOrder) + .as("Should find first sorted table sort order") + .isEqualTo(table.sortOrders().get(1)); + } + + @Test + public void testReturnsEmptyForFindingNonMatchingSortOrder() { + PartitionSpec spec = PartitionSpec.unpartitioned(); + SortOrder order = SortOrder.builderFor(SCHEMA).withOrderId(1).asc("id", NULLS_LAST).build(); + TestTables.TestTable table = TestTables.create(tableDir, "test", SCHEMA, spec, order, 2); + + table.replaceSortOrder().asc("data").desc("ts").commit(); + + SortOrder userSuppliedOrder = + SortOrder.builderFor(table.schema()).desc("id", NULLS_LAST).build(); + + SortOrder actualOrder = SortOrderUtil.maybeFindTableSortOrder(table, userSuppliedOrder); + + assertThat(actualOrder) + .as( + "Should return unsorted order if user supplied order does not match any table sort order") + .isEqualTo(SortOrder.unsorted()); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java index b3e8af5fe056..9a750caf920a 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java @@ -42,6 +42,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.IsolationLevel; import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableUtil; @@ -162,6 +163,22 @@ public int outputSpecId() { return outputSpecId; } + public SortOrder outputSortOrder() { + int outputSortOrderId = + confParser + .intConf() + .option(SparkWriteOptions.OUTPUT_SORT_ORDER_ID) + .defaultValue(SortOrder.unsorted().orderId()) + .parse(); + + Preconditions.checkArgument( + table.sortOrders().containsKey(outputSortOrderId), + "Output sort order id %s is not a valid sort order id for table", + outputSortOrderId); + + return table.sortOrders().get(outputSortOrderId); + } + public FileFormat dataFileFormat() { String valueAsString = confParser @@ -280,6 +297,21 @@ public SparkWriteRequirements writeRequirements() { table, distributionMode(), fanoutWriterEnabled(), dataAdvisoryPartitionSize()); } + public SparkWriteRequirements rewriteFilesWriteRequirements() { + Preconditions.checkNotNull( + rewrittenFileSetId(), "Can only use rewrite files write requirements during rewrite job!"); + + SortOrder outputSortOrder = outputSortOrder(); + if (outputSortOrder.isSorted()) { + LOG.info( + "Found explicit sort order {} set in job configuration. Going to apply that to the sort-order-id of the rewritten files", + Spark3Util.describe(outputSortOrder)); + return writeRequirements().withTableSortOrder(outputSortOrder); + } + + return writeRequirements(); + } + @VisibleForTesting DistributionMode distributionMode() { String modeName = diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java index 33db70bae587..1be02feaf0c0 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java @@ -54,6 +54,7 @@ private SparkWriteOptions() {} public static final String REWRITTEN_FILE_SCAN_TASK_SET_ID = "rewritten-file-scan-task-set-id"; public static final String OUTPUT_SPEC_ID = "output-spec-id"; + public static final String OUTPUT_SORT_ORDER_ID = "output-sort-order-id"; public static final String OVERWRITE_MODE = "overwrite-mode"; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java index 833e0e44e391..dd4bc863912f 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java @@ -26,18 +26,32 @@ /** A set of requirements such as distribution and ordering reported to Spark during writes. */ public class SparkWriteRequirements { + public static final long NO_ADVISORY_PARTITION_SIZE = 0; public static final SparkWriteRequirements EMPTY = - new SparkWriteRequirements(Distributions.unspecified(), new SortOrder[0], 0); + new SparkWriteRequirements( + Distributions.unspecified(), + new SortOrder[0], + org.apache.iceberg.SortOrder.unsorted(), + NO_ADVISORY_PARTITION_SIZE); private final Distribution distribution; private final SortOrder[] ordering; + private final org.apache.iceberg.SortOrder icebergOrdering; private final long advisoryPartitionSize; SparkWriteRequirements( - Distribution distribution, SortOrder[] ordering, long advisoryPartitionSize) { + Distribution distribution, + SortOrder[] ordering, + org.apache.iceberg.SortOrder icebergOrdering, + long advisoryPartitionSize) { this.distribution = distribution; this.ordering = ordering; - this.advisoryPartitionSize = advisoryPartitionSize; + this.icebergOrdering = icebergOrdering; + // Spark prohibits requesting a particular advisory partition size without distribution + this.advisoryPartitionSize = + distribution instanceof UnspecifiedDistribution + ? NO_ADVISORY_PARTITION_SIZE + : advisoryPartitionSize; } public Distribution distribution() { @@ -48,12 +62,19 @@ public SortOrder[] ordering() { return ordering; } + public org.apache.iceberg.SortOrder icebergOrdering() { + return icebergOrdering; + } + public boolean hasOrdering() { return ordering.length != 0; } public long advisoryPartitionSize() { - // Spark prohibits requesting a particular advisory partition size without distribution - return distribution instanceof UnspecifiedDistribution ? 0 : advisoryPartitionSize; + return advisoryPartitionSize; + } + + public SparkWriteRequirements withTableSortOrder(org.apache.iceberg.SortOrder sortOrder) { + return new SparkWriteRequirements(distribution, ordering, sortOrder, advisoryPartitionSize); } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java index 0d68a0d8cdd0..535674aba977 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java @@ -55,13 +55,23 @@ public class SparkWriteUtil { private static final Expression[] PARTITION_FILE_CLUSTERING = clusterBy(SPEC_ID, PARTITION, FILE_PATH); - private static final SortOrder[] EMPTY_ORDERING = new SortOrder[0]; - private static final SortOrder[] EXISTING_ROW_ORDERING = orderBy(FILE_PATH, ROW_POSITION); - private static final SortOrder[] PARTITION_ORDERING = orderBy(SPEC_ID, PARTITION); - private static final SortOrder[] PARTITION_FILE_ORDERING = orderBy(SPEC_ID, PARTITION, FILE_PATH); - private static final SortOrder[] POSITION_DELETE_ORDERING = + private static final SortOrder[] EMPTY_SPARK_ORDERING = new SortOrder[0]; + private static final SortOrder[] EXISTING_ROW_SPARK_ORDERING = orderBy(FILE_PATH, ROW_POSITION); + private static final SortOrder[] PARTITION_SPARK_ORDERING = orderBy(SPEC_ID, PARTITION); + private static final SortOrder[] PARTITION_FILE_SPARK_ORDERING = + orderBy(SPEC_ID, PARTITION, FILE_PATH); + private static final SortOrder[] POSITION_DELETE_SPARK_ORDERING = orderBy(SPEC_ID, PARTITION, FILE_PATH, ROW_POSITION); + private static final SparkAndIcebergOrdering EXISTING_ROW_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(EXISTING_ROW_SPARK_ORDERING); + private static final SparkAndIcebergOrdering PARTITION_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(PARTITION_SPARK_ORDERING); + private static final SparkAndIcebergOrdering PARTITION_FILE_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(PARTITION_FILE_SPARK_ORDERING); + private static final SparkAndIcebergOrdering POSITION_DELETE_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(POSITION_DELETE_SPARK_ORDERING); + private SparkWriteUtil() {} /** Builds requirements for batch and micro-batch writes such as append or overwrite. */ @@ -69,8 +79,9 @@ public static SparkWriteRequirements writeRequirements( Table table, DistributionMode mode, boolean fanoutEnabled, long advisoryPartitionSize) { Distribution distribution = writeDistribution(table, mode); - SortOrder[] ordering = writeOrdering(table, fanoutEnabled); - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = writeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } private static Distribution writeDistribution(Table table, DistributionMode mode) { @@ -82,7 +93,7 @@ private static Distribution writeDistribution(Table table, DistributionMode mode return Distributions.clustered(clustering(table)); case RANGE: - return Distributions.ordered(ordering(table)); + return Distributions.ordered(ordering(table).sparkOrder()); default: throw new IllegalArgumentException("Unsupported distribution mode: " + mode); @@ -99,8 +110,9 @@ public static SparkWriteRequirements copyOnWriteRequirements( if (command == DELETE || command == UPDATE) { Distribution distribution = copyOnWriteDeleteUpdateDistribution(table, mode); - SortOrder[] ordering = writeOrdering(table, fanoutEnabled); - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = writeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } else { return writeRequirements(table, mode, fanoutEnabled, advisoryPartitionSize); } @@ -122,9 +134,9 @@ private static Distribution copyOnWriteDeleteUpdateDistribution( case RANGE: if (table.spec().isPartitioned() || table.sortOrder().isSorted()) { - return Distributions.ordered(ordering(table)); + return Distributions.ordered(ordering(table).sparkOrder()); } else { - return Distributions.ordered(EXISTING_ROW_ORDERING); + return Distributions.ordered(EXISTING_ROW_ORDERING.sparkOrder()); } default: @@ -142,12 +154,15 @@ public static SparkWriteRequirements positionDeltaRequirements( if (command == UPDATE || command == MERGE) { Distribution distribution = positionDeltaUpdateMergeDistribution(table, mode); - SortOrder[] ordering = positionDeltaUpdateMergeOrdering(table, fanoutEnabled); - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = positionDeltaUpdateMergeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } else { Distribution distribution = positionDeltaDeleteDistribution(table, mode); - SortOrder[] ordering = fanoutEnabled ? EMPTY_ORDERING : POSITION_DELETE_ORDERING; - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = + fanoutEnabled ? SparkAndIcebergOrdering.unsorted() : POSITION_DELETE_ORDERING; + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } } @@ -167,9 +182,15 @@ private static Distribution positionDeltaUpdateMergeDistribution( case RANGE: if (table.spec().isUnpartitioned()) { - return Distributions.ordered(concat(PARTITION_FILE_ORDERING, ordering(table))); + return Distributions.ordered( + SparkAndIcebergOrdering.forTable(table) + .prependOrder(PARTITION_FILE_SPARK_ORDERING) + .sparkOrder()); } else { - return Distributions.ordered(concat(PARTITION_ORDERING, ordering(table))); + return Distributions.ordered( + SparkAndIcebergOrdering.forTable(table) + .prependOrder(PARTITION_SPARK_ORDERING) + .sparkOrder()); } default: @@ -177,11 +198,12 @@ private static Distribution positionDeltaUpdateMergeDistribution( } } - private static SortOrder[] positionDeltaUpdateMergeOrdering(Table table, boolean fanoutEnabled) { + private static SparkAndIcebergOrdering positionDeltaUpdateMergeOrdering( + Table table, boolean fanoutEnabled) { if (fanoutEnabled && table.sortOrder().isUnsorted()) { - return EMPTY_ORDERING; + return SparkAndIcebergOrdering.unsorted(); } else { - return concat(POSITION_DELETE_ORDERING, ordering(table)); + return SparkAndIcebergOrdering.forTable(table).prependOrder(POSITION_DELETE_SPARK_ORDERING); } } @@ -199,9 +221,9 @@ private static Distribution positionDeltaDeleteDistribution(Table table, Distrib case RANGE: if (table.spec().isUnpartitioned()) { - return Distributions.ordered(PARTITION_FILE_ORDERING); + return Distributions.ordered(PARTITION_FILE_ORDERING.sparkOrder()); } else { - return Distributions.ordered(PARTITION_ORDERING); + return Distributions.ordered(PARTITION_ORDERING.sparkOrder()); } default: @@ -213,9 +235,9 @@ private static Distribution positionDeltaDeleteDistribution(Table table, Distrib // - there is a defined table sort order, so it is clear how the data should be ordered // - the table is partitioned and fanout writers are disabled, // so records for one partition must be co-located within a task - private static SortOrder[] writeOrdering(Table table, boolean fanoutEnabled) { + private static SparkAndIcebergOrdering writeOrdering(Table table, boolean fanoutEnabled) { if (fanoutEnabled && table.sortOrder().isUnsorted()) { - return EMPTY_ORDERING; + return SparkAndIcebergOrdering.unsorted(); } else { return ordering(table); } @@ -225,8 +247,8 @@ private static Expression[] clustering(Table table) { return Spark3Util.toTransforms(table.spec()); } - private static SortOrder[] ordering(Table table) { - return Spark3Util.toOrdering(SortOrderUtil.buildSortOrder(table)); + private static SparkAndIcebergOrdering ordering(Table table) { + return SparkAndIcebergOrdering.forTable(table); } private static Expression[] concat(Expression[] clustering, Expression... otherClustering) { @@ -256,4 +278,39 @@ private static SortOrder[] orderBy(Expression... exprs) { private static SortOrder sort(Expression expr) { return Expressions.sort(expr, SortDirection.ASCENDING); } + + private static class SparkAndIcebergOrdering { + private static final SparkAndIcebergOrdering UNSORTED = + new SparkAndIcebergOrdering(org.apache.iceberg.SortOrder.unsorted(), EMPTY_SPARK_ORDERING); + + private final org.apache.iceberg.SortOrder icebergSortOrder; + private final SortOrder[] sparkSortOrder; + + private SparkAndIcebergOrdering( + org.apache.iceberg.SortOrder icebergSortOrder, SortOrder[] sparkSortOrder) { + this.icebergSortOrder = icebergSortOrder; + this.sparkSortOrder = sparkSortOrder; + } + + public static SparkAndIcebergOrdering forTable(Table table) { + return new SparkAndIcebergOrdering( + table.sortOrder(), Spark3Util.toOrdering(SortOrderUtil.buildSortOrder(table))); + } + + public static SparkAndIcebergOrdering unsorted() { + return UNSORTED; + } + + public SparkAndIcebergOrdering prependOrder(SortOrder[] ordering) { + return new SparkAndIcebergOrdering(icebergSortOrder, concat(ordering, sparkSortOrder)); + } + + public org.apache.iceberg.SortOrder icebergOrder() { + return icebergSortOrder; + } + + public SortOrder[] sparkOrder() { + return sparkSortOrder; + } + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java index b1c5a5c0901a..a6178eff316b 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java @@ -47,10 +47,14 @@ import org.apache.spark.sql.connector.expressions.SortOrder; import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering; import org.apache.spark.sql.execution.datasources.v2.DistributionAndOrderingUtils$; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.Option; abstract class SparkShufflingFileRewriteRunner extends SparkDataFileRewriteRunner { + private static final Logger LOG = LoggerFactory.getLogger(SparkShufflingFileRewriteRunner.class); + /** * The number of shuffle partitions to use for each output file. By default, this file rewriter * assumes each shuffle partition would become a separate output file. Attempting to generate @@ -119,6 +123,17 @@ public void doRewrite(String groupId, RewriteFileGroup fileGroup) { spec(fileGroup.outputSpecId()), fileGroup.expectedOutputFiles())); + org.apache.iceberg.SortOrder sortOrderInJobSpec = sortOrder(); + + org.apache.iceberg.SortOrder maybeMatchingTableSortOrder = + SortOrderUtil.maybeFindTableSortOrder(table(), sortOrder()); + + if (sortOrderInJobSpec.isSorted() && maybeMatchingTableSortOrder.isUnsorted()) { + LOG.warn( + "Sort order specified for job {} doesn't match any table sort orders, so going to not mark rewritten files as sorted in the manifest files", + Spark3Util.describe(sortOrderInJobSpec)); + } + sortedDF .write() .format("iceberg") @@ -126,6 +141,7 @@ public void doRewrite(String groupId, RewriteFileGroup fileGroup) { .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, fileGroup.maxOutputFileSize()) .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") .option(SparkWriteOptions.OUTPUT_SPEC_ID, fileGroup.outputSpecId()) + .option(SparkWriteOptions.OUTPUT_SORT_ORDER_ID, maybeMatchingTableSortOrder.orderId()) .mode("append") .save(groupId); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index ddad1a749aa9..5d92ea2cd67d 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -180,7 +180,8 @@ public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { broadcastRewritableDeletes(), command, context, - writeProperties); + writeProperties, + writeRequirements.icebergOrdering()); } private Broadcast> broadcastRewritableDeletes() { @@ -390,18 +391,21 @@ private static class PositionDeltaWriteFactory implements DeltaWriterFactory { private final Command command; private final Context context; private final Map writeProperties; + private final org.apache.iceberg.SortOrder sortOrder; PositionDeltaWriteFactory( Broadcast
    tableBroadcast, Broadcast> rewritableDeletesBroadcast, Command command, Context context, - Map writeProperties) { + Map writeProperties, + org.apache.iceberg.SortOrder sortOrder) { this.tableBroadcast = tableBroadcast; this.rewritableDeletesBroadcast = rewritableDeletesBroadcast; this.command = command; this.context = context; this.writeProperties = writeProperties; + this.sortOrder = sortOrder; } @Override @@ -428,6 +432,7 @@ public DeltaWriter createWriter(int partitionId, long taskId) { .deleteFileFormat(context.deleteFileFormat()) .positionDeleteSparkType(context.deleteSparkType()) .writeProperties(writeProperties) + .dataSortOrder(sortOrder) .build(); if (command == DELETE) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index 15c70e4a6621..97360307a297 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -202,7 +202,8 @@ private WriterFactory createWriterFactory() { writeSchema, dsSchema, useFanoutWriter, - writeProperties); + writeProperties, + writeRequirements.icebergOrdering()); } private void commitOperation(SnapshotUpdate operation, String description) { @@ -672,6 +673,7 @@ private static class WriterFactory implements DataWriterFactory, StreamingDataWr private final boolean useFanoutWriter; private final String queryId; private final Map writeProperties; + private final org.apache.iceberg.SortOrder sortOrder; protected WriterFactory( Broadcast
    tableBroadcast, @@ -682,7 +684,8 @@ protected WriterFactory( Schema writeSchema, StructType dsSchema, boolean useFanoutWriter, - Map writeProperties) { + Map writeProperties, + org.apache.iceberg.SortOrder sortOrder) { this.tableBroadcast = tableBroadcast; this.format = format; this.outputSpecId = outputSpecId; @@ -692,6 +695,7 @@ protected WriterFactory( this.useFanoutWriter = useFanoutWriter; this.queryId = queryId; this.writeProperties = writeProperties; + this.sortOrder = sortOrder; } @Override @@ -716,6 +720,7 @@ public DataWriter createWriter(int partitionId, long taskId, long e .dataSchema(writeSchema) .dataSparkType(dsSchema) .writeProperties(writeProperties) + .dataSortOrder(sortOrder) .build(); if (spec.isUnpartitioned()) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java index e608a40b72ad..4b76904e7c3e 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java @@ -182,7 +182,9 @@ public StreamingWrite toStreaming() { } private SparkWriteRequirements writeRequirements() { - if (overwriteFiles) { + if (rewrittenFileSetId != null) { + return writeConf.rewriteFilesWriteRequirements(); + } else if (overwriteFiles) { return writeConf.copyOnWriteRequirements(copyOnWriteCommand); } else { return writeConf.writeRequirements(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java index a9b5d1a237b4..add19571ca44 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java @@ -45,6 +45,7 @@ import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.MERGE; import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.UPDATE; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.time.Duration; @@ -552,6 +553,27 @@ public void testDVWriteConf() { assertThat(writeConf.deleteFileFormat()).isEqualTo(FileFormat.PUFFIN); } + @TestTemplate + public void testSortOrderWriteConf() { + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").commit(); + + SparkWriteConf writeConf = + new SparkWriteConf( + spark, table, ImmutableMap.of(SparkWriteOptions.OUTPUT_SORT_ORDER_ID, "1")); + + assertThat(writeConf.outputSortOrder()).isEqualTo(table.sortOrder()); + + SparkWriteConf writeConfForUnknownSortOrder = + new SparkWriteConf( + spark, table, ImmutableMap.of(SparkWriteOptions.OUTPUT_SORT_ORDER_ID, "999")); + + assertThatIllegalArgumentException() + .isThrownBy(writeConfForUnknownSortOrder::outputSortOrder) + .withMessage("Output sort order id 999 is not a valid sort order id for table"); + } + private void testWriteProperties(List> propertiesSuite) { withSQLConf( propertiesSuite.get(0), diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 2b4a2a211ec0..1d55f15c7878 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -1584,6 +1584,7 @@ public void testSimpleSort() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + dataFilesSortOrderShouldMatchTableSortOrder(table); } @TestTemplate @@ -1620,6 +1621,7 @@ public void testSortAfterPartitionChange() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + dataFilesSortOrderShouldMatchTableSortOrder(table); } @TestTemplate @@ -1651,6 +1653,9 @@ public void testSortCustomSortOrder() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + // Since the SortOrder isn't in the table spec, these data files should report having the + // default table sort order e.g. unsorted + dataFilesShouldHaveSortOrderIdMatching(table, SortOrder.unsorted()); } @TestTemplate @@ -1691,6 +1696,58 @@ public void testSortCustomSortOrderRequiresRepartition() { shouldHaveMultipleFiles(table); shouldHaveLastCommitUnsorted(table, "c2"); shouldHaveLastCommitSorted(table, "c3"); + // Since the table ordering is on C2, but we rewrote explicitly with C3 which isn't a sort order + // spec, the table files should report unsorted + dataFilesShouldHaveSortOrderIdMatching(table, SortOrder.unsorted()); + } + + @TestTemplate + public void testSortPastTableSortOrderGetsAppliedToFiles() { + int partitions = 4; + Table table = createTable(); + writeRecords(20, SCALE, partitions); + shouldHaveLastCommitUnsorted(table, "c3"); + + // Add a partition column so this requires repartitioning + table.updateSpec().addField("c1").commit(); + + // Add the sort order we want to use during the rewrite job + table.replaceSortOrder().asc("c3").commit(); + SortOrder c3SortOrder = table.sortOrder(); + + // Replace that sort order with a newer one that we aren't going to use, but is the current + // table ordering + table.replaceSortOrder().asc("c2").commit(); + shouldHaveFiles(table, 20); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + RewriteDataFiles.Result result = + basicRewrite(table) + .sort(SortOrder.builderFor(table.schema()).asc("c3").build()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + RewriteDataFiles.TARGET_FILE_SIZE_BYTES, + Integer.toString(averageFileSize(table) / partitions)) + .execute(); + + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + shouldHaveMultipleFiles(table); + shouldHaveLastCommitUnsorted(table, "c2"); + shouldHaveLastCommitSorted(table, "c3"); + // Since the table ordering is on C2, but we rewrote explicitly with C3 which is in the table + // sort order spec, the table files should report C3 sort order + dataFilesShouldHaveSortOrderIdMatching(table, c3SortOrder); } @TestTemplate @@ -1731,6 +1788,9 @@ public void testAutoSortShuffleOutput() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + // Since the sort order being applied here isn't anywhere on the table spec, all files despite + // being physically sorted should report unsorted in the manifest entry + dataFilesShouldHaveSortOrderIdMatching(table, SortOrder.unsorted()); } @TestTemplate @@ -2634,4 +2694,18 @@ public boolean matches(RewriteFileGroup argument) { return groupIDs.contains(argument.info().globalIndex()); } } + + private void dataFilesSortOrderShouldMatchTableSortOrder(Table table) { + dataFilesShouldHaveSortOrderIdMatching(table, table.sortOrder()); + } + + private void dataFilesShouldHaveSortOrderIdMatching(Table table, SortOrder sortOrder) { + try (CloseableIterable files = table.newScan().planFiles()) { + assertThat(files) + .extracting(fileScanTask -> fileScanTask.file().sortOrderId()) + .containsOnly(sortOrder.orderId()); + } catch (IOException e) { + throw new RuntimeException("Failed to close file scan tasks", e); + } + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index 94547c2cf8fb..1c9fdd54b633 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -44,6 +44,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.CommitStateUnknownException; @@ -154,6 +155,7 @@ public void testBasicWrite() { assertThat(file.splitOffsets()).as("Split offsets not present").isNotNull(); } assertThat(file.recordCount()).as("Should have reported record count as 1").isEqualTo(1); + assertThat(file.sortOrderId()).isEqualTo(SortOrder.unsorted().orderId()); // TODO: append more metric info if (format.equals(FileFormat.PARQUET)) { assertThat(file.columnSizes()).as("Column sizes metric not present").isNotNull(); @@ -555,6 +557,55 @@ public void testViewsReturnRecentResults() { assertThat(actual2).hasSameSizeAs(expected2).isEqualTo(expected2); } + @TestTemplate + public void testWriteDataFilesInTableSortOrder() { + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + table.replaceSortOrder().asc("id").commit(); + + List expected = Lists.newArrayListWithCapacity(4000); + for (int i = 0; i < 4000; i++) { + expected.add(new SimpleRecord(i, "a")); + } + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + createBranch(table); + table.refresh(); + + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + + List files = Lists.newArrayList(); + for (ManifestFile manifest : + SnapshotUtil.latestSnapshot(table, branch).allManifests(table.io())) { + for (DataFile file : ManifestFiles.read(manifest, table.io())) { + files.add(file); + } + } + + assertThat(files) + .extracting(DataFile::sortOrderId) + .as("All DataFiles are written with the table sort order id") + .containsOnly(table.sortOrder().orderId()); + } + public void partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType option) { File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test"); diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java index 96131e0e56dd..f85fb0dfb9ff 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteConf.java @@ -42,6 +42,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.IsolationLevel; import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableUtil; @@ -164,6 +165,22 @@ public int outputSpecId() { return outputSpecId; } + public SortOrder outputSortOrder() { + int outputSortOrderId = + confParser + .intConf() + .option(SparkWriteOptions.OUTPUT_SORT_ORDER_ID) + .defaultValue(SortOrder.unsorted().orderId()) + .parse(); + + Preconditions.checkArgument( + table.sortOrders().containsKey(outputSortOrderId), + "Output sort order id %s is not a valid sort order id for table", + outputSortOrderId); + + return table.sortOrders().get(outputSortOrderId); + } + public FileFormat dataFileFormat() { String valueAsString = confParser @@ -284,6 +301,21 @@ public SparkWriteRequirements writeRequirements() { table, distributionMode(), fanoutWriterEnabled(), dataAdvisoryPartitionSize()); } + public SparkWriteRequirements rewriteFilesWriteRequirements() { + Preconditions.checkNotNull( + rewrittenFileSetId(), "Can only use rewrite files write requirements during rewrite job!"); + + SortOrder outputSortOrder = outputSortOrder(); + if (outputSortOrder.isSorted()) { + LOG.info( + "Found explicit sort order {} set in job configuration. Going to apply that to the sort-order-id of the rewritten files", + Spark3Util.describe(outputSortOrder)); + return writeRequirements().withTableSortOrder(outputSortOrder); + } + + return writeRequirements(); + } + @VisibleForTesting DistributionMode distributionMode() { String modeName = diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java index 33db70bae587..1be02feaf0c0 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java @@ -54,6 +54,7 @@ private SparkWriteOptions() {} public static final String REWRITTEN_FILE_SCAN_TASK_SET_ID = "rewritten-file-scan-task-set-id"; public static final String OUTPUT_SPEC_ID = "output-spec-id"; + public static final String OUTPUT_SORT_ORDER_ID = "output-sort-order-id"; public static final String OVERWRITE_MODE = "overwrite-mode"; diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java index 833e0e44e391..dd4bc863912f 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteRequirements.java @@ -26,18 +26,32 @@ /** A set of requirements such as distribution and ordering reported to Spark during writes. */ public class SparkWriteRequirements { + public static final long NO_ADVISORY_PARTITION_SIZE = 0; public static final SparkWriteRequirements EMPTY = - new SparkWriteRequirements(Distributions.unspecified(), new SortOrder[0], 0); + new SparkWriteRequirements( + Distributions.unspecified(), + new SortOrder[0], + org.apache.iceberg.SortOrder.unsorted(), + NO_ADVISORY_PARTITION_SIZE); private final Distribution distribution; private final SortOrder[] ordering; + private final org.apache.iceberg.SortOrder icebergOrdering; private final long advisoryPartitionSize; SparkWriteRequirements( - Distribution distribution, SortOrder[] ordering, long advisoryPartitionSize) { + Distribution distribution, + SortOrder[] ordering, + org.apache.iceberg.SortOrder icebergOrdering, + long advisoryPartitionSize) { this.distribution = distribution; this.ordering = ordering; - this.advisoryPartitionSize = advisoryPartitionSize; + this.icebergOrdering = icebergOrdering; + // Spark prohibits requesting a particular advisory partition size without distribution + this.advisoryPartitionSize = + distribution instanceof UnspecifiedDistribution + ? NO_ADVISORY_PARTITION_SIZE + : advisoryPartitionSize; } public Distribution distribution() { @@ -48,12 +62,19 @@ public SortOrder[] ordering() { return ordering; } + public org.apache.iceberg.SortOrder icebergOrdering() { + return icebergOrdering; + } + public boolean hasOrdering() { return ordering.length != 0; } public long advisoryPartitionSize() { - // Spark prohibits requesting a particular advisory partition size without distribution - return distribution instanceof UnspecifiedDistribution ? 0 : advisoryPartitionSize; + return advisoryPartitionSize; + } + + public SparkWriteRequirements withTableSortOrder(org.apache.iceberg.SortOrder sortOrder) { + return new SparkWriteRequirements(distribution, ordering, sortOrder, advisoryPartitionSize); } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java index 0d68a0d8cdd0..535674aba977 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkWriteUtil.java @@ -55,13 +55,23 @@ public class SparkWriteUtil { private static final Expression[] PARTITION_FILE_CLUSTERING = clusterBy(SPEC_ID, PARTITION, FILE_PATH); - private static final SortOrder[] EMPTY_ORDERING = new SortOrder[0]; - private static final SortOrder[] EXISTING_ROW_ORDERING = orderBy(FILE_PATH, ROW_POSITION); - private static final SortOrder[] PARTITION_ORDERING = orderBy(SPEC_ID, PARTITION); - private static final SortOrder[] PARTITION_FILE_ORDERING = orderBy(SPEC_ID, PARTITION, FILE_PATH); - private static final SortOrder[] POSITION_DELETE_ORDERING = + private static final SortOrder[] EMPTY_SPARK_ORDERING = new SortOrder[0]; + private static final SortOrder[] EXISTING_ROW_SPARK_ORDERING = orderBy(FILE_PATH, ROW_POSITION); + private static final SortOrder[] PARTITION_SPARK_ORDERING = orderBy(SPEC_ID, PARTITION); + private static final SortOrder[] PARTITION_FILE_SPARK_ORDERING = + orderBy(SPEC_ID, PARTITION, FILE_PATH); + private static final SortOrder[] POSITION_DELETE_SPARK_ORDERING = orderBy(SPEC_ID, PARTITION, FILE_PATH, ROW_POSITION); + private static final SparkAndIcebergOrdering EXISTING_ROW_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(EXISTING_ROW_SPARK_ORDERING); + private static final SparkAndIcebergOrdering PARTITION_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(PARTITION_SPARK_ORDERING); + private static final SparkAndIcebergOrdering PARTITION_FILE_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(PARTITION_FILE_SPARK_ORDERING); + private static final SparkAndIcebergOrdering POSITION_DELETE_ORDERING = + SparkAndIcebergOrdering.unsorted().prependOrder(POSITION_DELETE_SPARK_ORDERING); + private SparkWriteUtil() {} /** Builds requirements for batch and micro-batch writes such as append or overwrite. */ @@ -69,8 +79,9 @@ public static SparkWriteRequirements writeRequirements( Table table, DistributionMode mode, boolean fanoutEnabled, long advisoryPartitionSize) { Distribution distribution = writeDistribution(table, mode); - SortOrder[] ordering = writeOrdering(table, fanoutEnabled); - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = writeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } private static Distribution writeDistribution(Table table, DistributionMode mode) { @@ -82,7 +93,7 @@ private static Distribution writeDistribution(Table table, DistributionMode mode return Distributions.clustered(clustering(table)); case RANGE: - return Distributions.ordered(ordering(table)); + return Distributions.ordered(ordering(table).sparkOrder()); default: throw new IllegalArgumentException("Unsupported distribution mode: " + mode); @@ -99,8 +110,9 @@ public static SparkWriteRequirements copyOnWriteRequirements( if (command == DELETE || command == UPDATE) { Distribution distribution = copyOnWriteDeleteUpdateDistribution(table, mode); - SortOrder[] ordering = writeOrdering(table, fanoutEnabled); - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = writeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } else { return writeRequirements(table, mode, fanoutEnabled, advisoryPartitionSize); } @@ -122,9 +134,9 @@ private static Distribution copyOnWriteDeleteUpdateDistribution( case RANGE: if (table.spec().isPartitioned() || table.sortOrder().isSorted()) { - return Distributions.ordered(ordering(table)); + return Distributions.ordered(ordering(table).sparkOrder()); } else { - return Distributions.ordered(EXISTING_ROW_ORDERING); + return Distributions.ordered(EXISTING_ROW_ORDERING.sparkOrder()); } default: @@ -142,12 +154,15 @@ public static SparkWriteRequirements positionDeltaRequirements( if (command == UPDATE || command == MERGE) { Distribution distribution = positionDeltaUpdateMergeDistribution(table, mode); - SortOrder[] ordering = positionDeltaUpdateMergeOrdering(table, fanoutEnabled); - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = positionDeltaUpdateMergeOrdering(table, fanoutEnabled); + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } else { Distribution distribution = positionDeltaDeleteDistribution(table, mode); - SortOrder[] ordering = fanoutEnabled ? EMPTY_ORDERING : POSITION_DELETE_ORDERING; - return new SparkWriteRequirements(distribution, ordering, advisoryPartitionSize); + SparkAndIcebergOrdering ordering = + fanoutEnabled ? SparkAndIcebergOrdering.unsorted() : POSITION_DELETE_ORDERING; + return new SparkWriteRequirements( + distribution, ordering.sparkOrder(), ordering.icebergOrder(), advisoryPartitionSize); } } @@ -167,9 +182,15 @@ private static Distribution positionDeltaUpdateMergeDistribution( case RANGE: if (table.spec().isUnpartitioned()) { - return Distributions.ordered(concat(PARTITION_FILE_ORDERING, ordering(table))); + return Distributions.ordered( + SparkAndIcebergOrdering.forTable(table) + .prependOrder(PARTITION_FILE_SPARK_ORDERING) + .sparkOrder()); } else { - return Distributions.ordered(concat(PARTITION_ORDERING, ordering(table))); + return Distributions.ordered( + SparkAndIcebergOrdering.forTable(table) + .prependOrder(PARTITION_SPARK_ORDERING) + .sparkOrder()); } default: @@ -177,11 +198,12 @@ private static Distribution positionDeltaUpdateMergeDistribution( } } - private static SortOrder[] positionDeltaUpdateMergeOrdering(Table table, boolean fanoutEnabled) { + private static SparkAndIcebergOrdering positionDeltaUpdateMergeOrdering( + Table table, boolean fanoutEnabled) { if (fanoutEnabled && table.sortOrder().isUnsorted()) { - return EMPTY_ORDERING; + return SparkAndIcebergOrdering.unsorted(); } else { - return concat(POSITION_DELETE_ORDERING, ordering(table)); + return SparkAndIcebergOrdering.forTable(table).prependOrder(POSITION_DELETE_SPARK_ORDERING); } } @@ -199,9 +221,9 @@ private static Distribution positionDeltaDeleteDistribution(Table table, Distrib case RANGE: if (table.spec().isUnpartitioned()) { - return Distributions.ordered(PARTITION_FILE_ORDERING); + return Distributions.ordered(PARTITION_FILE_ORDERING.sparkOrder()); } else { - return Distributions.ordered(PARTITION_ORDERING); + return Distributions.ordered(PARTITION_ORDERING.sparkOrder()); } default: @@ -213,9 +235,9 @@ private static Distribution positionDeltaDeleteDistribution(Table table, Distrib // - there is a defined table sort order, so it is clear how the data should be ordered // - the table is partitioned and fanout writers are disabled, // so records for one partition must be co-located within a task - private static SortOrder[] writeOrdering(Table table, boolean fanoutEnabled) { + private static SparkAndIcebergOrdering writeOrdering(Table table, boolean fanoutEnabled) { if (fanoutEnabled && table.sortOrder().isUnsorted()) { - return EMPTY_ORDERING; + return SparkAndIcebergOrdering.unsorted(); } else { return ordering(table); } @@ -225,8 +247,8 @@ private static Expression[] clustering(Table table) { return Spark3Util.toTransforms(table.spec()); } - private static SortOrder[] ordering(Table table) { - return Spark3Util.toOrdering(SortOrderUtil.buildSortOrder(table)); + private static SparkAndIcebergOrdering ordering(Table table) { + return SparkAndIcebergOrdering.forTable(table); } private static Expression[] concat(Expression[] clustering, Expression... otherClustering) { @@ -256,4 +278,39 @@ private static SortOrder[] orderBy(Expression... exprs) { private static SortOrder sort(Expression expr) { return Expressions.sort(expr, SortDirection.ASCENDING); } + + private static class SparkAndIcebergOrdering { + private static final SparkAndIcebergOrdering UNSORTED = + new SparkAndIcebergOrdering(org.apache.iceberg.SortOrder.unsorted(), EMPTY_SPARK_ORDERING); + + private final org.apache.iceberg.SortOrder icebergSortOrder; + private final SortOrder[] sparkSortOrder; + + private SparkAndIcebergOrdering( + org.apache.iceberg.SortOrder icebergSortOrder, SortOrder[] sparkSortOrder) { + this.icebergSortOrder = icebergSortOrder; + this.sparkSortOrder = sparkSortOrder; + } + + public static SparkAndIcebergOrdering forTable(Table table) { + return new SparkAndIcebergOrdering( + table.sortOrder(), Spark3Util.toOrdering(SortOrderUtil.buildSortOrder(table))); + } + + public static SparkAndIcebergOrdering unsorted() { + return UNSORTED; + } + + public SparkAndIcebergOrdering prependOrder(SortOrder[] ordering) { + return new SparkAndIcebergOrdering(icebergSortOrder, concat(ordering, sparkSortOrder)); + } + + public org.apache.iceberg.SortOrder icebergOrder() { + return icebergSortOrder; + } + + public SortOrder[] sparkOrder() { + return sparkSortOrder; + } + } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java index 569eb252cba5..1ba4c7e2fac2 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkShufflingFileRewriteRunner.java @@ -47,10 +47,14 @@ import org.apache.spark.sql.connector.expressions.SortOrder; import org.apache.spark.sql.connector.write.RequiresDistributionAndOrdering; import org.apache.spark.sql.execution.datasources.v2.DistributionAndOrderingUtils$; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.Option; abstract class SparkShufflingFileRewriteRunner extends SparkDataFileRewriteRunner { + private static final Logger LOG = LoggerFactory.getLogger(SparkShufflingFileRewriteRunner.class); + /** * The number of shuffle partitions to use for each output file. By default, this file rewriter * assumes each shuffle partition would become a separate output file. Attempting to generate @@ -119,6 +123,17 @@ public void doRewrite(String groupId, RewriteFileGroup fileGroup) { spec(fileGroup.outputSpecId()), fileGroup.expectedOutputFiles())); + org.apache.iceberg.SortOrder sortOrderInJobSpec = sortOrder(); + + org.apache.iceberg.SortOrder maybeMatchingTableSortOrder = + SortOrderUtil.maybeFindTableSortOrder(table(), sortOrder()); + + if (sortOrderInJobSpec.isSorted() && maybeMatchingTableSortOrder.isUnsorted()) { + LOG.warn( + "Sort order specified for job {} doesn't match any table sort orders, so going to not mark rewritten files as sorted in the manifest files", + Spark3Util.describe(sortOrderInJobSpec)); + } + sortedDF .write() .format("iceberg") @@ -126,6 +141,7 @@ public void doRewrite(String groupId, RewriteFileGroup fileGroup) { .option(SparkWriteOptions.TARGET_FILE_SIZE_BYTES, fileGroup.maxOutputFileSize()) .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") .option(SparkWriteOptions.OUTPUT_SPEC_ID, fileGroup.outputSpecId()) + .option(SparkWriteOptions.OUTPUT_SORT_ORDER_ID, maybeMatchingTableSortOrder.orderId()) .mode("append") .save(groupId); } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index d072397dc6a3..e0c842e9a6d7 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -182,7 +182,8 @@ public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { broadcastRewritableDeletes(), command, context, - writeProperties); + writeProperties, + writeRequirements.icebergOrdering()); } private Broadcast> broadcastRewritableDeletes() { @@ -392,18 +393,21 @@ private static class PositionDeltaWriteFactory implements DeltaWriterFactory { private final Command command; private final Context context; private final Map writeProperties; + private final org.apache.iceberg.SortOrder sortOrder; PositionDeltaWriteFactory( Broadcast
    tableBroadcast, Broadcast> rewritableDeletesBroadcast, Command command, Context context, - Map writeProperties) { + Map writeProperties, + org.apache.iceberg.SortOrder sortOrder) { this.tableBroadcast = tableBroadcast; this.rewritableDeletesBroadcast = rewritableDeletesBroadcast; this.command = command; this.context = context; this.writeProperties = writeProperties; + this.sortOrder = sortOrder; } @Override @@ -430,6 +434,7 @@ public DeltaWriter createWriter(int partitionId, long taskId) { .deleteFileFormat(context.deleteFileFormat()) .positionDeleteSparkType(context.deleteSparkType()) .writeProperties(writeProperties) + .dataSortOrder(sortOrder) .build(); if (command == DELETE) { diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index c9a94090ef89..521cb8a2287f 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -205,7 +205,8 @@ private WriterFactory createWriterFactory() { writeSchema, dsSchema, useFanoutWriter, - writeProperties); + writeProperties, + writeRequirements.icebergOrdering()); } private void commitOperation(SnapshotUpdate operation, String description) { @@ -675,6 +676,7 @@ private static class WriterFactory implements DataWriterFactory, StreamingDataWr private final boolean useFanoutWriter; private final String queryId; private final Map writeProperties; + private final org.apache.iceberg.SortOrder sortOrder; protected WriterFactory( Broadcast
    tableBroadcast, @@ -685,7 +687,8 @@ protected WriterFactory( Schema writeSchema, StructType dsSchema, boolean useFanoutWriter, - Map writeProperties) { + Map writeProperties, + org.apache.iceberg.SortOrder sortOrder) { this.tableBroadcast = tableBroadcast; this.format = format; this.outputSpecId = outputSpecId; @@ -695,6 +698,7 @@ protected WriterFactory( this.useFanoutWriter = useFanoutWriter; this.queryId = queryId; this.writeProperties = writeProperties; + this.sortOrder = sortOrder; } @Override @@ -719,6 +723,7 @@ public DataWriter createWriter(int partitionId, long taskId, long e .dataSchema(writeSchema) .dataSparkType(dsSchema) .writeProperties(writeProperties) + .dataSortOrder(sortOrder) .build(); Function rowLineageExtractor = new ExtractRowLineage(writeSchema); diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java index 89af7740d988..70230a91fc28 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkWriteBuilder.java @@ -190,7 +190,9 @@ public StreamingWrite toStreaming() { } private SparkWriteRequirements writeRequirements() { - if (overwriteFiles) { + if (rewrittenFileSetId != null) { + return writeConf.rewriteFilesWriteRequirements(); + } else if (overwriteFiles) { return writeConf.copyOnWriteRequirements(copyOnWriteCommand); } else { return writeConf.writeRequirements(); diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java index 61aacfa4589d..247119523756 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/TestSparkWriteConf.java @@ -45,6 +45,7 @@ import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.MERGE; import static org.apache.spark.sql.connector.write.RowLevelOperation.Command.UPDATE; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.time.Duration; @@ -600,6 +601,27 @@ public void testDVWriteConf() { assertThat(writeConf.deleteFileFormat()).isEqualTo(FileFormat.PUFFIN); } + @TestTemplate + public void testSortOrderWriteConf() { + Table table = validationCatalog.loadTable(tableIdent); + + table.replaceSortOrder().asc("id").commit(); + + SparkWriteConf writeConf = + new SparkWriteConf( + spark, table, ImmutableMap.of(SparkWriteOptions.OUTPUT_SORT_ORDER_ID, "1")); + + assertThat(writeConf.outputSortOrder()).isEqualTo(table.sortOrder()); + + SparkWriteConf writeConfForUnknownSortOrder = + new SparkWriteConf( + spark, table, ImmutableMap.of(SparkWriteOptions.OUTPUT_SORT_ORDER_ID, "999")); + + assertThatIllegalArgumentException() + .isThrownBy(writeConfForUnknownSortOrder::outputSortOrder) + .withMessage("Output sort order id 999 is not a valid sort order id for table"); + } + private void testWriteProperties(List> propertiesSuite) { withSQLConf( propertiesSuite.get(0), diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java index 6d965f3dcc62..5d8a6da51c4e 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteDataFilesAction.java @@ -1587,6 +1587,7 @@ public void testSimpleSort() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + dataFilesSortOrderShouldMatchTableSortOrder(table); } @TestTemplate @@ -1623,6 +1624,7 @@ public void testSortAfterPartitionChange() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + dataFilesSortOrderShouldMatchTableSortOrder(table); } @TestTemplate @@ -1654,6 +1656,9 @@ public void testSortCustomSortOrder() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + // Since the SortOrder isn't in the table spec, these data files should report having the + // default table sort order e.g. unsorted + dataFilesShouldHaveSortOrderIdMatching(table, SortOrder.unsorted()); } @TestTemplate @@ -1694,6 +1699,58 @@ public void testSortCustomSortOrderRequiresRepartition() { shouldHaveMultipleFiles(table); shouldHaveLastCommitUnsorted(table, "c2"); shouldHaveLastCommitSorted(table, "c3"); + // Since the table ordering is on C2, but we rewrote explicitly with C3 which isn't a sort order + // spec, the table files should report unsorted + dataFilesShouldHaveSortOrderIdMatching(table, SortOrder.unsorted()); + } + + @TestTemplate + public void testSortPastTableSortOrderGetsAppliedToFiles() { + int partitions = 4; + Table table = createTable(); + writeRecords(20, SCALE, partitions); + shouldHaveLastCommitUnsorted(table, "c3"); + + // Add a partition column so this requires repartitioning + table.updateSpec().addField("c1").commit(); + + // Add the sort order we want to use during the rewrite job + table.replaceSortOrder().asc("c3").commit(); + SortOrder c3SortOrder = table.sortOrder(); + + // Replace that sort order with a newer one that we aren't going to use, but is the current + // table ordering + table.replaceSortOrder().asc("c2").commit(); + shouldHaveFiles(table, 20); + + List originalData = currentData(); + long dataSizeBefore = testDataSize(table); + + RewriteDataFiles.Result result = + basicRewrite(table) + .sort(SortOrder.builderFor(table.schema()).asc("c3").build()) + .option(SizeBasedFileRewritePlanner.REWRITE_ALL, "true") + .option( + RewriteDataFiles.TARGET_FILE_SIZE_BYTES, + Integer.toString(averageFileSize(table) / partitions)) + .execute(); + + assertThat(result.rewriteResults()).as("Should have 1 fileGroups").hasSize(1); + assertThat(result.rewrittenBytesCount()).isEqualTo(dataSizeBefore); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 2); + shouldHaveACleanCache(table); + shouldHaveMultipleFiles(table); + shouldHaveLastCommitUnsorted(table, "c2"); + shouldHaveLastCommitSorted(table, "c3"); + // Since the table ordering is on C2, but we rewrote explicitly with C3 which is in the table + // sort order spec, the table files should report C3 sort order + dataFilesShouldHaveSortOrderIdMatching(table, c3SortOrder); } @TestTemplate @@ -1734,6 +1791,9 @@ public void testAutoSortShuffleOutput() { shouldHaveACleanCache(table); shouldHaveMultipleFiles(table); shouldHaveLastCommitSorted(table, "c2"); + // Since the sort order being applied here isn't anywhere on the table spec, all files despite + // being physically sorted should report unsorted in the manifest entry + dataFilesShouldHaveSortOrderIdMatching(table, SortOrder.unsorted()); } @TestTemplate @@ -2657,4 +2717,18 @@ public boolean matches(RewriteFileGroup argument) { return groupIDs.contains(argument.info().globalIndex()); } } + + private void dataFilesSortOrderShouldMatchTableSortOrder(Table table) { + dataFilesShouldHaveSortOrderIdMatching(table, table.sortOrder()); + } + + private void dataFilesShouldHaveSortOrderIdMatching(Table table, SortOrder sortOrder) { + try (CloseableIterable files = table.newScan().planFiles()) { + assertThat(files) + .extracting(fileScanTask -> fileScanTask.file().sortOrderId()) + .containsOnly(sortOrder.orderId()); + } catch (IOException e) { + throw new RuntimeException("Failed to close file scan tasks", e); + } + } } diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index 94547c2cf8fb..1c9fdd54b633 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -44,6 +44,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.CommitStateUnknownException; @@ -154,6 +155,7 @@ public void testBasicWrite() { assertThat(file.splitOffsets()).as("Split offsets not present").isNotNull(); } assertThat(file.recordCount()).as("Should have reported record count as 1").isEqualTo(1); + assertThat(file.sortOrderId()).isEqualTo(SortOrder.unsorted().orderId()); // TODO: append more metric info if (format.equals(FileFormat.PARQUET)) { assertThat(file.columnSizes()).as("Column sizes metric not present").isNotNull(); @@ -555,6 +557,55 @@ public void testViewsReturnRecentResults() { assertThat(actual2).hasSameSizeAs(expected2).isEqualTo(expected2); } + @TestTemplate + public void testWriteDataFilesInTableSortOrder() { + File parent = temp.resolve(format.toString()).toFile(); + File location = new File(parent, "test"); + String targetLocation = locationWithBranch(location); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, location.toString()); + + table.replaceSortOrder().asc("id").commit(); + + List expected = Lists.newArrayListWithCapacity(4000); + for (int i = 0; i < 4000; i++) { + expected.add(new SimpleRecord(i, "a")); + } + + Dataset df = spark.createDataFrame(expected, SimpleRecord.class); + + df.select("id", "data") + .write() + .format("iceberg") + .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) + .mode(SaveMode.Append) + .save(location.toString()); + + createBranch(table); + table.refresh(); + + Dataset result = spark.read().format("iceberg").load(targetLocation); + + List actual = + result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(actual).hasSameSizeAs(expected).isEqualTo(expected); + + List files = Lists.newArrayList(); + for (ManifestFile manifest : + SnapshotUtil.latestSnapshot(table, branch).allManifests(table.io())) { + for (DataFile file : ManifestFiles.read(manifest, table.io())) { + files.add(file); + } + } + + assertThat(files) + .extracting(DataFile::sortOrderId) + .as("All DataFiles are written with the table sort order id") + .containsOnly(table.sortOrder().orderId()); + } + public void partitionedCreateWithTargetFileSizeViaOption(IcebergOptionsType option) { File parent = temp.resolve(format.toString()).toFile(); File location = new File(parent, "test");